You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by pm...@apache.org on 2018/10/13 01:34:41 UTC

[01/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Repository: samza
Updated Branches:
  refs/heads/master 9a5094d68 -> 74675cea5


http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java
----------------------------------------------------------------------
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 79a25e7..fdf0761 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,16 +21,16 @@ package org.apache.samza.test.operator;
 
 import java.time.Duration;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.windows.Windows;
 import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.test.operator.data.PageView;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java
----------------------------------------------------------------------
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 f116f1d..6dd4303 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.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
@@ -33,9 +33,9 @@ import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.test.operator.data.PageView;
 import org.apache.samza.util.CommandLine;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java
----------------------------------------------------------------------
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 0184013..4b87169 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.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
@@ -33,9 +33,9 @@ import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.test.operator.data.PageView;
 import org.apache.samza.util.CommandLine;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java
----------------------------------------------------------------------
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 51f33b5..0991fa1 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.IOException;
 import java.io.ObjectInputStream;
 import java.io.Serializable;
 import java.util.concurrent.CountDownLatch;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
@@ -33,9 +33,9 @@ import org.apache.samza.operators.OutputStream;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.serializers.NoOpSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
----------------------------------------------------------------------
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 d24cf57..a9741b4 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,7 @@ import java.util.List;
 import java.util.Map;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.application.TaskApplication;
-import org.apache.samza.application.TaskApplicationDescriptor;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptor;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;
 import org.apache.samza.config.MapConfig;
@@ -35,14 +35,14 @@ import org.apache.samza.config.TaskConfig;
 import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
 import org.apache.samza.context.Context;
 import org.apache.samza.context.TaskContext;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
 import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.Gauge;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.Timer;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.serializers.IntegerSerde;
@@ -52,7 +52,7 @@ import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 import org.apache.samza.storage.kv.Entry;
 import org.apache.samza.storage.kv.KeyValueStore;
 import org.apache.samza.storage.kv.LocalStoreBackedReadWriteTable;
-import org.apache.samza.storage.kv.inmemory.InMemoryTableDescriptor;
+import org.apache.samza.storage.kv.inmemory.descriptors.InMemoryTableDescriptor;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.table.ReadWriteTable;
 import org.apache.samza.table.ReadableTable;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
----------------------------------------------------------------------
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 4410b87..5852de5 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
@@ -28,23 +28,23 @@ import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.StreamConfig;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.operators.KV;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
 import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
-import org.apache.samza.storage.kv.inmemory.InMemoryTableDescriptor;
-import org.apache.samza.system.kafka.KafkaSystemDescriptor;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.inmemory.descriptors.InMemoryTableDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.table.Table;
 import org.apache.samza.test.framework.TestRunner;
-import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
-import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
-import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemorySystemDescriptor;
 import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
----------------------------------------------------------------------
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 a42f2e6..98c4193 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
@@ -35,25 +35,25 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.context.Context;
 import org.apache.samza.context.TaskContext;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
 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.TableDescriptor;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 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.caching.descriptors.CachingTableDescriptor;
+import org.apache.samza.table.caching.guava.descriptors.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.descriptors.RemoteTableDescriptor;
 import org.apache.samza.table.remote.TableRateLimiter;
 import org.apache.samza.table.remote.TableReadFunction;
 import org.apache.samza.table.remote.TableWriteFunction;
@@ -212,7 +212,7 @@ public class TestRemoteTable extends AbstractIntegrationTestHarness {
           : appDesc.getTable(inputTableDesc);
 
       DelegatingSystemDescriptor ksd = new DelegatingSystemDescriptor("test");
-      GenericInputDescriptor<TestTableData.PageView> isd = ksd.getInputDescriptor("PageView", new NoOpSerde<>());
+      GenericInputDescriptor<PageView> isd = ksd.getInputDescriptor("PageView", new NoOpSerde<>());
       appDesc.getInputStream(isd)
           .map(pv -> new KV<>(pv.getMemberId(), pv))
           .join(inputTable, new PageViewToProfileJoinFunction())

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java
----------------------------------------------------------------------
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 f868fdc..f4c05e4 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
@@ -32,17 +32,17 @@ 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.table.descriptors.TableDescriptor;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.LongSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
-import org.apache.samza.storage.kv.RocksDbTableProviderFactory;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableProviderFactory;
 import org.apache.samza.table.TableConfigGenerator;
 import org.apache.samza.table.TableDescriptorsProvider;
-import org.apache.samza.table.remote.RemoteTableDescriptor;
-import org.apache.samza.table.remote.RemoteTableProviderFactory;
+import org.apache.samza.table.remote.descriptors.RemoteTableDescriptor;
+import org.apache.samza.table.remote.descriptors.RemoteTableProviderFactory;
 import org.apache.samza.table.remote.TableReadFunction;
 import org.apache.samza.util.RateLimiter;
 import org.apache.samza.util.Util;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java
----------------------------------------------------------------------
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 f7805fe..c9f29eb 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
@@ -36,9 +36,9 @@ import org.apache.samza.config.JobCoordinatorConfig;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.SystemConfig;
 import org.apache.samza.config.TaskConfig;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
-import org.apache.samza.operators.descriptors.GenericSystemDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.runtime.ApplicationRunners;


[11/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SimpleInputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SimpleInputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SimpleInputDescriptorProvider.java
deleted file mode 100644
index de40932..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SimpleInputDescriptorProvider.java
+++ /dev/null
@@ -1,43 +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.descriptors.base.system;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.serializers.Serde;
-
-
-/**
- * Interface for simple {@code SystemDescriptors} that return {@code InputDescriptors} parameterized by the type of
- * the provided stream level serde.
- */
-public interface SimpleInputDescriptorProvider {
-
-  /**
-   * Gets an {@link InputDescriptor} for an input stream on this system. The stream has the provided
-   * stream level serde.
-   * <p>
-   * The type of messages in the stream is the type of the provided stream level serde.
-   *
-   * @param streamId id of the input stream
-   * @param serde stream level serde for the input stream
-   * @param <StreamMessageType> type of messages in this stream
-   * @return an {@link InputDescriptor} for the input stream
-   */
-  <StreamMessageType> InputDescriptor<StreamMessageType, ? extends InputDescriptor> getInputDescriptor(String streamId, Serde<StreamMessageType> serde);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SystemDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SystemDescriptor.java
deleted file mode 100644
index 4bb121d..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/SystemDescriptor.java
+++ /dev/null
@@ -1,177 +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.descriptors.base.system;
-
-import com.google.common.base.Preconditions;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.regex.Pattern;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.operators.functions.StreamExpander;
-import org.apache.samza.system.SystemStreamMetadata.OffsetType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The base descriptor for a system. Allows setting properties that are common to all systems.
- * <p>
- * System properties configured using a descriptor override corresponding properties provided in configuration.
- * <p>
- * Systems may provide an {@link InputTransformer} to be used for input streams on the system. An
- * {@link InputTransformer} transforms an {@code IncomingMessageEnvelope} with deserialized key and message
- * to another message that is delivered to the {@code MessageStream}. It is applied at runtime in
- * {@code InputOperatorImpl}.
- * <p>
- * Systems may provide a {@link StreamExpander} to be used for input streams on the system. A {@link StreamExpander}
- * expands the provided {@code InputDescriptor} to a sub-DAG of one or more operators on the {@code StreamGraph},
- * and returns a new {@code MessageStream} with the combined results. It is called during graph description
- * in {@code StreamGraph#getInputStream}.
- * <p>
- * Systems that support consuming messages from a stream should provide users means of obtaining an
- * {@code InputDescriptor}. Recommended interfaces for doing so are {@link TransformingInputDescriptorProvider} for
- * systems that support system level {@link InputTransformer}, {@link ExpandingInputDescriptorProvider} for systems
- * that support system level {@link StreamExpander} functions, and {@link SimpleInputDescriptorProvider} otherwise.
- * <p>
- * Systems that support producing messages to a stream should provide users means of obtaining an
- * {@code OutputDescriptor}. Recommended interface for doing so is {@link OutputDescriptorProvider}.
- * <p>
- * It is not required for SystemDescriptors to implement one of the Provider interfaces above. System implementers
- * may choose to expose additional or alternate APIs for obtaining Input/Output Descriptors by extending
- * SystemDescriptor directly.
- *
- * @param <SubClass> type of the concrete sub-class
- */
-public abstract class SystemDescriptor<SubClass extends SystemDescriptor<SubClass>> {
-  private static final Logger LOGGER = LoggerFactory.getLogger(SystemDescriptor.class);
-  private static final String FACTORY_CONFIG_KEY = "systems.%s.samza.factory";
-  private static final String DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY = "systems.%s.default.stream.samza.offset.default";
-  private static final String DEFAULT_STREAM_CONFIGS_CONFIG_KEY = "systems.%s.default.stream.%s";
-  private static final String SYSTEM_CONFIGS_CONFIG_KEY = "systems.%s.%s";
-  private static final Pattern SYSTEM_NAME_PATTERN = Pattern.compile("[\\d\\w-_]+");
-
-  private final String systemName;
-  private final Optional<String> factoryClassNameOptional;
-  private final Optional<InputTransformer> transformerOptional;
-  private final Optional<StreamExpander> expanderOptional;
-
-  private final Map<String, String> systemConfigs = new HashMap<>();
-  private final Map<String, String> defaultStreamConfigs = new HashMap<>();
-  private Optional<OffsetType> defaultStreamOffsetDefaultOptional = Optional.empty();
-
-  /**
-   * Constructs a {@link SystemDescriptor} instance.
-   *
-   * @param systemName name of this system
-   * @param factoryClassName name of the SystemFactory class for this system
-   * @param transformer the {@link InputTransformer} for the system if any, else null
-   * @param expander the {@link StreamExpander} for the system if any, else null
-   */
-  public SystemDescriptor(String systemName, String factoryClassName, InputTransformer transformer, StreamExpander expander) {
-    Preconditions.checkArgument(isValidSystemName(systemName),
-        String.format("systemName: %s must be non-empty and must not contain spaces or special characters.", systemName));
-    if (StringUtils.isBlank(factoryClassName)) {
-      LOGGER.warn("Blank SystemFactory class name for system: {}. A value must be provided in configuration using {}.",
-          systemName, String.format(FACTORY_CONFIG_KEY, systemName));
-    }
-    this.systemName = systemName;
-    this.factoryClassNameOptional = Optional.ofNullable(StringUtils.stripToNull(factoryClassName));
-    this.transformerOptional = Optional.ofNullable(transformer);
-    this.expanderOptional = Optional.ofNullable(expander);
-  }
-
-  /**
-   * If a container starts up without a checkpoint, this property determines where in the input stream we should start
-   * consuming. The value must be an {@link OffsetType}, one of the following:
-   * <ul>
-   *  <li>upcoming: Start processing messages that are published after the job starts.
-   *                Any messages published while the job was not running are not processed.
-   *  <li>oldest: Start processing at the oldest available message in the system,
-   *              and reprocess the entire available message history.
-   * </ul>
-   * This property is for all streams obtained using this system descriptor. To set it for an individual stream,
-   * see {@link org.apache.samza.operators.descriptors.base.stream.InputDescriptor#withOffsetDefault}.
-   * If both are defined, the stream-level definition takes precedence.
-   *
-   * @param offsetType offset type to start processing from
-   * @return this system descriptor
-   */
-  public SubClass withDefaultStreamOffsetDefault(OffsetType offsetType) {
-    this.defaultStreamOffsetDefaultOptional = Optional.ofNullable(offsetType);
-    return (SubClass) this;
-  }
-
-  /**
-   * Additional system-specific properties for this system.
-   * <p>
-   * These properties are added under the {@code systems.system-name.*} scope.
-   *
-   * @param systemConfigs system-specific properties for this system
-   * @return this system descriptor
-   */
-  public SubClass withSystemConfigs(Map<String, String> systemConfigs) {
-    this.systemConfigs.putAll(systemConfigs);
-    return (SubClass) this;
-  }
-
-  /**
-   * Default properties for any stream obtained using this system descriptor.
-   * <p>
-   * For example, if "systems.kafka-system.default.stream.replication.factor"=2 was configured,
-   * then every Kafka stream created on the kafka-system will have a replication factor of 2
-   * unless the property is explicitly overridden using the stream descriptor.
-   *
-   * @param defaultStreamConfigs default stream properties
-   * @return this system descriptor
-   */
-  public SubClass withDefaultStreamConfigs(Map<String, String> defaultStreamConfigs) {
-    this.defaultStreamConfigs.putAll(defaultStreamConfigs);
-    return (SubClass) this;
-  }
-
-  public String getSystemName() {
-    return this.systemName;
-  }
-
-  public Optional<InputTransformer> getTransformer() {
-    return this.transformerOptional;
-  }
-
-  public Optional<StreamExpander> getExpander() {
-    return this.expanderOptional;
-  }
-
-  private boolean isValidSystemName(String id) {
-    return StringUtils.isNotBlank(id) && SYSTEM_NAME_PATTERN.matcher(id).matches();
-  }
-
-  public Map<String, String> toConfig() {
-    HashMap<String, String> configs = new HashMap<>();
-    this.factoryClassNameOptional.ifPresent(name -> configs.put(String.format(FACTORY_CONFIG_KEY, systemName), name));
-    this.defaultStreamOffsetDefaultOptional.ifPresent(dsod ->
-        configs.put(String.format(DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY, systemName), dsod.name().toLowerCase()));
-    this.defaultStreamConfigs.forEach((key, value) ->
-        configs.put(String.format(DEFAULT_STREAM_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
-    this.systemConfigs.forEach((key, value) ->
-        configs.put(String.format(SYSTEM_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
-    return configs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/TransformingInputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/TransformingInputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/TransformingInputDescriptorProvider.java
deleted file mode 100644
index 5b43fbd..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/TransformingInputDescriptorProvider.java
+++ /dev/null
@@ -1,44 +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.descriptors.base.system;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * Interface for advanced {@code SystemDescriptor}s that constrain the type of returned {@code InputDescriptor}s to
- * their own {@code InputTransformer} function result types.
- *
- * @param <InputTransformerType> type of the system level {@code InputTransformer} results
- */
-public interface TransformingInputDescriptorProvider<InputTransformerType> {
-
-  /**
-   * Gets a {@link InputDescriptor} for an input stream on this system. The stream has the provided
-   * stream level serde, and the default system level {@code InputTransformer}.
-   * <p>
-   * The type of messages in the stream is the type of messages returned by the default system level
-   * {@code InputTransformer}
-   *
-   * @param streamId id of the input stream
-   * @param serde stream level serde for the input stream
-   * @return an {@link InputDescriptor} for the input stream
-   */
-  InputDescriptor<InputTransformerType, ? extends InputDescriptor> getInputDescriptor(String streamId, Serde serde);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java
----------------------------------------------------------------------
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 12823cc..d3c1a23 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
@@ -24,12 +24,12 @@ import org.apache.samza.annotation.InterfaceStability;
 
 /**
  * A function that can be closed after its execution.
- *
- * <p> Implement {@link #close()} to free resources used during the execution of the function, clean up state etc.
- *
- * <p> Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the
- * {@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.
+ * <p>
+ * Implement {@link #close()} to free resources used during the execution of the function, clean up state etc.
+ * <p>
+ * Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the
+ * {@link org.apache.samza.application.descriptors.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.
  *
  */
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
----------------------------------------------------------------------
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 7f950de..c808876 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
@@ -24,10 +24,11 @@ import org.apache.samza.context.Context;
 
 /**
  * A function that can be initialized before execution.
- *
- * <p> Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the
- * {@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.
+ * <p>
+ * Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the
+ * {@link org.apache.samza.application.descriptors.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.
  *
  */
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/functions/InputTransformer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InputTransformer.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InputTransformer.java
deleted file mode 100644
index 704d187..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/InputTransformer.java
+++ /dev/null
@@ -1,45 +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.functions;
-
-import java.io.Serializable;
-import org.apache.samza.system.IncomingMessageEnvelope;
-
-/**
- * Transforms an {@link IncomingMessageEnvelope} with deserialized key and message to a message of type {@code OM}
- * which is delivered to the {@code MessageStream}. Called in {@code InputOperatorImpl} when incoming messages
- * from a {@code SystemConsumer} are being delivered to the application.
- * <p>
- * This is provided by default by transforming system descriptor implementations and can not be overridden
- * or set on a per stream level.
- *
- * @param <OM> type of the transformed message
- */
-public interface InputTransformer<OM> extends InitableFunction, ClosableFunction, Serializable {
-
-  /**
-   * Transforms the provided {@link IncomingMessageEnvelope} with deserialized key and message into another message
-   * which is delivered to the {@code MessageStream}.
-   *
-   * @param ime  the {@link IncomingMessageEnvelope} to be transformed
-   * @return  the transformed message
-   */
-  OM apply(IncomingMessageEnvelope ime);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/functions/StreamExpander.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 7bbf601..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/StreamExpander.java
+++ /dev/null
@@ -1,58 +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.functions;
-
-import java.io.Serializable;
-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 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.
- * <p>
- * This is provided by default by expanding system descriptor implementations and can not be overridden
- * or set on a per stream level.
- *
- * @param <OM> type of the messages in the resultant {@link MessageStream}
- */
-public interface StreamExpander<OM> extends Serializable {
-
-  /**
-   * 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 StreamApplicationDescriptor#getInputStream}.
-   * <p>
-   * Notes for system implementers:
-   * <p>
-   * Take care to avoid infinite recursion in the implementation; e.g., by ensuring that it doesn't call
-   * {@link StreamApplicationDescriptor#getInputStream} with an {@link InputDescriptor} from an expanding system descriptor
-   * (like this one) again.
-   * <p>
-   * 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 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<OM> apply(StreamApplicationDescriptor streamAppDesc, InputDescriptor inputDescriptor);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/ExpandingInputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/ExpandingInputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/ExpandingInputDescriptorProvider.java
new file mode 100644
index 0000000..d180e1e
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/ExpandingInputDescriptorProvider.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * Interface for advanced {@code SystemDescriptor}s that constrain the type of returned {@code InputDescriptor}s to
+ * their own {@code StreamExpander} function result types.
+ *
+ * @param <StreamExpanderType> type of the system level {@code StreamExpander} results
+ */
+public interface ExpandingInputDescriptorProvider<StreamExpanderType> {
+
+  /**
+   * Gets a {@link InputDescriptor} for an input stream on this system. The stream has the provided
+   * stream level serde, and the default system level {@code StreamExpander}
+   * <p>
+   * The type of messages in the stream is the type of messages returned by the default system level
+   * {@code StreamExpander}
+   *
+   * @param streamId id of the input stream
+   * @param serde stream level serde to be propagated to expanded input streams
+   * @return an {@link InputDescriptor} for the input stream
+   */
+  InputDescriptor<StreamExpanderType, ? extends InputDescriptor> getInputDescriptor(String streamId, Serde serde);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java
new file mode 100644
index 0000000..6d18afc
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * A descriptor for a generic input stream.
+ * <p>
+ * An instance of this descriptor may be obtained from an appropriately configured
+ * {@link GenericSystemDescriptor}.
+ * <p>
+ * If the system being used provides its own system and stream descriptor implementations, they should be used instead.
+ * Otherwise, this {@link GenericInputDescriptor} may be used to provide Samza-specific properties of the input stream.
+ * Additional system stream specific properties may be provided using {@link #withStreamConfigs}
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ */
+public final class GenericInputDescriptor<StreamMessageType>
+    extends InputDescriptor<StreamMessageType, GenericInputDescriptor<StreamMessageType>> {
+  GenericInputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde) {
+    super(streamId, serde, systemDescriptor, null);
+  }
+
+  @Override
+  public GenericInputDescriptor<StreamMessageType> withPhysicalName(String physicalName) {
+    return super.withPhysicalName(physicalName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java
new file mode 100644
index 0000000..5fe3a98
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * A descriptor for a generic output stream.
+ * <p>
+ * An instance of this descriptor may be obtained from an appropriately configured
+ * {@link GenericSystemDescriptor}.
+ * <p>
+ * If the system being used provides its own system and stream descriptor implementations, they should be used instead.
+ * Otherwise, this {@link GenericOutputDescriptor} may be used to provide Samza-specific properties of the output stream.
+ * Additional system stream specific properties may be provided using {@link #withStreamConfigs}
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ */
+public final class GenericOutputDescriptor<StreamMessageType>
+    extends OutputDescriptor<StreamMessageType, GenericOutputDescriptor<StreamMessageType>> {
+  GenericOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde) {
+    super(streamId, serde, systemDescriptor);
+  }
+
+  @Override
+  public GenericOutputDescriptor<StreamMessageType> withPhysicalName(String physicalName) {
+    return super.withPhysicalName(physicalName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java
new file mode 100644
index 0000000..59b2a12
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * A descriptor for a generic system.
+ * <p>
+ * If the system being used provides its own system and stream descriptor implementations, they should be used instead.
+ * Otherwise, this {@link GenericSystemDescriptor} may be used to provide Samza-specific properties of the system.
+ * Additional system specific properties may be provided using {@link #withSystemConfigs}
+ * <p>
+ * System properties configured using a descriptor override corresponding properties provided in configuration.
+ */
+public final class GenericSystemDescriptor extends SystemDescriptor<GenericSystemDescriptor>
+    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
+
+  /**
+   * Constructs a {@link GenericSystemDescriptor} instance with no system level serde.
+   * Serdes must be provided explicitly at stream level when getting input or output descriptors.
+   *
+   * @param systemName name of this system
+   * @param factoryClassName name of the SystemFactory class for this system
+   */
+  public GenericSystemDescriptor(String systemName, String factoryClassName) {
+    super(systemName, factoryClassName, null, null);
+  }
+
+  @Override
+  public <StreamMessageType> GenericInputDescriptor<StreamMessageType> getInputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new GenericInputDescriptor<>(streamId, this, serde);
+  }
+
+  @Override
+  public <StreamMessageType> GenericOutputDescriptor<StreamMessageType> getOutputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new GenericOutputDescriptor<>(streamId, this, serde);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java
new file mode 100644
index 0000000..2c0ca88
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.SystemStreamMetadata.OffsetType;
+
+/**
+ * The base descriptor for an input stream. Allows setting properties that are common to all input streams.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ * @param <SubClass> type of the concrete sub-class
+ */
+public abstract class InputDescriptor<StreamMessageType, SubClass extends InputDescriptor<StreamMessageType, SubClass>>
+    extends StreamDescriptor<StreamMessageType, SubClass> {
+  private static final String RESET_OFFSET_CONFIG_KEY = "streams.%s.samza.reset.offset";
+  private static final String OFFSET_DEFAULT_CONFIG_KEY = "streams.%s.samza.offset.default";
+  private static final String PRIORITY_CONFIG_KEY = "streams.%s.samza.priority";
+  private static final String BOOTSTRAP_CONFIG_KEY = "streams.%s.samza.bootstrap";
+  private static final String BOUNDED_CONFIG_KEY = "streams.%s.samza.bounded";
+  private static final String DELETE_COMMITTED_MESSAGES_CONFIG_KEY = "streams.%s.samza.delete.committed.messages";
+
+  private final Optional<InputTransformer> transformerOptional;
+
+  private Optional<Boolean> resetOffsetOptional = Optional.empty();
+  private Optional<OffsetType> offsetDefaultOptional = Optional.empty();
+  private Optional<Integer> priorityOptional = Optional.empty();
+  private Optional<Boolean> isBootstrapOptional = Optional.empty();
+  private Optional<Boolean> isBoundedOptional = Optional.empty();
+  private Optional<Boolean> deleteCommittedMessagesOptional = Optional.empty();
+
+  /**
+   * Constructs an {@link InputDescriptor} instance.
+   *
+   * @param streamId id of the stream
+   * @param serde serde for messages in the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   * @param transformer stream level input stream transform function if available, else null
+   */
+  public InputDescriptor(String streamId, Serde serde, SystemDescriptor systemDescriptor, InputTransformer transformer) {
+    super(streamId, serde, systemDescriptor);
+
+    // stream level transformer takes precedence over system level transformer
+    if (transformer != null) {
+      this.transformerOptional = Optional.of(transformer);
+    } else {
+      this.transformerOptional = systemDescriptor.getTransformer();
+    }
+  }
+
+  /**
+   * If set, when a Samza container starts up, it ignores any checkpointed offset for this particular
+   * input stream. Its behavior is thus determined by the {@link #withOffsetDefault} setting.
+   * Note that the reset takes effect every time a container is started, which may be every time you restart your job,
+   * or more frequently if a container fails and is restarted by the framework.
+   *
+   * @return this input descriptor
+   */
+  public SubClass shouldResetOffset() {
+    this.resetOffsetOptional = Optional.of(true);
+    return (SubClass) this;
+  }
+
+  /**
+   * If a container starts up without a checkpoint, this property determines where in the input stream we should start
+   * consuming. The value must be an OffsetType, one of the following:
+   * <ul>
+   *  <li>upcoming: Start processing messages that are published after the job starts.
+   *                Any messages published while the job was not running are not processed.
+   *  <li>oldest: Start processing at the oldest available message in the system,
+   *              and reprocess the entire available message history.
+   * </ul>
+   * This property is for an individual stream. To set it for all streams within a system, see
+   * {@link SystemDescriptor#withDefaultStreamOffsetDefault}. If both are defined, the stream-level definition
+   * takes precedence.
+   *
+   * @param offsetDefault offset type to start processing from
+   * @return this input descriptor
+   */
+  public SubClass withOffsetDefault(OffsetType offsetDefault) {
+    this.offsetDefaultOptional = Optional.ofNullable(offsetDefault);
+    return (SubClass) this;
+  }
+
+  /**
+   * If one or more streams have a priority set (any positive integer), they will be processed with higher priority
+   * than the other streams.
+   * <p>
+   * You can set several streams to the same priority, or define multiple priority levels by assigning a
+   * higher number to the higher-priority streams.
+   * <p>
+   * If a higher-priority stream has any messages available, they will always be processed first;
+   * messages from lower-priority streams are only processed when there are no new messages on higher-priority inputs.
+   *
+   * @param priority priority for this input stream
+   * @return this input descriptor
+   */
+  public SubClass withPriority(int priority) {
+    this.priorityOptional = Optional.of(priority);
+    return (SubClass) this;
+  }
+
+  /**
+   * If set, this stream will be processed as a bootstrap stream. This means that every time a Samza container
+   * starts up, this stream will be fully consumed before messages from any other stream are processed.
+   *
+   * @return this input descriptor
+   */
+  public SubClass shouldBootstrap() {
+    this.isBootstrapOptional = Optional.of(true);
+    return (SubClass) this;
+  }
+
+  /**
+   * If set, this stream will be considered a bounded stream. If all input streams in an application are
+   * bounded, the job is considered to be running in batch processing mode.
+   *
+   * @return this input descriptor
+   */
+  public SubClass isBounded() {
+    this.isBoundedOptional = Optional.of(true);
+    return (SubClass) this;
+  }
+
+  /**
+   * If set, and supported by the system implementation, messages older than the latest checkpointed offset
+   * for this stream may be deleted after the commit.
+   *
+   * @return this input descriptor
+   */
+  public SubClass shouldDeleteCommittedMessages() {
+    this.deleteCommittedMessagesOptional = Optional.of(true);
+    return (SubClass) this;
+  }
+
+  public Optional<InputTransformer> getTransformer() {
+    return this.transformerOptional;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    HashMap<String, String> configs = new HashMap<>(super.toConfig());
+    String streamId = getStreamId();
+    this.offsetDefaultOptional.ifPresent(od ->
+        configs.put(String.format(OFFSET_DEFAULT_CONFIG_KEY, streamId), od.name().toLowerCase()));
+    this.resetOffsetOptional.ifPresent(resetOffset ->
+        configs.put(String.format(RESET_OFFSET_CONFIG_KEY, streamId), Boolean.toString(resetOffset)));
+    this.priorityOptional.ifPresent(priority ->
+        configs.put(String.format(PRIORITY_CONFIG_KEY, streamId), Integer.toString(priority)));
+    this.isBootstrapOptional.ifPresent(bootstrap ->
+        configs.put(String.format(BOOTSTRAP_CONFIG_KEY, streamId), Boolean.toString(bootstrap)));
+    this.isBoundedOptional.ifPresent(bounded ->
+        configs.put(String.format(BOUNDED_CONFIG_KEY, streamId), Boolean.toString(bounded)));
+    this.deleteCommittedMessagesOptional.ifPresent(deleteCommittedMessages ->
+        configs.put(String.format(DELETE_COMMITTED_MESSAGES_CONFIG_KEY, streamId),
+            Boolean.toString(deleteCommittedMessages)));
+    return Collections.unmodifiableMap(configs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/InputTransformer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/InputTransformer.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/InputTransformer.java
new file mode 100644
index 0000000..4435f06
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/InputTransformer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import java.io.Serializable;
+import org.apache.samza.operators.functions.ClosableFunction;
+import org.apache.samza.operators.functions.InitableFunction;
+import org.apache.samza.system.IncomingMessageEnvelope;
+
+/**
+ * Transforms an {@link IncomingMessageEnvelope} with deserialized key and message to a message of type {@code OM}
+ * which is delivered to the {@code MessageStream}. Called in {@code InputOperatorImpl} when incoming messages
+ * from a {@code SystemConsumer} are being delivered to the application.
+ * <p>
+ * This is provided by default by transforming system descriptor implementations and can not be overridden
+ * or set on a per stream level.
+ *
+ * @param <OM> type of the transformed message
+ */
+public interface InputTransformer<OM> extends InitableFunction, ClosableFunction, Serializable {
+
+  /**
+   * Transforms the provided {@link IncomingMessageEnvelope} with deserialized key and message into another message
+   * which is delivered to the {@code MessageStream}.
+   *
+   * @param ime  the {@link IncomingMessageEnvelope} to be transformed
+   * @return  the transformed message
+   */
+  OM apply(IncomingMessageEnvelope ime);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java
new file mode 100644
index 0000000..70a5d0f
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * The base descriptor for an output stream. Allows setting properties that are common to all output streams.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ * @param <SubClass> type of the concrete sub-class
+ */
+public abstract class OutputDescriptor<StreamMessageType, SubClass extends OutputDescriptor<StreamMessageType, SubClass>>
+    extends StreamDescriptor<StreamMessageType, SubClass> {
+  /**
+   * Constructs an {@link OutputDescriptor} instance.
+   *
+   * @param streamId id of the stream
+   * @param serde serde for messages in the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   */
+  public OutputDescriptor(String streamId, Serde serde, SystemDescriptor systemDescriptor) {
+    super(streamId, serde, systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptorProvider.java
new file mode 100644
index 0000000..2ebfe79
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptorProvider.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+
+/**
+ * Interface for simple {@code SystemDescriptors} that return {@code OutputDescriptors} parameterized by the type of
+ * the provided stream level serde.
+ */
+public interface OutputDescriptorProvider {
+
+  /**
+   * Gets an {@link OutputDescriptor} representing an output stream on this system that uses the provided
+   * stream specific serde instead of the default system serde.
+   * <p>
+   * An {@code OutputStream<KV<K, V>>}, obtained using a descriptor with a {@code KVSerde<K, V>}, can send messages
+   * of type {@code KV<K, V>}. An {@code OutputStream<M>} with any other {@code Serde<M>} can send messages of
+   * type M without a key.
+   * <p>
+   * A {@code KVSerde<NoOpSerde, NoOpSerde>} or {@code NoOpSerde} may be used if the {@code SystemProducer}
+   * serializes the outgoing messages itself, and no prior serialization is required from the framework.
+   *
+   * @param streamId id of the output stream
+   * @param serde serde for this output stream that overrides the default system serde, if any.
+   * @param <StreamMessageType> type of messages in the output stream
+   * @return the {@link OutputDescriptor} for the output stream
+   */
+  <StreamMessageType> OutputDescriptor<StreamMessageType, ? extends OutputDescriptor> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/SimpleInputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/SimpleInputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/SimpleInputDescriptorProvider.java
new file mode 100644
index 0000000..ed1c6a4
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/SimpleInputDescriptorProvider.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.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+
+/**
+ * Interface for simple {@code SystemDescriptors} that return {@code InputDescriptors} parameterized by the type of
+ * the provided stream level serde.
+ */
+public interface SimpleInputDescriptorProvider {
+
+  /**
+   * Gets an {@link InputDescriptor} for an input stream on this system. The stream has the provided
+   * stream level serde.
+   * <p>
+   * The type of messages in the stream is the type of the provided stream level serde.
+   *
+   * @param streamId id of the input stream
+   * @param serde stream level serde for the input stream
+   * @param <StreamMessageType> type of messages in this stream
+   * @return an {@link InputDescriptor} for the input stream
+   */
+  <StreamMessageType> InputDescriptor<StreamMessageType, ? extends InputDescriptor> getInputDescriptor(String streamId, Serde<StreamMessageType> serde);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java
new file mode 100644
index 0000000..d2b25f9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.serializers.Serde;
+
+/**
+ * The base descriptor for an input or output stream. Allows setting properties that are common to all streams.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ * @param <SubClass> type of the concrete sub-class
+ */
+public abstract class StreamDescriptor<StreamMessageType, SubClass extends StreamDescriptor<StreamMessageType, SubClass>> {
+  private static final String SYSTEM_CONFIG_KEY = "streams.%s.samza.system";
+  private static final String PHYSICAL_NAME_CONFIG_KEY = "streams.%s.samza.physical.name";
+  private static final String STREAM_CONFIGS_CONFIG_KEY = "streams.%s.%s";
+  private static final Pattern STREAM_ID_PATTERN = Pattern.compile("[\\d\\w-_]+");
+
+  private final String streamId;
+  private final Serde serde;
+  private final SystemDescriptor systemDescriptor;
+
+  private final Map<String, String> streamConfigs = new HashMap<>();
+  private Optional<String> physicalNameOptional = Optional.empty();
+
+  /**
+   * Constructs a {@link StreamDescriptor} instance.
+   *
+   * @param streamId id of the stream
+   * @param serde serde for messages in the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   */
+  StreamDescriptor(String streamId, Serde serde, SystemDescriptor systemDescriptor) {
+    Preconditions.checkArgument(systemDescriptor != null,
+        String.format("SystemDescriptor must not be null. streamId: %s", streamId));
+    String systemName = systemDescriptor.getSystemName();
+    Preconditions.checkState(isValidStreamId(streamId),
+        String.format("streamId must be non-empty and must not contain spaces or special characters. " +
+            "streamId: %s, systemName: %s", streamId, systemName));
+    Preconditions.checkArgument(serde != null,
+        String.format("Serde must not be null. streamId: %s systemName: %s", streamId, systemName));
+    this.streamId = streamId;
+    this.serde = serde;
+    this.systemDescriptor = systemDescriptor;
+  }
+
+  /**
+   * The physical name of the stream on the system on which this stream will be accessed.
+   * This is opposed to the {@code streamId} which is the logical name that Samza uses to identify the stream.
+   * <p>
+   * A physical name could be a Kafka topic name, an HDFS file URN, or any other system-specific identifier.
+   * <p>
+   * If not provided, the logical {@code streamId} is used as the physical name.
+   *
+   * @param physicalName physical name for this stream.
+   * @return this stream descriptor.
+   */
+  protected SubClass withPhysicalName(String physicalName) {
+    this.physicalNameOptional = Optional.ofNullable(physicalName);
+    return (SubClass) this;
+  }
+
+  /**
+   * Additional system-specific properties for this stream.
+   * <p>
+   * These properties are added under the {@code streams.stream-id.*} scope.
+   *
+   * @param streamConfigs system-specific properties for this stream
+   * @return this stream descriptor
+   */
+  public SubClass withStreamConfigs(Map<String, String> streamConfigs) {
+    this.streamConfigs.putAll(streamConfigs);
+    return (SubClass) this;
+  }
+
+  public String getStreamId() {
+    return this.streamId;
+  }
+
+  public String getSystemName() {
+    return this.systemDescriptor.getSystemName();
+  }
+
+  public Serde getSerde() {
+    return this.serde;
+  }
+
+  public SystemDescriptor getSystemDescriptor() {
+    return this.systemDescriptor;
+  }
+
+  public Optional<String> getPhysicalName() {
+    return physicalNameOptional;
+  }
+
+  private boolean isValidStreamId(String id) {
+    return StringUtils.isNotBlank(id) && STREAM_ID_PATTERN.matcher(id).matches();
+  }
+
+  public Map<String, String> toConfig() {
+    HashMap<String, String> configs = new HashMap<>();
+    configs.put(String.format(SYSTEM_CONFIG_KEY, streamId), getSystemName());
+    this.physicalNameOptional.ifPresent(physicalName ->
+        configs.put(String.format(PHYSICAL_NAME_CONFIG_KEY, streamId), physicalName));
+    this.streamConfigs.forEach((key, value) ->
+        configs.put(String.format(STREAM_CONFIGS_CONFIG_KEY, streamId, key), value));
+    return Collections.unmodifiableMap(configs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamExpander.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamExpander.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamExpander.java
new file mode 100644
index 0000000..4b8e6f0
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamExpander.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import java.io.Serializable;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
+import org.apache.samza.operators.MessageStream;
+
+/**
+ * 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.
+ * <p>
+ * This is provided by default by expanding system descriptor implementations and can not be overridden
+ * or set on a per stream level.
+ *
+ * @param <OM> type of the messages in the resultant {@link MessageStream}
+ */
+public interface StreamExpander<OM> extends Serializable {
+
+  /**
+   * 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 StreamApplicationDescriptor#getInputStream}.
+   * <p>
+   * Notes for system implementers:
+   * <p>
+   * Take care to avoid infinite recursion in the implementation; e.g., by ensuring that it doesn't call
+   * {@link StreamApplicationDescriptor#getInputStream} with an {@link InputDescriptor} from an expanding system descriptor
+   * (like this one) again.
+   * <p>
+   * 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 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<OM> apply(StreamApplicationDescriptor streamAppDesc, InputDescriptor inputDescriptor);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java
new file mode 100644
index 0000000..4b93a32
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import com.google.common.base.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.system.SystemStreamMetadata.OffsetType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The base descriptor for a system. Allows setting properties that are common to all systems.
+ * <p>
+ * System properties configured using a descriptor override corresponding properties provided in configuration.
+ * <p>
+ * Systems may provide an {@link InputTransformer} to be used for input streams on the system. An
+ * {@link InputTransformer} transforms an {@code IncomingMessageEnvelope} with deserialized key and message
+ * to another message that is delivered to the {@code MessageStream}. It is applied at runtime in
+ * {@code InputOperatorImpl}.
+ * <p>
+ * Systems may provide a {@link StreamExpander} to be used for input streams on the system. A {@link StreamExpander}
+ * expands the provided {@code InputDescriptor} to a sub-DAG of one or more operators on the {@code StreamGraph},
+ * and returns a new {@code MessageStream} with the combined results. It is called during graph description
+ * in {@code StreamGraph#getInputStream}.
+ * <p>
+ * Systems that support consuming messages from a stream should provide users means of obtaining an
+ * {@code InputDescriptor}. Recommended interfaces for doing so are {@link TransformingInputDescriptorProvider} for
+ * systems that support system level {@link InputTransformer}, {@link ExpandingInputDescriptorProvider} for systems
+ * that support system level {@link StreamExpander} functions, and {@link SimpleInputDescriptorProvider} otherwise.
+ * <p>
+ * Systems that support producing messages to a stream should provide users means of obtaining an
+ * {@code OutputDescriptor}. Recommended interface for doing so is {@link OutputDescriptorProvider}.
+ * <p>
+ * It is not required for SystemDescriptors to implement one of the Provider interfaces above. System implementers
+ * may choose to expose additional or alternate APIs for obtaining Input/Output Descriptors by extending
+ * SystemDescriptor directly.
+ *
+ * @param <SubClass> type of the concrete sub-class
+ */
+public abstract class SystemDescriptor<SubClass extends SystemDescriptor<SubClass>> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SystemDescriptor.class);
+  private static final String FACTORY_CONFIG_KEY = "systems.%s.samza.factory";
+  private static final String DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY = "systems.%s.default.stream.samza.offset.default";
+  private static final String DEFAULT_STREAM_CONFIGS_CONFIG_KEY = "systems.%s.default.stream.%s";
+  private static final String SYSTEM_CONFIGS_CONFIG_KEY = "systems.%s.%s";
+  private static final Pattern SYSTEM_NAME_PATTERN = Pattern.compile("[\\d\\w-_]+");
+
+  private final String systemName;
+  private final Optional<String> factoryClassNameOptional;
+  private final Optional<InputTransformer> transformerOptional;
+  private final Optional<StreamExpander> expanderOptional;
+
+  private final Map<String, String> systemConfigs = new HashMap<>();
+  private final Map<String, String> defaultStreamConfigs = new HashMap<>();
+  private Optional<OffsetType> defaultStreamOffsetDefaultOptional = Optional.empty();
+
+  /**
+   * Constructs a {@link SystemDescriptor} instance.
+   *
+   * @param systemName name of this system
+   * @param factoryClassName name of the SystemFactory class for this system
+   * @param transformer the {@link InputTransformer} for the system if any, else null
+   * @param expander the {@link StreamExpander} for the system if any, else null
+   */
+  public SystemDescriptor(String systemName, String factoryClassName, InputTransformer transformer, StreamExpander expander) {
+    Preconditions.checkArgument(isValidSystemName(systemName),
+        String.format("systemName: %s must be non-empty and must not contain spaces or special characters.", systemName));
+    if (StringUtils.isBlank(factoryClassName)) {
+      LOGGER.warn("Blank SystemFactory class name for system: {}. A value must be provided in configuration using {}.",
+          systemName, String.format(FACTORY_CONFIG_KEY, systemName));
+    }
+    this.systemName = systemName;
+    this.factoryClassNameOptional = Optional.ofNullable(StringUtils.stripToNull(factoryClassName));
+    this.transformerOptional = Optional.ofNullable(transformer);
+    this.expanderOptional = Optional.ofNullable(expander);
+  }
+
+  /**
+   * If a container starts up without a checkpoint, this property determines where in the input stream we should start
+   * consuming. The value must be an {@link OffsetType}, one of the following:
+   * <ul>
+   *  <li>upcoming: Start processing messages that are published after the job starts.
+   *                Any messages published while the job was not running are not processed.
+   *  <li>oldest: Start processing at the oldest available message in the system,
+   *              and reprocess the entire available message history.
+   * </ul>
+   * This property is for all streams obtained using this system descriptor. To set it for an individual stream,
+   * see {@link InputDescriptor#withOffsetDefault}.
+   * If both are defined, the stream-level definition takes precedence.
+   *
+   * @param offsetType offset type to start processing from
+   * @return this system descriptor
+   */
+  public SubClass withDefaultStreamOffsetDefault(OffsetType offsetType) {
+    this.defaultStreamOffsetDefaultOptional = Optional.ofNullable(offsetType);
+    return (SubClass) this;
+  }
+
+  /**
+   * Additional system-specific properties for this system.
+   * <p>
+   * These properties are added under the {@code systems.system-name.*} scope.
+   *
+   * @param systemConfigs system-specific properties for this system
+   * @return this system descriptor
+   */
+  public SubClass withSystemConfigs(Map<String, String> systemConfigs) {
+    this.systemConfigs.putAll(systemConfigs);
+    return (SubClass) this;
+  }
+
+  /**
+   * Default properties for any stream obtained using this system descriptor.
+   * <p>
+   * For example, if "systems.kafka-system.default.stream.replication.factor"=2 was configured,
+   * then every Kafka stream created on the kafka-system will have a replication factor of 2
+   * unless the property is explicitly overridden using the stream descriptor.
+   *
+   * @param defaultStreamConfigs default stream properties
+   * @return this system descriptor
+   */
+  public SubClass withDefaultStreamConfigs(Map<String, String> defaultStreamConfigs) {
+    this.defaultStreamConfigs.putAll(defaultStreamConfigs);
+    return (SubClass) this;
+  }
+
+  public String getSystemName() {
+    return this.systemName;
+  }
+
+  public Optional<InputTransformer> getTransformer() {
+    return this.transformerOptional;
+  }
+
+  public Optional<StreamExpander> getExpander() {
+    return this.expanderOptional;
+  }
+
+  private boolean isValidSystemName(String id) {
+    return StringUtils.isNotBlank(id) && SYSTEM_NAME_PATTERN.matcher(id).matches();
+  }
+
+  public Map<String, String> toConfig() {
+    HashMap<String, String> configs = new HashMap<>();
+    this.factoryClassNameOptional.ifPresent(name -> configs.put(String.format(FACTORY_CONFIG_KEY, systemName), name));
+    this.defaultStreamOffsetDefaultOptional.ifPresent(dsod ->
+        configs.put(String.format(DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY, systemName), dsod.name().toLowerCase()));
+    this.defaultStreamConfigs.forEach((key, value) ->
+        configs.put(String.format(DEFAULT_STREAM_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
+    this.systemConfigs.forEach((key, value) ->
+        configs.put(String.format(SYSTEM_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
+    return configs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/system/descriptors/TransformingInputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/TransformingInputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/TransformingInputDescriptorProvider.java
new file mode 100644
index 0000000..e626efd
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/TransformingInputDescriptorProvider.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * Interface for advanced {@code SystemDescriptor}s that constrain the type of returned {@code InputDescriptor}s to
+ * their own {@code InputTransformer} function result types.
+ *
+ * @param <InputTransformerType> type of the system level {@code InputTransformer} results
+ */
+public interface TransformingInputDescriptorProvider<InputTransformerType> {
+
+  /**
+   * Gets a {@link InputDescriptor} for an input stream on this system. The stream has the provided
+   * stream level serde, and the default system level {@code InputTransformer}.
+   * <p>
+   * The type of messages in the stream is the type of messages returned by the default system level
+   * {@code InputTransformer}
+   *
+   * @param streamId id of the input stream
+   * @param serde stream level serde for the input stream
+   * @return an {@link InputDescriptor} for the input stream
+   */
+  InputDescriptor<InputTransformerType, ? extends InputDescriptor> getInputDescriptor(String streamId, Serde serde);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java b/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
index 7b8754a..296edf4 100644
--- a/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
+++ b/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/TableProvider.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
deleted file mode 100644
index 350324c..0000000
--- a/samza-api/src/main/java/org/apache/samza/table/TableProvider.java
+++ /dev/null
@@ -1,61 +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.table;
-
-import java.util.Map;
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.Context;
-
-/**
- * A table provider provides the implementation for a table. It ensures a table is
- * properly constructed and also manages its lifecycle.
- */
-@InterfaceStability.Unstable
-public interface TableProvider {
-  /**
-   * Initialize TableProvider with container and task context
-   * @param context context for the task
-   */
-  void init(Context context);
-
-  /**
-   * Get an instance of the table for read/write operations
-   * @return the underlying table
-   */
-  Table getTable();
-
-  /**
-   * Generate any configuration for this table, the generated configuration
-   * is used by Samza container to construct this table and any components
-   * necessary. 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<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig);
-
-  /**
-   * Shutdown the underlying table
-   */
-  void close();
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java b/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java
deleted file mode 100644
index 1bb0196..0000000
--- a/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java
+++ /dev/null
@@ -1,35 +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.table;
-
-import org.apache.samza.annotation.InterfaceStability;
-
-
-/**
- * Factory of a table provider object
- */
-@InterfaceStability.Unstable
-public interface TableProviderFactory {
-  /**
-   * Constructs an instances of the table provider based on a given table spec
-   * @param tableSpec the table spec
-   * @return the table provider
-   */
-  TableProvider getTableProvider(TableSpec tableSpec);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/TableSpec.java b/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
index 82577c4..2883a93 100644
--- a/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
@@ -32,8 +32,9 @@ import org.apache.samza.storage.SideInputsProcessor;
 /**
  * TableSpec is a blueprint for creating, validating, or simply describing a table in the runtime environment.
  *
- * It is typically created indirectly by constructing an instance of {@link org.apache.samza.operators.TableDescriptor},
- * and then invoke <code>BaseTableDescriptor.getTableSpec()</code>.
+ * It is typically created indirectly by constructing an instance of
+ * {@link org.apache.samza.table.descriptors.TableDescriptor}, and then invoke
+ * <code>BaseTableDescriptor.getTableSpec()</code>.
  *
  * It has specific attributes for common behaviors that Samza uses.
  *

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java
new file mode 100644
index 0000000..5d7b89e
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.table.descriptors;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+/**
+ * User facing class to collect metadata that fully describes a
+ * Samza table. This interface should be implemented by concrete table implementations.
+ * <p>
+ * Typical user code should look like the following, notice <code>withConfig()</code>
+ * is defined in this class and the rest in subclasses.
+ *
+ * <pre>
+ * {@code
+ * TableDescriptor<Integer, String, ?> tableDesc = new RocksDbTableDescriptor("tbl",
+ *         KVSerde.of(new IntegerSerde(), new StringSerde("UTF-8")))
+ *     .withBlockSize(1024)
+ *     .withConfig("some-key", "some-value");
+ * }
+ * </pre>
+
+ * Once constructed, a table descriptor can be registered with the system. Internally,
+ * the table descriptor is then converted to a {@link org.apache.samza.table.TableSpec},
+ * which is used to track tables internally.
+ *
+ * @param <K> the type of the key in this table
+ * @param <V> the type of the value in this table
+ * @param <D> the type of the concrete table descriptor
+ */
+@InterfaceStability.Unstable
+public interface TableDescriptor<K, V, D extends TableDescriptor<K, V, D>> {
+
+  /**
+   * Get the Id of the table
+   * @return Id of the table
+   */
+  String getTableId();
+
+  /**
+   * Add a configuration entry for the table
+   * @param key the key
+   * @param value the value
+   * @return this table descriptor instance
+   */
+  D withConfig(String key, String value);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProvider.java b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProvider.java
new file mode 100644
index 0000000..8640b2a
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProvider.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import java.util.Map;
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.context.Context;
+import org.apache.samza.table.Table;
+
+/**
+ * A table provider provides the implementation for a table. It ensures a table is
+ * properly constructed and also manages its lifecycle.
+ */
+@InterfaceStability.Unstable
+public interface TableProvider {
+  /**
+   * Initialize TableProvider with container and task context
+   * @param context context for the task
+   */
+  void init(Context context);
+
+  /**
+   * Get an instance of the table for read/write operations
+   * @return the underlying table
+   */
+  Table getTable();
+
+  /**
+   * Generate any configuration for this table, the generated configuration
+   * is used by Samza container to construct this table and any components
+   * necessary. 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<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig);
+
+  /**
+   * Shutdown the underlying table
+   */
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProviderFactory.java b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProviderFactory.java
new file mode 100644
index 0000000..2f9a607
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableProviderFactory.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.table.descriptors;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Factory of a table provider object
+ */
+@InterfaceStability.Unstable
+public interface TableProviderFactory {
+  /**
+   * Constructs an instances of the table provider based on a given table spec
+   * @param tableSpec the table spec
+   * @return the table provider
+   */
+  TableProvider getTableProvider(TableSpec tableSpec);
+}


[07/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------
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 a5eeba1..85306ef 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,10 +32,10 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptorUtil;
 import org.apache.samza.application.SamzaApplication;
-import org.apache.samza.application.ApplicationDescriptorUtil;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java
----------------------------------------------------------------------
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 94ff1eb..c5c0d78 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,9 +24,9 @@ import java.util.Map;
 import java.util.Random;
 import org.slf4j.MDC;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
-import org.apache.samza.application.ApplicationDescriptorUtil;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptorUtil;
 import org.apache.samza.application.ApplicationUtil;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java
----------------------------------------------------------------------
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 69eb5fe..1b38c9b 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,10 +22,10 @@ package org.apache.samza.runtime;
 import java.time.Duration;
 import java.util.List;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptorUtil;
 import org.apache.samza.application.SamzaApplication;
-import org.apache.samza.application.ApplicationDescriptorUtil;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.execution.RemoteJobPlanner;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/system/descriptors/DelegatingSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/descriptors/DelegatingSystemDescriptor.java b/samza-core/src/main/java/org/apache/samza/system/descriptors/DelegatingSystemDescriptor.java
new file mode 100644
index 0000000..aa0f6a4
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/system/descriptors/DelegatingSystemDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.samza.serializers.Serde;
+
+/**
+ * A descriptor for samza framework internal usage.
+ * <p>
+ * Allows creating a {@link SystemDescriptor} without setting the factory class name, and delegating
+ * rest of the system customization to configurations.
+ * <p>
+ * Useful for code-generation and testing use cases where the factory name is not known in advance.
+ */
+@SuppressWarnings("unchecked")
+public final class DelegatingSystemDescriptor extends SystemDescriptor<DelegatingSystemDescriptor>
+    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
+
+  /**
+   * Constructs an {@link DelegatingSystemDescriptor} instance with no system level serde.
+   * Serdes must be provided explicitly at stream level when getting input or output descriptors.
+   * SystemFactory class name must be provided in configuration.
+   *
+   * @param systemName name of this system
+   */
+  @VisibleForTesting
+  public DelegatingSystemDescriptor(String systemName) {
+    super(systemName, null, null, null);
+  }
+
+  @Override
+  public <StreamMessageType> GenericInputDescriptor<StreamMessageType> getInputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new GenericInputDescriptor<>(streamId, this, serde);
+  }
+
+  @Override
+  public <StreamMessageType> GenericOutputDescriptor<StreamMessageType> getOutputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new GenericOutputDescriptor<>(streamId, this, serde);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java
----------------------------------------------------------------------
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 03be758..05454a6 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
@@ -27,8 +27,10 @@ import java.util.Map;
 
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JavaTableConfig;
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.operators.TableImpl;
 import org.apache.samza.util.Util;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/TableManager.java
----------------------------------------------------------------------
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 d7b15a4..1d1e8b1 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
@@ -23,6 +23,8 @@ import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JavaTableConfig;
 import org.apache.samza.context.Context;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.Serde;
 import org.apache.samza.util.Util;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index f9d4007..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java
+++ /dev/null
@@ -1,164 +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.table.caching;
-
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.TableDescriptor;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.hybrid.BaseHybridTableDescriptor;
-
-import com.google.common.base.Preconditions;
-
-
-/**
- * Table descriptor for {@link CachingTable}.
- * @param <K> type of the key in the cache
- * @param <V> type of the value in the cache
- */
-public class CachingTableDescriptor<K, V> extends BaseHybridTableDescriptor<K, V, CachingTableDescriptor<K, V>> {
-  private Duration readTtl;
-  private Duration writeTtl;
-  private long cacheSize;
-  private TableDescriptor<K, V, ?> cache;
-  private TableDescriptor<K, V, ?> table;
-  private boolean isWriteAround;
-
-  /**
-   * Constructs a table descriptor instance with internal cache
-   *
-   * @param tableId Id of the table, it must conform to pattern { @literal [\\d\\w-_]+ }
-   * @param table target table descriptor
-   */
-  public CachingTableDescriptor(String tableId, TableDescriptor<K, V, ?> table) {
-    super(tableId);
-    this.table = table;
-  }
-
-  /**
-   * Constructs a table descriptor instance and specify a cache (as Table descriptor)
-   * to be used for caching. Cache get is not synchronized with put for better parallelism
-   * in the read path of {@link CachingTable}. As such, cache table implementation is
-   * expected to be thread-safe for concurrent accesses.
-   *
-   * @param tableId Id of the table, it must conform to pattern { @literal [\\d\\w-_]+ }
-   * @param table target table descriptor
-   * @param cache cache table descriptor
-   */
-  public CachingTableDescriptor(String tableId, TableDescriptor<K, V, ?> table,
-      TableDescriptor<K, V, ?> cache) {
-    this(tableId, table);
-    this.cache = cache;
-  }
-
-  @Override
-  public List<? extends TableDescriptor<K, V, ?>> getTableDescriptors() {
-    return cache != null
-        ? Arrays.asList(cache, table)
-        : Arrays.asList(table);
-  }
-
-  @Override
-  public TableSpec getTableSpec() {
-    validate();
-
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    generateTableSpecConfig(tableSpecConfig);
-
-    if (cache != null) {
-      tableSpecConfig.put(CachingTableProvider.CACHE_TABLE_ID, ((BaseTableDescriptor) cache).getTableSpec().getId());
-    } else {
-      if (readTtl != null) {
-        tableSpecConfig.put(CachingTableProvider.READ_TTL_MS, String.valueOf(readTtl.toMillis()));
-      }
-      if (writeTtl != null) {
-        tableSpecConfig.put(CachingTableProvider.WRITE_TTL_MS, String.valueOf(writeTtl.toMillis()));
-      }
-      if (cacheSize > 0) {
-        tableSpecConfig.put(CachingTableProvider.CACHE_SIZE, String.valueOf(cacheSize));
-      }
-    }
-
-    tableSpecConfig.put(CachingTableProvider.REAL_TABLE_ID, ((BaseTableDescriptor) table).getTableSpec().getId());
-    tableSpecConfig.put(CachingTableProvider.WRITE_AROUND, String.valueOf(isWriteAround));
-
-    return new TableSpec(tableId, serde, CachingTableProviderFactory.class.getName(), tableSpecConfig);
-  }
-
-  /**
-   * Specify the TTL for each read access, ie. record is expired after
-   * the TTL duration since last read access of each key.
-   * @param readTtl read TTL
-   * @return this descriptor
-   */
-  public CachingTableDescriptor withReadTtl(Duration readTtl) {
-    this.readTtl = readTtl;
-    return this;
-  }
-
-  /**
-   * Specify the TTL for each write access, ie. record is expired after
-   * the TTL duration since last write access of each key.
-   * @param writeTtl write TTL
-   * @return this descriptor
-   */
-  public CachingTableDescriptor withWriteTtl(Duration writeTtl) {
-    this.writeTtl = writeTtl;
-    return this;
-  }
-
-  /**
-   * Specify the max cache size for size-based eviction.
-   * @param cacheSize max size of the cache
-   * @return this descriptor
-   */
-  public CachingTableDescriptor withCacheSize(long cacheSize) {
-    this.cacheSize = cacheSize;
-    return this;
-  }
-
-  /**
-   * Specify if write-around policy should be used to bypass writing
-   * to cache for put operations. This is useful when put() is the
-   * dominant operation and get() has no locality with recent puts.
-   * @return this descriptor
-   */
-  public CachingTableDescriptor withWriteAround() {
-    this.isWriteAround = true;
-    return this;
-  }
-
-  @Override
-  protected void validate() {
-    super.validate();
-    Preconditions.checkNotNull(table, "Actual table is required.");
-    if (cache == null) {
-      Preconditions.checkNotNull(readTtl, "readTtl must be specified.");
-    } else {
-      Preconditions.checkArgument(readTtl == null && writeTtl == null && cacheSize == 0,
-          "Invalid to specify both {cache} and {readTtl|writeTtl|cacheSize} at the same time.");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProvider.java
deleted file mode 100644
index c959a56..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProvider.java
+++ /dev/null
@@ -1,104 +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.table.caching;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.samza.table.ReadWriteTable;
-import org.apache.samza.table.ReadableTable;
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.caching.guava.GuavaCacheTable;
-import org.apache.samza.table.utils.BaseTableProvider;
-
-import com.google.common.cache.CacheBuilder;
-
-/**
- * Table provider for {@link CachingTable}.
- */
-public class CachingTableProvider extends BaseTableProvider {
-
-  public static final String REAL_TABLE_ID = "realTableId";
-  public static final String CACHE_TABLE_ID = "cacheTableId";
-  public static final String READ_TTL_MS = "readTtl";
-  public static final String WRITE_TTL_MS = "writeTtl";
-  public static final String CACHE_SIZE = "cacheSize";
-  public static final String WRITE_AROUND = "writeAround";
-
-  // Store the cache instances created by default
-  private final List<ReadWriteTable> defaultCaches = new ArrayList<>();
-
-  public CachingTableProvider(TableSpec tableSpec) {
-    super(tableSpec);
-  }
-
-  @Override
-  public Table getTable() {
-    String realTableId = tableSpec.getConfig().get(REAL_TABLE_ID);
-    ReadableTable table = (ReadableTable) this.context.getTaskContext().getTable(realTableId);
-
-    String cacheTableId = tableSpec.getConfig().get(CACHE_TABLE_ID);
-    ReadWriteTable cache;
-
-    if (cacheTableId != null) {
-      cache = (ReadWriteTable) this.context.getTaskContext().getTable(cacheTableId);
-    } else {
-      cache = createDefaultCacheTable(realTableId);
-      defaultCaches.add(cache);
-    }
-
-    boolean isWriteAround = Boolean.parseBoolean(tableSpec.getConfig().get(WRITE_AROUND));
-    CachingTable cachingTable = new CachingTable(tableSpec.getId(), table, cache, isWriteAround);
-    cachingTable.init(this.context);
-    return cachingTable;
-  }
-
-  @Override
-  public void close() {
-    defaultCaches.forEach(c -> c.close());
-  }
-
-  private ReadWriteTable createDefaultCacheTable(String tableId) {
-    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) {
-      cacheBuilder.expireAfterAccess(readTtlMs, TimeUnit.MILLISECONDS);
-    }
-    if (writeTtlMs != -1) {
-      cacheBuilder.expireAfterWrite(writeTtlMs, TimeUnit.MILLISECONDS);
-    }
-    if (cacheSize != -1) {
-      cacheBuilder.maximumSize(cacheSize);
-    }
-
-    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());
-    cacheTable.init(this.context);
-
-    return cacheTable;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProviderFactory.java
deleted file mode 100644
index 9262207..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableProviderFactory.java
+++ /dev/null
@@ -1,34 +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.table.caching;
-
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableProviderFactory;
-import org.apache.samza.table.TableSpec;
-
-/**
- * Table provider factory for {@link CachingTable}.
- */
-public class CachingTableProviderFactory implements TableProviderFactory {
-  @Override
-  public TableProvider getTableProvider(TableSpec tableSpec) {
-    return new CachingTableProvider(tableSpec);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableDescriptor.java
new file mode 100644
index 0000000..10665a3
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableDescriptor.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.caching.descriptors;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.descriptors.BaseHybridTableDescriptor;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+
+/**
+ * Table descriptor for {@link org.apache.samza.table.caching.CachingTable}.
+ * @param <K> type of the key in the cache
+ * @param <V> type of the value in the cache
+ */
+public class CachingTableDescriptor<K, V> extends BaseHybridTableDescriptor<K, V, CachingTableDescriptor<K, V>> {
+  private Duration readTtl;
+  private Duration writeTtl;
+  private long cacheSize;
+  private TableDescriptor<K, V, ?> cache;
+  private TableDescriptor<K, V, ?> table;
+  private boolean isWriteAround;
+
+  /**
+   * Constructs a table descriptor instance with internal cache
+   *
+   * @param tableId Id of the table, it must conform to pattern { @literal [\\d\\w-_]+ }
+   * @param table target table descriptor
+   */
+  public CachingTableDescriptor(String tableId, TableDescriptor<K, V, ?> table) {
+    super(tableId);
+    this.table = table;
+  }
+
+  /**
+   * Constructs a table descriptor instance and specify a cache (as Table descriptor)
+   * to be used for caching. Cache get is not synchronized with put for better parallelism
+   * in the read path of {@link org.apache.samza.table.caching.CachingTable}. As such, cache table implementation is
+   * expected to be thread-safe for concurrent accesses.
+   *
+   * @param tableId Id of the table, it must conform to pattern { @literal [\\d\\w-_]+ }
+   * @param table target table descriptor
+   * @param cache cache table descriptor
+   */
+  public CachingTableDescriptor(String tableId, TableDescriptor<K, V, ?> table,
+      TableDescriptor<K, V, ?> cache) {
+    this(tableId, table);
+    this.cache = cache;
+  }
+
+  @Override
+  public List<? extends TableDescriptor<K, V, ?>> getTableDescriptors() {
+    return cache != null
+        ? Arrays.asList(cache, table)
+        : Arrays.asList(table);
+  }
+
+  @Override
+  public TableSpec getTableSpec() {
+    validate();
+
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    generateTableSpecConfig(tableSpecConfig);
+
+    if (cache != null) {
+      tableSpecConfig.put(CachingTableProvider.CACHE_TABLE_ID, ((BaseTableDescriptor) cache).getTableSpec().getId());
+    } else {
+      if (readTtl != null) {
+        tableSpecConfig.put(CachingTableProvider.READ_TTL_MS, String.valueOf(readTtl.toMillis()));
+      }
+      if (writeTtl != null) {
+        tableSpecConfig.put(CachingTableProvider.WRITE_TTL_MS, String.valueOf(writeTtl.toMillis()));
+      }
+      if (cacheSize > 0) {
+        tableSpecConfig.put(CachingTableProvider.CACHE_SIZE, String.valueOf(cacheSize));
+      }
+    }
+
+    tableSpecConfig.put(CachingTableProvider.REAL_TABLE_ID, ((BaseTableDescriptor) table).getTableSpec().getId());
+    tableSpecConfig.put(CachingTableProvider.WRITE_AROUND, String.valueOf(isWriteAround));
+
+    return new TableSpec(tableId, serde, CachingTableProviderFactory.class.getName(), tableSpecConfig);
+  }
+
+  /**
+   * Specify the TTL for each read access, ie. record is expired after
+   * the TTL duration since last read access of each key.
+   * @param readTtl read TTL
+   * @return this descriptor
+   */
+  public CachingTableDescriptor withReadTtl(Duration readTtl) {
+    this.readTtl = readTtl;
+    return this;
+  }
+
+  /**
+   * Specify the TTL for each write access, ie. record is expired after
+   * the TTL duration since last write access of each key.
+   * @param writeTtl write TTL
+   * @return this descriptor
+   */
+  public CachingTableDescriptor withWriteTtl(Duration writeTtl) {
+    this.writeTtl = writeTtl;
+    return this;
+  }
+
+  /**
+   * Specify the max cache size for size-based eviction.
+   * @param cacheSize max size of the cache
+   * @return this descriptor
+   */
+  public CachingTableDescriptor withCacheSize(long cacheSize) {
+    this.cacheSize = cacheSize;
+    return this;
+  }
+
+  /**
+   * Specify if write-around policy should be used to bypass writing
+   * to cache for put operations. This is useful when put() is the
+   * dominant operation and get() has no locality with recent puts.
+   * @return this descriptor
+   */
+  public CachingTableDescriptor withWriteAround() {
+    this.isWriteAround = true;
+    return this;
+  }
+
+  @Override
+  @VisibleForTesting
+  public void validate() {
+    super.validate();
+    Preconditions.checkNotNull(table, "Actual table is required.");
+    if (cache == null) {
+      Preconditions.checkNotNull(readTtl, "readTtl must be specified.");
+    } else {
+      Preconditions.checkArgument(readTtl == null && writeTtl == null && cacheSize == 0,
+          "Invalid to specify both {cache} and {readTtl|writeTtl|cacheSize} at the same time.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProvider.java
new file mode 100644
index 0000000..007f372
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProvider.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.caching.descriptors;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.samza.table.ReadWriteTable;
+import org.apache.samza.table.ReadableTable;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.caching.CachingTable;
+import org.apache.samza.table.caching.guava.GuavaCacheTable;
+import org.apache.samza.table.utils.descriptors.BaseTableProvider;
+
+import com.google.common.cache.CacheBuilder;
+
+/**
+ * Table provider for {@link CachingTable}.
+ */
+public class CachingTableProvider extends BaseTableProvider {
+
+  public static final String REAL_TABLE_ID = "realTableId";
+  public static final String CACHE_TABLE_ID = "cacheTableId";
+  public static final String READ_TTL_MS = "readTtl";
+  public static final String WRITE_TTL_MS = "writeTtl";
+  public static final String CACHE_SIZE = "cacheSize";
+  public static final String WRITE_AROUND = "writeAround";
+
+  // Store the cache instances created by default
+  private final List<ReadWriteTable> defaultCaches = new ArrayList<>();
+
+  public CachingTableProvider(TableSpec tableSpec) {
+    super(tableSpec);
+  }
+
+  @Override
+  public Table getTable() {
+    String realTableId = tableSpec.getConfig().get(REAL_TABLE_ID);
+    ReadableTable table = (ReadableTable) this.context.getTaskContext().getTable(realTableId);
+
+    String cacheTableId = tableSpec.getConfig().get(CACHE_TABLE_ID);
+    ReadWriteTable cache;
+
+    if (cacheTableId != null) {
+      cache = (ReadWriteTable) this.context.getTaskContext().getTable(cacheTableId);
+    } else {
+      cache = createDefaultCacheTable(realTableId);
+      defaultCaches.add(cache);
+    }
+
+    boolean isWriteAround = Boolean.parseBoolean(tableSpec.getConfig().get(WRITE_AROUND));
+    CachingTable cachingTable = new CachingTable(tableSpec.getId(), table, cache, isWriteAround);
+    cachingTable.init(this.context);
+    return cachingTable;
+  }
+
+  @Override
+  public void close() {
+    defaultCaches.forEach(c -> c.close());
+  }
+
+  private ReadWriteTable createDefaultCacheTable(String tableId) {
+    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) {
+      cacheBuilder.expireAfterAccess(readTtlMs, TimeUnit.MILLISECONDS);
+    }
+    if (writeTtlMs != -1) {
+      cacheBuilder.expireAfterWrite(writeTtlMs, TimeUnit.MILLISECONDS);
+    }
+    if (cacheSize != -1) {
+      cacheBuilder.maximumSize(cacheSize);
+    }
+
+    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());
+    cacheTable.init(this.context);
+
+    return cacheTable;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProviderFactory.java
new file mode 100644
index 0000000..68eb162
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/descriptors/CachingTableProviderFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.caching.descriptors;
+
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
+import org.apache.samza.table.TableSpec;
+
+/**
+ * Table provider factory for {@link org.apache.samza.table.caching.CachingTable}.
+ */
+public class CachingTableProviderFactory implements TableProviderFactory {
+  @Override
+  public TableProvider getTableProvider(TableSpec tableSpec) {
+    return new CachingTableProvider(tableSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 4a05013..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableDescriptor.java
+++ /dev/null
@@ -1,75 +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.table.caching.guava;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.utils.SerdeUtils;
-
-import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
-
-
-/**
- * Table descriptor for {@link GuavaCacheTable}.
- * @param <K> type of the key in the cache
- * @param <V> type of the value in the cache
- */
-public class GuavaCacheTableDescriptor<K, V> extends BaseTableDescriptor<K, V, GuavaCacheTableDescriptor<K, V>> {
-  private Cache<K, V> cache;
-
-  /**
-   * {@inheritDoc}
-   */
-  public GuavaCacheTableDescriptor(String tableId) {
-    super(tableId);
-  }
-
-  @Override
-  public TableSpec getTableSpec() {
-    validate();
-
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    generateTableSpecConfig(tableSpecConfig);
-
-    tableSpecConfig.put(GuavaCacheTableProvider.GUAVA_CACHE, SerdeUtils.serialize("Guava cache", cache));
-
-    return new TableSpec(tableId, serde, GuavaCacheTableProviderFactory.class.getName(), tableSpecConfig);
-  }
-
-  /**
-   * Specify a pre-configured Guava cache instance to be used for caching table.
-   * @param cache Guava cache instance
-   * @return this descriptor
-   */
-  public GuavaCacheTableDescriptor withCache(Cache<K, V> cache) {
-    this.cache = cache;
-    return this;
-  }
-
-  @Override
-  protected void validate() {
-    super.validate();
-    Preconditions.checkArgument(cache != null, "Must provide a Guava cache instance.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 39f332e..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProvider.java
+++ /dev/null
@@ -1,59 +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.table.caching.guava;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.utils.BaseTableProvider;
-import org.apache.samza.table.utils.SerdeUtils;
-
-import com.google.common.cache.Cache;
-
-
-/**
- * Table provider for {@link GuavaCacheTable}.
- */
-public class GuavaCacheTableProvider extends BaseTableProvider {
-
-  public static final String GUAVA_CACHE = "guavaCache";
-
-  private List<GuavaCacheTable> guavaTables = new ArrayList<>();
-
-  public GuavaCacheTableProvider(TableSpec tableSpec) {
-    super(tableSpec);
-  }
-
-  @Override
-  public Table getTable() {
-    Cache guavaCache = SerdeUtils.deserialize(GUAVA_CACHE, tableSpec.getConfig().get(GUAVA_CACHE));
-    GuavaCacheTable table = new GuavaCacheTable(tableSpec.getId(), guavaCache);
-    table.init(this.context);
-    guavaTables.add(table);
-    return table;
-  }
-
-  @Override
-  public void close() {
-    guavaTables.forEach(t -> t.close());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProviderFactory.java
deleted file mode 100644
index 066c6f9..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProviderFactory.java
+++ /dev/null
@@ -1,34 +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.table.caching.guava;
-
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableProviderFactory;
-import org.apache.samza.table.TableSpec;
-
-/**
- * Table provider factory for {@link GuavaCacheTable}.
- */
-public class GuavaCacheTableProviderFactory implements TableProviderFactory {
-  @Override
-  public TableProvider getTableProvider(TableSpec tableSpec) {
-    return new GuavaCacheTableProvider(tableSpec);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableDescriptor.java
new file mode 100644
index 0000000..e0224bb
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.table.caching.guava.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.utils.SerdeUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.cache.Cache;
+
+
+/**
+ * Table descriptor for {@link org.apache.samza.table.caching.guava.GuavaCacheTable}.
+ * @param <K> type of the key in the cache
+ * @param <V> type of the value in the cache
+ */
+public class GuavaCacheTableDescriptor<K, V> extends BaseTableDescriptor<K, V, GuavaCacheTableDescriptor<K, V>> {
+  private Cache<K, V> cache;
+
+  /**
+   * {@inheritDoc}
+   */
+  public GuavaCacheTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  @Override
+  public TableSpec getTableSpec() {
+    validate();
+
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    generateTableSpecConfig(tableSpecConfig);
+
+    tableSpecConfig.put(GuavaCacheTableProvider.GUAVA_CACHE, SerdeUtils.serialize("Guava cache", cache));
+
+    return new TableSpec(tableId, serde, GuavaCacheTableProviderFactory.class.getName(), tableSpecConfig);
+  }
+
+  /**
+   * Specify a pre-configured Guava cache instance to be used for caching table.
+   * @param cache Guava cache instance
+   * @return this descriptor
+   */
+  public GuavaCacheTableDescriptor withCache(Cache<K, V> cache) {
+    this.cache = cache;
+    return this;
+  }
+
+  @Override
+  protected void validate() {
+    super.validate();
+    Preconditions.checkArgument(cache != null, "Must provide a Guava cache instance.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProvider.java
new file mode 100644
index 0000000..45d1fdc
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProvider.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.table.caching.guava.descriptors;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.caching.guava.GuavaCacheTable;
+import org.apache.samza.table.utils.descriptors.BaseTableProvider;
+import org.apache.samza.table.utils.SerdeUtils;
+
+import com.google.common.cache.Cache;
+
+
+/**
+ * Table provider for {@link GuavaCacheTable}.
+ */
+public class GuavaCacheTableProvider extends BaseTableProvider {
+
+  public static final String GUAVA_CACHE = "guavaCache";
+
+  private List<GuavaCacheTable> guavaTables = new ArrayList<>();
+
+  public GuavaCacheTableProvider(TableSpec tableSpec) {
+    super(tableSpec);
+  }
+
+  @Override
+  public Table getTable() {
+    Cache guavaCache = SerdeUtils.deserialize(GUAVA_CACHE, tableSpec.getConfig().get(GUAVA_CACHE));
+    GuavaCacheTable table = new GuavaCacheTable(tableSpec.getId(), guavaCache);
+    table.init(this.context);
+    guavaTables.add(table);
+    return table;
+  }
+
+  @Override
+  public void close() {
+    guavaTables.forEach(t -> t.close());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProviderFactory.java
new file mode 100644
index 0000000..01228cc
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/guava/descriptors/GuavaCacheTableProviderFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.caching.guava.descriptors;
+
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
+import org.apache.samza.table.TableSpec;
+
+/**
+ * Table provider factory for {@link org.apache.samza.table.caching.guava.GuavaCacheTable}.
+ */
+public class GuavaCacheTableProviderFactory implements TableProviderFactory {
+  @Override
+  public TableProvider getTableProvider(TableSpec tableSpec) {
+    return new GuavaCacheTableProvider(tableSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseHybridTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseHybridTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseHybridTableDescriptor.java
new file mode 100644
index 0000000..15486c7
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseHybridTableDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.table.descriptors;
+
+import java.util.List;
+
+/**
+ * Base class for hybrid table descriptors. A hybrid table consists of one or more
+ * table descriptors, and it orchestrates operations between them to achieve more advanced
+ * functionality.
+ *
+ * @param <K> the type of the key
+ * @param <V> the type of the value
+ * @param <D> the type of this table descriptor
+ */
+abstract public class BaseHybridTableDescriptor<K, V, D extends BaseHybridTableDescriptor<K, V, D>>
+    extends BaseTableDescriptor<K, V, D> {
+
+  /**
+   * {@inheritDoc}
+   */
+  public BaseHybridTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  /**
+   * Get tables contained within this table.
+   * @return list of tables
+   */
+  abstract public List<? extends TableDescriptor<K, V, ?>> getTableDescriptors();
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseTableDescriptor.java
new file mode 100644
index 0000000..246216b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/descriptors/BaseTableDescriptor.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Base class for all table descriptor implementations.
+ *
+ * @param <K> the type of the key in this table
+ * @param <V> the type of the value in this table
+ * @param <D> the type of the concrete table descriptor
+ */
+abstract public class BaseTableDescriptor<K, V, D extends BaseTableDescriptor<K, V, D>>
+    implements TableDescriptor<K, V, D> {
+
+  protected final String tableId;
+
+  protected KVSerde<K, V> serde = KVSerde.of(new NoOpSerde(), new NoOpSerde());
+
+  protected final Map<String, String> config = new HashMap<>();
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   */
+  protected BaseTableDescriptor(String tableId) {
+    this.tableId = tableId;
+  }
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   * @param serde the serde for key and value
+   */
+  protected BaseTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    this.tableId = tableId;
+    this.serde = serde;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public D withConfig(String key, String value) {
+    config.put(key, value);
+    return (D) this;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String getTableId() {
+    return tableId;
+  }
+
+  /**
+   * Get the serde assigned to this {@link TableDescriptor}
+   *
+   * @return {@link KVSerde} used by this table
+   */
+  public KVSerde<K, V> getSerde() {
+    return serde;
+  }
+
+  /**
+   * Generate config for {@link TableSpec}; this method is used internally.
+   * @param tableSpecConfig configuration for the {@link TableSpec}
+   */
+  protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
+    tableSpecConfig.putAll(config);
+  }
+
+  /**
+   * Validate that this table descriptor is constructed properly; this method is used internally.
+   */
+  protected void validate() {
+  }
+
+  /**
+   * Create a {@link TableSpec} from this table descriptor; this method is used internally.
+   *
+   * @return the {@link TableSpec}
+   */
+  abstract public TableSpec getTableSpec();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java
deleted file mode 100644
index 48efd0c..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java
+++ /dev/null
@@ -1,50 +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.table.hybrid;
-
-import java.util.List;
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.TableDescriptor;
-
-/**
- * Base class for hybrid table descriptors. A hybrid table consists of one or more
- * table descriptors, and it orchestrates operations between them to achieve more advanced
- * functionality.
- *
- * @param <K> the type of the key
- * @param <V> the type of the value
- * @param <D> the type of this table descriptor
- */
-abstract public class BaseHybridTableDescriptor<K, V, D extends BaseHybridTableDescriptor<K, V, D>>
-    extends BaseTableDescriptor<K, V, D> {
-
-  /**
-   * {@inheritDoc}
-   */
-  public BaseHybridTableDescriptor(String tableId) {
-    super(tableId);
-  }
-
-  /**
-   * Get tables contained within this table.
-   * @return list of tables
-   */
-  abstract public List<? extends TableDescriptor<K, V, ?>> getTableDescriptors();
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java
index 4cbc270..52bdc71 100644
--- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java
+++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java
@@ -44,9 +44,8 @@ public class RemoteReadWriteTable<K, V> extends RemoteReadableTable<K, V> implem
 
   private DefaultTableWriteMetrics writeMetrics;
 
-  @VisibleForTesting
-  final TableWriteFunction<K, V> writeFn;
-  final TableRateLimiter writeRateLimiter;
+  protected final TableWriteFunction<K, V> writeFn;
+  protected final TableRateLimiter writeRateLimiter;
 
   public RemoteReadWriteTable(String tableId, TableReadFunction readFn, TableWriteFunction writeFn,
       TableRateLimiter<K, V> readRateLimiter, TableRateLimiter<K, V> writeRateLimiter,
@@ -219,4 +218,14 @@ public class RemoteReadWriteTable<K, V> extends RemoteReadableTable<K, V> implem
     writeFn.close();
     super.close();
   }
+
+  @VisibleForTesting
+  public TableWriteFunction<K, V> getWriteFn() {
+    return writeFn;
+  }
+
+  @VisibleForTesting
+  public TableRateLimiter getWriteRateLimiter() {
+    return writeRateLimiter;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java
----------------------------------------------------------------------
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 9487e39..0ae2728 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
@@ -77,13 +77,11 @@ public class RemoteReadableTable<K, V> implements ReadableTable<K, V> {
 
   protected final ExecutorService callbackExecutor;
   protected final ExecutorService tableExecutor;
+  protected final TableReadFunction<K, V> readFn;
+  protected final TableRateLimiter<K, V> readRateLimiter;
 
   private DefaultTableReadMetrics readMetrics;
 
-  @VisibleForTesting
-  final TableReadFunction<K, V> readFn;
-  final TableRateLimiter<K, V> readRateLimiter;
-
   /**
    * Construct a RemoteReadableTable instance
    * @param tableId table id
@@ -298,4 +296,24 @@ public class RemoteReadableTable<K, V> implements ReadableTable<K, V> {
   public void close() {
     readFn.close();
   }
+
+  @VisibleForTesting
+  public ExecutorService getCallbackExecutor() {
+    return callbackExecutor;
+  }
+
+  @VisibleForTesting
+  public ExecutorService getTableExecutor() {
+    return tableExecutor;
+  }
+
+  @VisibleForTesting
+  public TableReadFunction<K, V> getReadFn() {
+    return readFn;
+  }
+
+  @VisibleForTesting
+  public TableRateLimiter<K, V> getReadRateLimiter() {
+    return readRateLimiter;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index fe01e6b..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java
+++ /dev/null
@@ -1,275 +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.table.remote;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.retry.TableRetryPolicy;
-import org.apache.samza.table.utils.SerdeUtils;
-import org.apache.samza.util.EmbeddedTaggedRateLimiter;
-import org.apache.samza.util.RateLimiter;
-
-import com.google.common.base.Preconditions;
-
-
-/**
- * Table descriptor for remote store backed tables
- *
- * @param <K> the type of the key
- * @param <V> the type of the value
- */
-public class RemoteTableDescriptor<K, V> extends BaseTableDescriptor<K, V, RemoteTableDescriptor<K, V>> {
-  /**
-   * Tag to be used for provision credits for rate limiting read operations from the remote table.
-   * Caller must pre-populate the credits with this tag when specifying a custom rate limiter instance
-   * through {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, TableRateLimiter.CreditFunction,
-   * TableRateLimiter.CreditFunction)}
-   */
-  public static final String RL_READ_TAG = "readTag";
-
-  /**
-   * Tag to be used for provision credits for rate limiting write operations into the remote table.
-   * Caller can optionally populate the credits with this tag when specifying a custom rate limiter instance
-   * through {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, TableRateLimiter.CreditFunction,
-   * TableRateLimiter.CreditFunction)} and it needs the write functionality.
-   */
-  public static final String RL_WRITE_TAG = "writeTag";
-
-  // Input support for a specific remote store (required)
-  private TableReadFunction<K, V> readFn;
-
-  // Output support for a specific remote store (optional)
-  private TableWriteFunction<K, V> writeFn;
-
-  // Rate limiter for client-side throttling;
-  // can either be constructed indirectly from rates or overridden by withRateLimiter()
-  private RateLimiter rateLimiter;
-
-  // Rates for constructing the default rate limiter when they are non-zero
-  private Map<String, Integer> tagCreditsMap = new HashMap<>();
-
-  private TableRateLimiter.CreditFunction<K, V> readCreditFn;
-  private TableRateLimiter.CreditFunction<K, V> writeCreditFn;
-
-  private TableRetryPolicy readRetryPolicy;
-  private TableRetryPolicy writeRetryPolicy;
-
-  // By default execute future callbacks on the native client threads
-  // ie. no additional thread pool for callbacks.
-  private int asyncCallbackPoolSize = -1;
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   */
-  public RemoteTableDescriptor(String tableId) {
-    super(tableId);
-  }
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   * @param serde the serde for key and value
-   */
-  public RemoteTableDescriptor(String tableId, KVSerde<K, V> serde) {
-    super(tableId, serde);
-  }
-
-  @Override
-  public TableSpec getTableSpec() {
-    validate();
-
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    generateTableSpecConfig(tableSpecConfig);
-
-    // Serialize and store reader/writer functions
-    tableSpecConfig.put(RemoteTableProvider.READ_FN, SerdeUtils.serialize("read function", readFn));
-
-    if (writeFn != null) {
-      tableSpecConfig.put(RemoteTableProvider.WRITE_FN, SerdeUtils.serialize("write function", writeFn));
-    }
-
-    // Serialize the rate limiter if specified
-    if (!tagCreditsMap.isEmpty()) {
-      rateLimiter = new EmbeddedTaggedRateLimiter(tagCreditsMap);
-    }
-
-    if (rateLimiter != null) {
-      tableSpecConfig.put(RemoteTableProvider.RATE_LIMITER, SerdeUtils.serialize("rate limiter", rateLimiter));
-    }
-
-    // Serialize the readCredit and writeCredit functions
-    if (readCreditFn != null) {
-      tableSpecConfig.put(RemoteTableProvider.READ_CREDIT_FN, SerdeUtils.serialize(
-          "read credit function", readCreditFn));
-    }
-
-    if (writeCreditFn != null) {
-      tableSpecConfig.put(RemoteTableProvider.WRITE_CREDIT_FN, SerdeUtils.serialize(
-          "write credit function", writeCreditFn));
-    }
-
-    if (readRetryPolicy != null) {
-      tableSpecConfig.put(RemoteTableProvider.READ_RETRY_POLICY, SerdeUtils.serialize(
-          "read retry policy", readRetryPolicy));
-    }
-
-    if (writeRetryPolicy != null) {
-      tableSpecConfig.put(RemoteTableProvider.WRITE_RETRY_POLICY, SerdeUtils.serialize(
-          "write retry policy", writeRetryPolicy));
-    }
-
-    tableSpecConfig.put(RemoteTableProvider.ASYNC_CALLBACK_POOL_SIZE, String.valueOf(asyncCallbackPoolSize));
-
-    return new TableSpec(tableId, serde, RemoteTableProviderFactory.class.getName(), tableSpecConfig);
-  }
-
-  /**
-   * Use specified TableReadFunction with remote table and a retry policy.
-   * @param readFn read function instance
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withReadFunction(TableReadFunction<K, V> readFn) {
-    Preconditions.checkNotNull(readFn, "null read function");
-    this.readFn = readFn;
-    return this;
-  }
-
-  /**
-   * Use specified TableWriteFunction with remote table and a retry policy.
-   * @param writeFn write function instance
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withWriteFunction(TableWriteFunction<K, V> writeFn) {
-    Preconditions.checkNotNull(writeFn, "null write function");
-    this.writeFn = writeFn;
-    return this;
-  }
-
-  /**
-   * Use specified TableReadFunction with remote table.
-   * @param readFn read function instance
-   * @param retryPolicy retry policy for the read function
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withReadFunction(TableReadFunction<K, V> readFn, TableRetryPolicy retryPolicy) {
-    Preconditions.checkNotNull(readFn, "null read function");
-    Preconditions.checkNotNull(retryPolicy, "null retry policy");
-    this.readFn = readFn;
-    this.readRetryPolicy = retryPolicy;
-    return this;
-  }
-
-  /**
-   * Use specified TableWriteFunction with remote table.
-   * @param writeFn write function instance
-   * @param retryPolicy retry policy for the write function
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withWriteFunction(TableWriteFunction<K, V> writeFn, TableRetryPolicy retryPolicy) {
-    Preconditions.checkNotNull(writeFn, "null write function");
-    Preconditions.checkNotNull(retryPolicy, "null retry policy");
-    this.writeFn = writeFn;
-    this.writeRetryPolicy = retryPolicy;
-    return this;
-  }
-
-  /**
-   * Specify a rate limiter along with credit functions to map a table record (as KV) to the amount
-   * of credits to be charged from the rate limiter for table read and write operations.
-   * This is an advanced API that provides greater flexibility to throttle each record in the table
-   * with different number of credits. For most common use-cases eg: limit the number of read/write
-   * operations, please instead use the {@link RemoteTableDescriptor#withReadRateLimit(int)} and
-   * {@link RemoteTableDescriptor#withWriteRateLimit(int)}.
-   *
-   * @param rateLimiter rate limiter instance to be used for throttling
-   * @param readCreditFn credit function for rate limiting read operations
-   * @param writeCreditFn credit function for rate limiting write operations
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withRateLimiter(RateLimiter rateLimiter,
-      TableRateLimiter.CreditFunction<K, V> readCreditFn,
-      TableRateLimiter.CreditFunction<K, V> writeCreditFn) {
-    Preconditions.checkNotNull(rateLimiter, "null read rate limiter");
-    this.rateLimiter = rateLimiter;
-    this.readCreditFn = readCreditFn;
-    this.writeCreditFn = writeCreditFn;
-    return this;
-  }
-
-  /**
-   * Specify the rate limit for table read operations. If the read rate limit is set with this method
-   * it is invalid to call {@link RemoteTableDescriptor#withRateLimiter(RateLimiter,
-   * TableRateLimiter.CreditFunction, TableRateLimiter.CreditFunction)}
-   * and vice versa.
-   * @param creditsPerSec rate limit for read operations; must be positive
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withReadRateLimit(int creditsPerSec) {
-    Preconditions.checkArgument(creditsPerSec > 0, "Max read rate must be a positive number.");
-    tagCreditsMap.put(RL_READ_TAG, creditsPerSec);
-    return this;
-  }
-
-  /**
-   * Specify the rate limit for table write operations. If the write rate limit is set with this method
-   * it is invalid to call {@link RemoteTableDescriptor#withRateLimiter(RateLimiter,
-   * TableRateLimiter.CreditFunction, TableRateLimiter.CreditFunction)}
-   * and vice versa.
-   * @param creditsPerSec rate limit for write operations; must be positive
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withWriteRateLimit(int creditsPerSec) {
-    Preconditions.checkArgument(creditsPerSec > 0, "Max write rate must be a positive number.");
-    tagCreditsMap.put(RL_WRITE_TAG, creditsPerSec);
-    return this;
-  }
-
-  /**
-   * Specify the size of the thread pool for the executor used to execute
-   * callbacks of CompletableFutures of async Table operations. By default, these
-   * futures are completed (called) by the threads of the native store client. Depending
-   * on the implementation of the native client, it may or may not allow executing long
-   * running operations in the callbacks. This config can be used to execute the callbacks
-   * from a separate executor to decouple from the native client. If configured, this
-   * thread pool is shared by all read and write operations.
-   * @param poolSize max number of threads in the executor for async callbacks
-   * @return this table descriptor instance
-   */
-  public RemoteTableDescriptor<K, V> withAsyncCallbackExecutorPoolSize(int poolSize) {
-    this.asyncCallbackPoolSize = poolSize;
-    return this;
-  }
-
-  @Override
-  protected void validate() {
-    super.validate();
-    Preconditions.checkNotNull(readFn, "TableReadFunction is required.");
-    Preconditions.checkArgument(rateLimiter == null || tagCreditsMap.isEmpty(),
-        "Only one of rateLimiter instance or read/write limits can be specified");
-    // Assume callback executor pool should have no more than 20 threads
-    Preconditions.checkArgument(asyncCallbackPoolSize <= 20,
-        "too many threads for async callback executor.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java
deleted file mode 100644
index 9415e70..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java
+++ /dev/null
@@ -1,200 +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.table.remote;
-
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.retry.RetriableReadFunction;
-import org.apache.samza.table.retry.RetriableWriteFunction;
-import org.apache.samza.table.retry.TableRetryPolicy;
-import org.apache.samza.table.utils.BaseTableProvider;
-import org.apache.samza.table.utils.SerdeUtils;
-import org.apache.samza.table.utils.TableMetricsUtil;
-import org.apache.samza.util.RateLimiter;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-
-import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_READ_TAG;
-import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_WRITE_TAG;
-
-
-/**
- * Provide for remote table instances
- */
-public class RemoteTableProvider extends BaseTableProvider {
-
-  static final String READ_FN = "io.read.func";
-  static final String WRITE_FN = "io.write.func";
-  static final String RATE_LIMITER = "io.ratelimiter";
-  static final String READ_CREDIT_FN = "io.read.credit.func";
-  static final String WRITE_CREDIT_FN = "io.write.credit.func";
-  static final String ASYNC_CALLBACK_POOL_SIZE = "io.async.callback.pool.size";
-  static final String READ_RETRY_POLICY = "io.read.retry.policy";
-  static final String WRITE_RETRY_POLICY = "io.write.retry.policy";
-
-  private final boolean readOnly;
-  private final List<RemoteReadableTable<?, ?>> tables = new ArrayList<>();
-
-  /**
-   * Map of tableId -> executor service for async table IO and callbacks. The same executors
-   * are shared by both read/write operations such that tables of the same tableId all share
-   * the set same of executors globally whereas table itself is per-task.
-   */
-  private static Map<String, ExecutorService> tableExecutors = new ConcurrentHashMap<>();
-  private static Map<String, ExecutorService> callbackExecutors = new ConcurrentHashMap<>();
-  private static ScheduledExecutorService retryExecutor;
-
-  public RemoteTableProvider(TableSpec tableSpec) {
-    super(tableSpec);
-    this.readOnly = !tableSpec.getConfig().containsKey(WRITE_FN);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public Table getTable() {
-    RemoteReadableTable table;
-    String tableId = tableSpec.getId();
-
-    TableReadFunction readFn = getReadFn();
-    RateLimiter rateLimiter = deserializeObject(RATE_LIMITER);
-    if (rateLimiter != null) {
-      rateLimiter.init(this.context);
-    }
-    TableRateLimiter.CreditFunction<?, ?> readCreditFn = deserializeObject(READ_CREDIT_FN);
-    TableRateLimiter readRateLimiter = new TableRateLimiter(tableSpec.getId(), rateLimiter, readCreditFn, RL_READ_TAG);
-
-    TableRateLimiter.CreditFunction<?, ?> writeCreditFn;
-    TableRateLimiter writeRateLimiter = null;
-
-    TableRetryPolicy readRetryPolicy = deserializeObject(READ_RETRY_POLICY);
-    TableRetryPolicy writeRetryPolicy = null;
-
-    if ((readRetryPolicy != null || writeRetryPolicy != null) && retryExecutor == null) {
-      retryExecutor = Executors.newSingleThreadScheduledExecutor(runnable -> {
-          Thread thread = new Thread(runnable);
-          thread.setName("table-retry-executor");
-          thread.setDaemon(true);
-          return thread;
-        });
-    }
-
-    if (readRetryPolicy != null) {
-      readFn = new RetriableReadFunction<>(readRetryPolicy, readFn, retryExecutor);
-    }
-
-    TableWriteFunction writeFn = getWriteFn();
-
-    boolean isRateLimited = readRateLimiter.isRateLimited();
-    if (!readOnly) {
-      writeCreditFn = deserializeObject(WRITE_CREDIT_FN);
-      writeRateLimiter = new TableRateLimiter(tableSpec.getId(), rateLimiter, writeCreditFn, RL_WRITE_TAG);
-      isRateLimited |= writeRateLimiter.isRateLimited();
-      writeRetryPolicy = deserializeObject(WRITE_RETRY_POLICY);
-      if (writeRetryPolicy != null) {
-        writeFn = new RetriableWriteFunction(writeRetryPolicy, writeFn, retryExecutor);
-      }
-    }
-
-    // Optional executor for future callback/completion. Shared by both read and write operations.
-    int callbackPoolSize = Integer.parseInt(tableSpec.getConfig().get(ASYNC_CALLBACK_POOL_SIZE));
-    if (callbackPoolSize > 0) {
-      callbackExecutors.computeIfAbsent(tableId, (arg) ->
-          Executors.newFixedThreadPool(callbackPoolSize, (runnable) -> {
-              Thread thread = new Thread(runnable);
-              thread.setName("table-" + tableId + "-async-callback-pool");
-              thread.setDaemon(true);
-              return thread;
-            }));
-    }
-
-    if (isRateLimited) {
-      tableExecutors.computeIfAbsent(tableId, (arg) ->
-          Executors.newSingleThreadExecutor(runnable -> {
-              Thread thread = new Thread(runnable);
-              thread.setName("table-" + tableId + "-async-executor");
-              thread.setDaemon(true);
-              return thread;
-            }));
-    }
-
-    if (readOnly) {
-      table = new RemoteReadableTable(tableSpec.getId(), readFn, readRateLimiter,
-          tableExecutors.get(tableId), callbackExecutors.get(tableId));
-    } else {
-      table = new RemoteReadWriteTable(tableSpec.getId(), readFn, writeFn, readRateLimiter,
-          writeRateLimiter, tableExecutors.get(tableId), callbackExecutors.get(tableId));
-    }
-
-    TableMetricsUtil metricsUtil = new TableMetricsUtil(this.context, table, tableId);
-    if (readRetryPolicy != null) {
-      ((RetriableReadFunction) readFn).setMetrics(metricsUtil);
-    }
-    if (writeRetryPolicy != null) {
-      ((RetriableWriteFunction) writeFn).setMetrics(metricsUtil);
-    }
-
-    table.init(this.context);
-    tables.add(table);
-    return table;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void close() {
-    tables.forEach(t -> t.close());
-    tableExecutors.values().forEach(e -> e.shutdown());
-    callbackExecutors.values().forEach(e -> e.shutdown());
-  }
-
-  private <T> T deserializeObject(String key) {
-    String entry = tableSpec.getConfig().getOrDefault(key, "");
-    if (entry.isEmpty()) {
-      return null;
-    }
-    return SerdeUtils.deserialize(key, entry);
-  }
-
-  private TableReadFunction<?, ?> getReadFn() {
-    TableReadFunction<?, ?> readFn = deserializeObject(READ_FN);
-    if (readFn != null) {
-      readFn.init(this.context);
-    }
-    return readFn;
-  }
-
-  private TableWriteFunction<?, ?> getWriteFn() {
-    TableWriteFunction<?, ?> writeFn = deserializeObject(WRITE_FN);
-    if (writeFn != null) {
-      writeFn.init(this.context);
-    }
-    return writeFn;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java
deleted file mode 100644
index 0eb88fd..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java
+++ /dev/null
@@ -1,38 +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.table.remote;
-
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableProviderFactory;
-import org.apache.samza.table.TableSpec;
-
-import com.google.common.base.Preconditions;
-
-
-/**
- * Factory class for a remote table provider
- */
-public class RemoteTableProviderFactory implements TableProviderFactory {
-  @Override
-  public TableProvider getTableProvider(TableSpec tableSpec) {
-    Preconditions.checkNotNull(tableSpec, "null table spec");
-    return new RemoteTableProvider(tableSpec);
-  }
-}


[09/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsOutputDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsOutputDescriptor.java
deleted file mode 100644
index e59b4d0..0000000
--- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsOutputDescriptor.java
+++ /dev/null
@@ -1,104 +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.system.eventhub;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.config.ConfigException;
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for an Event Hubs output stream
- * <p>
- *   An instance of this descriptor may be obtained from and {@link EventHubsSystemDescriptor}
- * </p>
- * Stream properties configured using a descriptor overrides corresponding properties and property defaults provided
- * in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream
- */
-public class EventHubsOutputDescriptor<StreamMessageType>
-    extends OutputDescriptor<StreamMessageType, EventHubsOutputDescriptor<StreamMessageType>> {
-  private String namespace;
-  private String entityPath;
-  private Optional<String> sasKeyName = Optional.empty();
-  private Optional<String> sasToken = Optional.empty();
-
-  /**
-   * Constructs an {@link OutputDescriptor} instance.
-   *
-   * @param streamId id of the stream
-   * @param namespace namespace for the Event Hubs entity to produce to, not null
-   * @param entityPath entity path for the Event Hubs entity to produce to, not null
-   * @param serde serde for messages in the stream
-   * @param systemDescriptor system descriptor this stream descriptor was obtained from
-   */
-  EventHubsOutputDescriptor(String streamId, String namespace, String entityPath, Serde serde,
-      SystemDescriptor systemDescriptor) {
-    super(streamId, serde, systemDescriptor);
-    this.namespace = StringUtils.stripToNull(namespace);
-    this.entityPath = StringUtils.stripToNull(entityPath);
-    if (this.namespace == null || this.entityPath == null) {
-      throw new ConfigException(String.format("Missing namespace and entity path Event Hubs output descriptor in " //
-          + "system: {%s}, stream: {%s}", getSystemName(), streamId));
-    }
-  }
-
-  /**
-   * SAS Key name of the associated output stream. Required to access the output Event Hubs entity per stream.
-   *
-   * @param sasKeyName the name of the SAS key required to access the Event Hubs entity
-   * @return this output descriptor
-   */
-  public EventHubsOutputDescriptor<StreamMessageType> withSasKeyName(String sasKeyName) {
-    this.sasKeyName = Optional.of(StringUtils.stripToNull(sasKeyName));
-    return this;
-  }
-
-  /**
-   * SAS Token of the associated output stream. Required to access the output Event Hubs per stream.
-   *
-   * @param sasToken the SAS token required to access to Event Hubs entity
-   * @return this output descriptor
-   */
-  public EventHubsOutputDescriptor<StreamMessageType> withSasKey(String sasToken) {
-    this.sasToken = Optional.of(StringUtils.stripToNull(sasToken));
-    return this;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    HashMap<String, String> ehConfigs = new HashMap<>(super.toConfig());
-
-    String streamId = getStreamId();
-
-    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId), namespace);
-    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId), entityPath);
-    sasKeyName.ifPresent(keyName ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId), keyName));
-    sasToken.ifPresent(key ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId), key));
-    return ehConfigs;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsSystemDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsSystemDescriptor.java
deleted file mode 100644
index 189340f..0000000
--- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsSystemDescriptor.java
+++ /dev/null
@@ -1,217 +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.system.eventhub;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod;
-
-
-/**
- * A descriptor for a Event Hubs system.
- * <p>
- * System properties configured using a descriptor override corresponding properties provided in configuration.
- */
-public class EventHubsSystemDescriptor extends SystemDescriptor<EventHubsSystemDescriptor> {
-  private static final String FACTORY_CLASS_NAME = EventHubSystemFactory.class.getName();
-
-  private List<String> streamIds = new ArrayList<>();
-  private Optional<Integer> fetchRuntimeInfoTimeout = Optional.empty();
-  private Optional<Integer> numClientThreads = Optional.empty();
-  private Optional<Integer> consumerReceiveQueueSize = Optional.empty();
-  private Optional<Integer> consumerMaxEventCountPerPoll = Optional.empty();
-  private Optional<Integer> consumerPrefetchCount = Optional.empty();
-  private Optional<Boolean> producerEventhubsSendKey = Optional.empty();
-  private Optional<PartitioningMethod> producerEventhubsPartitioningMethod = Optional.empty();
-
-  /**
-   * Constructs a {@link SystemDescriptor} instance.
-   *  @param systemName name of this system
-   */
-  public EventHubsSystemDescriptor(String systemName) {
-    super(systemName, FACTORY_CLASS_NAME, null, null);
-  }
-
-  /**
-   * Gets an {@link EventHubsInputDescriptor} for the input stream of this system. The stream has the provided
-   * namespace and entity name of the associated Event Hubs entity and the provided stream level serde.
-   * <p>
-   * The type of messages in the stream is the type of the provided stream level serde.
-   *
-   * @param streamId id of the input stream
-   * @param namespace namespace of the Event Hubs entity to consume from
-   * @param entityPath entity path of the Event Hubs entity to consume from
-   * @param serde stream level serde for the input stream
-   * @param <StreamMessageType> type of messages in this stream
-   * @return an {@link EventHubsInputDescriptor} for the Event Hubs input stream
-   */
-  public <StreamMessageType> EventHubsInputDescriptor<StreamMessageType> getInputDescriptor(String streamId, String namespace,
-      String entityPath, Serde<StreamMessageType> serde) {
-    streamIds.add(streamId);
-    return new EventHubsInputDescriptor<>(streamId, namespace, entityPath, serde, this);
-  }
-
-  /**
-   * Gets an {@link EventHubsOutputDescriptor} for the output stream of this system. The stream has the provided
-   * namespace and entity name of the associated Event Hubs entity and the provided stream level serde.
-   * <p>
-   * The type of messages in the stream is the type of the provided stream level serde.
-   *
-   * @param streamId id of the output stream
-   * @param namespace namespace of the Event Hubs entity to produce to
-   * @param entityPath entity path of the Event Hubs entity to produce to
-   * @param serde stream level serde for the output stream
-   * @param <StreamMessageType> type of the messages in this stream
-   * @return an {@link EventHubsOutputDescriptor} for the Event Hubs output stream
-   */
-  public <StreamMessageType> EventHubsOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, String namespace,
-      String entityPath, Serde<StreamMessageType> serde) {
-    streamIds.add(streamId);
-    return new EventHubsOutputDescriptor<>(streamId, namespace, entityPath, serde, this);
-  }
-
-  /**
-   * Timeout for fetching the runtime metadata from an Event Hubs entity on startup in millis.
-   *
-   * @param timeoutMS the timeout in ms for getting runtime information from the Event Hubs system
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withRuntimeInfoTimeout(int timeoutMS) {
-    this.fetchRuntimeInfoTimeout = Optional.of(timeoutMS);
-    return this;
-  }
-
-  /**
-   * Number of threads in thread pool that will be used by the EventHubClient.
-   *
-   * @param numClientThreads the number of threads
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withNumClientThreads(int numClientThreads) {
-    this.numClientThreads = Optional.of(numClientThreads);
-    return this;
-  }
-
-  /**
-   *  Per partition capacity of the Event Hubs consumer buffer - the blocking queue used for storing messages.
-   *  Larger buffer capacity typically leads to better throughput but consumes more memory.
-   *
-   * @param receiveQueueSize the number of messages from Event Hubs that should be buffered in the
-   *                      {@link org.apache.samza.util.BlockingEnvelopeMap}
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withReceiveQueueSize(int receiveQueueSize) {
-    this.consumerReceiveQueueSize = Optional.of(receiveQueueSize);
-    return this;
-  }
-
-  /**
-   * Maximum number of events that Event Hubs client can return in a receive call.
-   *
-   * @param count the number of max events per poll
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withMaxEventCountPerPoll(int count) {
-    this.consumerMaxEventCountPerPoll = Optional.of(count);
-    return this;
-  }
-
-  /**
-   * Number of events that Event Hubs client should prefetch from the server.
-   *
-   * @param count the number of events that should be prefetched.
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withPrefetchCount(int count) {
-    this.consumerPrefetchCount = Optional.of(count);
-    return this;
-  }
-
-
-  /**
-   * Configure the method that the message is partitioned for the downstream Event Hubs in one of the following ways:
-   * <ul>
-   *   <li>ROUND_ROBIN:
-   *   The message key and partition key are ignored and the message
-   *   will be distributed in a round-robin fashion amongst all the partitions in the downstream Event Hubs entity.</li>
-   *   <li>EVENT_HUB_HASHING:
-   *   Employs the hashing mechanism in Event Hubs to determine, based on the key of the message,
-   *   which partition the message should go. Using this method still ensures that all the events with
-   *   the same key are sent to the same partition in the event hub. If this option is chosen, the partition
-   *   key used for the hash should be a string. If the partition key is not set, the message key is
-   *   used instead.</li>
-   *   <li>PARTITION_KEY_AS_PARTITION:
-   *   Use the integer key specified by the partition key or key of the message to a specific partition
-   *   on Event Hubs. If the integer key is greater than the number of partitions in the destination Event Hubs entity,
-   *   a modulo operation will be performed to determine the resulting paritition.
-   *   ie. if there are 6 partitions and the key is 9, the message will end up in partition 3.
-   *   Similarly to EVENT_HUB_HASHING, if the partition key is not set the message key is used instead.</li>
-   * </ul>
-   * @param partitioningMethod the desired partitioning method for the message in the downstream Event Hubs entity
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withPartitioningMethod(PartitioningMethod partitioningMethod) {
-    this.producerEventhubsPartitioningMethod = Optional.ofNullable(partitioningMethod);
-    return this;
-  }
-
-  /**
-   *  If set to true, the key of the Samza message will be included as the 'key' property in the outgoing EventData
-   *  message for Event Hubs. The Samza message key will not be sent otherwise.
-   *  Note: If the Samza Event Hubs consumer is used, this field is the partition key of the received EventData, or the
-   *  message key if the partition key is not present.
-   *
-   * @param sendKeys set to true if the message key should be sent in the EventData properties, the key is not sent otherwise
-   * @return this system descriptor
-   */
-  public EventHubsSystemDescriptor withSendKeys(boolean sendKeys) {
-    this.producerEventhubsSendKey = Optional.of(sendKeys);
-    return this;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    Map<String, String> ehConfigs = new HashMap<>(super.toConfig());
-    String systemName = getSystemName();
-
-    if (!this.streamIds.isEmpty()) {
-      ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), String.join(",", this.streamIds));
-    }
-    this.fetchRuntimeInfoTimeout.ifPresent(timeout ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName), Integer.toString(timeout)));
-    this.numClientThreads.ifPresent(numClientThreads ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_SYSTEM_NUM_CLIENT_THREADS, systemName), Integer.toString(numClientThreads)));
-    this.consumerReceiveQueueSize.ifPresent(receiveQueueSize ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_CONSUMER_BUFFER_CAPACITY, systemName), Integer.toString(receiveQueueSize)));
-    this.consumerMaxEventCountPerPoll.ifPresent(count ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_MAX_EVENT_COUNT_PER_POLL, systemName), Integer.toString(count)));
-    this.consumerPrefetchCount.ifPresent(count ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_PREFETCH_COUNT, systemName), Integer.toString(count)));
-    this.producerEventhubsSendKey.ifPresent(sendKeys ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName), Boolean.toString(sendKeys)));
-    this.producerEventhubsPartitioningMethod.ifPresent(partitioningMethod ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), partitioningMethod.toString()));
-    return ehConfigs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java
new file mode 100644
index 0000000..cce716c
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.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.system.eventhub.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.eventhub.EventHubConfig;
+
+
+/**
+ * A descriptor for the Event Hubs output stream
+ *<p>
+ *   An instance of this descriptor may be obtained from an {@link EventHubsSystemDescriptor}
+ *</p>
+ * Stream properties configured using a descriptor overrides corresponding properties and property defaults provided
+ * in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream
+ */
+public class EventHubsInputDescriptor<StreamMessageType>
+    extends InputDescriptor<StreamMessageType, EventHubsInputDescriptor<StreamMessageType>> {
+  private String namespace;
+  private String entityPath;
+  private Optional<String> sasKeyName = Optional.empty();
+  private Optional<String> sasToken = Optional.empty();
+  private Optional<String> consumerGroup = Optional.empty();
+
+  /**
+   * Constructs an {@link InputDescriptor} instance.
+   *
+   * @param streamId id of the stream
+   * @param namespace namespace for the Event Hubs entity to consume from, not null
+   * @param entityPath entity path for the Event Hubs entity to consume from, not null
+   * @param serde serde for messages in the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   */
+  EventHubsInputDescriptor(String streamId, String namespace, String entityPath, Serde serde,
+      SystemDescriptor systemDescriptor) {
+    super(streamId, serde, systemDescriptor, null);
+    this.namespace = StringUtils.stripToNull(namespace);
+    this.entityPath = StringUtils.stripToNull(entityPath);
+    if (this.namespace == null || this.entityPath == null) {
+      throw new ConfigException(String.format("Missing namespace and entity path Event Hubs input descriptor in " //
+          + "system: {%s}, stream: {%s}", getSystemName(), streamId));
+    }
+  }
+
+  /**
+   * SAS Key name of the associated input stream. Required to access the input Event Hubs entity per stream.
+   *
+   * @param sasKeyName the name of the SAS key required to access the Event Hubs entity
+   * @return this input descriptor
+   */
+  public EventHubsInputDescriptor<StreamMessageType> withSasKeyName(String sasKeyName) {
+    this.sasKeyName = Optional.of(StringUtils.stripToNull(sasKeyName));
+    return this;
+  }
+
+  /**
+   * SAS Token of the associated input stream. Required to access the input Event Hubs per stream.
+   *
+   * @param sasToken the SAS token required to access the Event Hubs entity
+   * @return this input descriptor
+   */
+  public EventHubsInputDescriptor<StreamMessageType> withSasKey(String sasToken) {
+    this.sasToken = Optional.of(StringUtils.stripToNull(sasToken));
+    return this;
+  }
+
+  /**
+   * Set the consumer group from the upstream Event Hubs entity that the consumer is part of. Defaults to the
+   * <code>$Default</code> group that is initially present in all Event Hubs entities (unless removed)
+   *
+   * @param consumerGroup the name of the consumer group upstream
+   * @return this input descriptor
+   */
+  public EventHubsInputDescriptor<StreamMessageType> withConsumerGroup(String consumerGroup) {
+    this.consumerGroup = Optional.of(StringUtils.stripToNull(consumerGroup));
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    HashMap<String, String> ehConfigs = new HashMap<>(super.toConfig());
+
+    String streamId = getStreamId();
+
+    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId), namespace);
+    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId), entityPath);
+
+    sasKeyName.ifPresent(keyName ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId), keyName));
+    sasToken.ifPresent(key ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId), key));
+    this.consumerGroup.ifPresent(consumerGroupName ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId), consumerGroupName));
+    return ehConfigs;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java
new file mode 100644
index 0000000..cd17033
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.eventhub.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.eventhub.EventHubConfig;
+
+/**
+ * A descriptor for an Event Hubs output stream
+ * <p>
+ *   An instance of this descriptor may be obtained from and {@link EventHubsSystemDescriptor}
+ * </p>
+ * Stream properties configured using a descriptor overrides corresponding properties and property defaults provided
+ * in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream
+ */
+public class EventHubsOutputDescriptor<StreamMessageType>
+    extends OutputDescriptor<StreamMessageType, EventHubsOutputDescriptor<StreamMessageType>> {
+  private String namespace;
+  private String entityPath;
+  private Optional<String> sasKeyName = Optional.empty();
+  private Optional<String> sasToken = Optional.empty();
+
+  /**
+   * Constructs an {@link OutputDescriptor} instance.
+   *
+   * @param streamId id of the stream
+   * @param namespace namespace for the Event Hubs entity to produce to, not null
+   * @param entityPath entity path for the Event Hubs entity to produce to, not null
+   * @param serde serde for messages in the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   */
+  EventHubsOutputDescriptor(String streamId, String namespace, String entityPath, Serde serde,
+      SystemDescriptor systemDescriptor) {
+    super(streamId, serde, systemDescriptor);
+    this.namespace = StringUtils.stripToNull(namespace);
+    this.entityPath = StringUtils.stripToNull(entityPath);
+    if (this.namespace == null || this.entityPath == null) {
+      throw new ConfigException(String.format("Missing namespace and entity path Event Hubs output descriptor in " //
+          + "system: {%s}, stream: {%s}", getSystemName(), streamId));
+    }
+  }
+
+  /**
+   * SAS Key name of the associated output stream. Required to access the output Event Hubs entity per stream.
+   *
+   * @param sasKeyName the name of the SAS key required to access the Event Hubs entity
+   * @return this output descriptor
+   */
+  public EventHubsOutputDescriptor<StreamMessageType> withSasKeyName(String sasKeyName) {
+    this.sasKeyName = Optional.of(StringUtils.stripToNull(sasKeyName));
+    return this;
+  }
+
+  /**
+   * SAS Token of the associated output stream. Required to access the output Event Hubs per stream.
+   *
+   * @param sasToken the SAS token required to access to Event Hubs entity
+   * @return this output descriptor
+   */
+  public EventHubsOutputDescriptor<StreamMessageType> withSasKey(String sasToken) {
+    this.sasToken = Optional.of(StringUtils.stripToNull(sasToken));
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    HashMap<String, String> ehConfigs = new HashMap<>(super.toConfig());
+
+    String streamId = getStreamId();
+
+    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId), namespace);
+    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId), entityPath);
+    sasKeyName.ifPresent(keyName ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId), keyName));
+    sasToken.ifPresent(key ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId), key));
+    return ehConfigs;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java
new file mode 100644
index 0000000..4e292d9
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.eventhub.descriptors;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.eventhub.EventHubConfig;
+import org.apache.samza.system.eventhub.EventHubSystemFactory;
+import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod;
+
+
+/**
+ * A descriptor for a Event Hubs system.
+ * <p>
+ * System properties configured using a descriptor override corresponding properties provided in configuration.
+ */
+public class EventHubsSystemDescriptor extends SystemDescriptor<EventHubsSystemDescriptor> {
+  private static final String FACTORY_CLASS_NAME = EventHubSystemFactory.class.getName();
+
+  private List<String> streamIds = new ArrayList<>();
+  private Optional<Integer> fetchRuntimeInfoTimeout = Optional.empty();
+  private Optional<Integer> numClientThreads = Optional.empty();
+  private Optional<Integer> consumerReceiveQueueSize = Optional.empty();
+  private Optional<Integer> consumerMaxEventCountPerPoll = Optional.empty();
+  private Optional<Integer> consumerPrefetchCount = Optional.empty();
+  private Optional<Boolean> producerEventhubsSendKey = Optional.empty();
+  private Optional<PartitioningMethod> producerEventhubsPartitioningMethod = Optional.empty();
+
+  /**
+   * Constructs a {@link SystemDescriptor} instance.
+   *  @param systemName name of this system
+   */
+  public EventHubsSystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME, null, null);
+  }
+
+  /**
+   * Gets an {@link EventHubsInputDescriptor} for the input stream of this system. The stream has the provided
+   * namespace and entity name of the associated Event Hubs entity and the provided stream level serde.
+   * <p>
+   * The type of messages in the stream is the type of the provided stream level serde.
+   *
+   * @param streamId id of the input stream
+   * @param namespace namespace of the Event Hubs entity to consume from
+   * @param entityPath entity path of the Event Hubs entity to consume from
+   * @param serde stream level serde for the input stream
+   * @param <StreamMessageType> type of messages in this stream
+   * @return an {@link EventHubsInputDescriptor} for the Event Hubs input stream
+   */
+  public <StreamMessageType> EventHubsInputDescriptor<StreamMessageType> getInputDescriptor(String streamId, String namespace,
+      String entityPath, Serde<StreamMessageType> serde) {
+    streamIds.add(streamId);
+    return new EventHubsInputDescriptor<>(streamId, namespace, entityPath, serde, this);
+  }
+
+  /**
+   * Gets an {@link EventHubsOutputDescriptor} for the output stream of this system. The stream has the provided
+   * namespace and entity name of the associated Event Hubs entity and the provided stream level serde.
+   * <p>
+   * The type of messages in the stream is the type of the provided stream level serde.
+   *
+   * @param streamId id of the output stream
+   * @param namespace namespace of the Event Hubs entity to produce to
+   * @param entityPath entity path of the Event Hubs entity to produce to
+   * @param serde stream level serde for the output stream
+   * @param <StreamMessageType> type of the messages in this stream
+   * @return an {@link EventHubsOutputDescriptor} for the Event Hubs output stream
+   */
+  public <StreamMessageType> EventHubsOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, String namespace,
+      String entityPath, Serde<StreamMessageType> serde) {
+    streamIds.add(streamId);
+    return new EventHubsOutputDescriptor<>(streamId, namespace, entityPath, serde, this);
+  }
+
+  /**
+   * Timeout for fetching the runtime metadata from an Event Hubs entity on startup in millis.
+   *
+   * @param timeoutMS the timeout in ms for getting runtime information from the Event Hubs system
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withRuntimeInfoTimeout(int timeoutMS) {
+    this.fetchRuntimeInfoTimeout = Optional.of(timeoutMS);
+    return this;
+  }
+
+  /**
+   * Number of threads in thread pool that will be used by the EventHubClient.
+   *
+   * @param numClientThreads the number of threads
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withNumClientThreads(int numClientThreads) {
+    this.numClientThreads = Optional.of(numClientThreads);
+    return this;
+  }
+
+  /**
+   *  Per partition capacity of the Event Hubs consumer buffer - the blocking queue used for storing messages.
+   *  Larger buffer capacity typically leads to better throughput but consumes more memory.
+   *
+   * @param receiveQueueSize the number of messages from Event Hubs that should be buffered in the
+   *                      {@link org.apache.samza.util.BlockingEnvelopeMap}
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withReceiveQueueSize(int receiveQueueSize) {
+    this.consumerReceiveQueueSize = Optional.of(receiveQueueSize);
+    return this;
+  }
+
+  /**
+   * Maximum number of events that Event Hubs client can return in a receive call.
+   *
+   * @param count the number of max events per poll
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withMaxEventCountPerPoll(int count) {
+    this.consumerMaxEventCountPerPoll = Optional.of(count);
+    return this;
+  }
+
+  /**
+   * Number of events that Event Hubs client should prefetch from the server.
+   *
+   * @param count the number of events that should be prefetched.
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withPrefetchCount(int count) {
+    this.consumerPrefetchCount = Optional.of(count);
+    return this;
+  }
+
+
+  /**
+   * Configure the method that the message is partitioned for the downstream Event Hubs in one of the following ways:
+   * <ul>
+   *   <li>ROUND_ROBIN:
+   *   The message key and partition key are ignored and the message
+   *   will be distributed in a round-robin fashion amongst all the partitions in the downstream Event Hubs entity.</li>
+   *   <li>EVENT_HUB_HASHING:
+   *   Employs the hashing mechanism in Event Hubs to determine, based on the key of the message,
+   *   which partition the message should go. Using this method still ensures that all the events with
+   *   the same key are sent to the same partition in the event hub. If this option is chosen, the partition
+   *   key used for the hash should be a string. If the partition key is not set, the message key is
+   *   used instead.</li>
+   *   <li>PARTITION_KEY_AS_PARTITION:
+   *   Use the integer key specified by the partition key or key of the message to a specific partition
+   *   on Event Hubs. If the integer key is greater than the number of partitions in the destination Event Hubs entity,
+   *   a modulo operation will be performed to determine the resulting paritition.
+   *   ie. if there are 6 partitions and the key is 9, the message will end up in partition 3.
+   *   Similarly to EVENT_HUB_HASHING, if the partition key is not set the message key is used instead.</li>
+   * </ul>
+   * @param partitioningMethod the desired partitioning method for the message in the downstream Event Hubs entity
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withPartitioningMethod(PartitioningMethod partitioningMethod) {
+    this.producerEventhubsPartitioningMethod = Optional.ofNullable(partitioningMethod);
+    return this;
+  }
+
+  /**
+   *  If set to true, the key of the Samza message will be included as the 'key' property in the outgoing EventData
+   *  message for Event Hubs. The Samza message key will not be sent otherwise.
+   *  Note: If the Samza Event Hubs consumer is used, this field is the partition key of the received EventData, or the
+   *  message key if the partition key is not present.
+   *
+   * @param sendKeys set to true if the message key should be sent in the EventData properties, the key is not sent otherwise
+   * @return this system descriptor
+   */
+  public EventHubsSystemDescriptor withSendKeys(boolean sendKeys) {
+    this.producerEventhubsSendKey = Optional.of(sendKeys);
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    Map<String, String> ehConfigs = new HashMap<>(super.toConfig());
+    String systemName = getSystemName();
+
+    if (!this.streamIds.isEmpty()) {
+      ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), String.join(",", this.streamIds));
+    }
+    this.fetchRuntimeInfoTimeout.ifPresent(timeout ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName), Integer.toString(timeout)));
+    this.numClientThreads.ifPresent(numClientThreads ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_SYSTEM_NUM_CLIENT_THREADS, systemName), Integer.toString(numClientThreads)));
+    this.consumerReceiveQueueSize.ifPresent(receiveQueueSize ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_CONSUMER_BUFFER_CAPACITY, systemName), Integer.toString(receiveQueueSize)));
+    this.consumerMaxEventCountPerPoll.ifPresent(count ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_MAX_EVENT_COUNT_PER_POLL, systemName), Integer.toString(count)));
+    this.consumerPrefetchCount.ifPresent(count ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_PREFETCH_COUNT, systemName), Integer.toString(count)));
+    this.producerEventhubsSendKey.ifPresent(sendKeys ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName), Boolean.toString(sendKeys)));
+    this.producerEventhubsPartitioningMethod.ifPresent(partitioningMethod ->
+        ehConfigs.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), partitioningMethod.toString()));
+    return ehConfigs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsInputDescriptor.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsInputDescriptor.java
deleted file mode 100644
index b3003a3..0000000
--- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsInputDescriptor.java
+++ /dev/null
@@ -1,91 +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.system.eventhub;
-
-import java.util.Map;
-import org.apache.samza.config.ConfigException;
-import org.apache.samza.operators.KV;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-
-public class TestEventHubsInputDescriptor {
-  @Test
-  public void testEntityConnectionConfigs() {
-    String systemName = "eventHub";
-    String streamId = "input-stream";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-
-    EventHubsInputDescriptor<KV<String, Integer>> inputDescriptor = systemDescriptor
-        .getInputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()))
-        .withSasKeyName("secretkey")
-        .withSasKey("sasToken-123")
-        .withConsumerGroup("$notdefault");
-
-    Map<String, String> generatedConfigs = inputDescriptor.toConfig();
-    assertEquals("eventHub", generatedConfigs.get("streams.input-stream.samza.system"));
-    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
-    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
-    assertEquals("secretkey", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
-    assertEquals("sasToken-123", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
-    assertEquals("$notdefault", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId)));
-  }
-
-  @Test
-  public void testWithoutEntityConnectionConfigs() {
-    String systemName = "eventHub";
-    String streamId = "input-stream";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-
-    EventHubsInputDescriptor<KV<String, Integer>> inputDescriptor = systemDescriptor
-        .getInputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()));
-
-    Map<String, String> generatedConfigs = inputDescriptor.toConfig();
-    assertEquals("eventHub", generatedConfigs.get("streams.input-stream.samza.system"));
-    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
-    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId)));
-    assertEquals(3, generatedConfigs.size()); // verify that there are no other configs
-  }
-
-  @Test
-  public void testMissingInputDescriptorFields() {
-    String systemName = "eventHub";
-    String streamId = "input-stream";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-    try {
-      systemDescriptor.getInputDescriptor(streamId, null, null, KVSerde.of(new StringSerde(), new IntegerSerde()));
-      fail("Should have thrown Config Exception");
-    } catch (ConfigException exception) {
-      assertEquals(String.format("Missing namespace and entity path Event Hubs input descriptor in " //
-          + "system: {%s}, stream: {%s}", systemName, streamId), exception.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsOutputDescriptor.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsOutputDescriptor.java
deleted file mode 100644
index fcfcdca..0000000
--- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsOutputDescriptor.java
+++ /dev/null
@@ -1,88 +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.system.eventhub;
-
-import java.util.Map;
-import org.apache.samza.config.ConfigException;
-import org.apache.samza.operators.KV;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-public class TestEventHubsOutputDescriptor {
-  @Test
-  public void testEntityConnectionConfigs() {
-    String systemName = "eventHub";
-    String streamId = "output-stream";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-
-    EventHubsOutputDescriptor<KV<String, Integer>> outputDescriptor = systemDescriptor
-        .getOutputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()))
-        .withSasKeyName("secretkey")
-        .withSasKey("sasToken-123");
-
-    Map<String, String> generatedConfigs = outputDescriptor.toConfig();
-    assertEquals("eventHub", generatedConfigs.get("streams.output-stream.samza.system"));
-    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
-    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
-    assertEquals("secretkey", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
-    assertEquals("sasToken-123", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
-  }
-
-  @Test
-  public void testWithoutEntityConnectionConfigs() {
-    String systemName = "eventHub";
-    String streamId = "output-stream";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-
-    EventHubsOutputDescriptor<KV<String, Integer>> outputDescriptor = systemDescriptor
-        .getOutputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()));
-
-    Map<String, String> generatedConfigs = outputDescriptor.toConfig();
-    assertEquals("eventHub", generatedConfigs.get("streams.output-stream.samza.system"));
-    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
-    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId)));
-    assertEquals(3, generatedConfigs.size()); // verify that there are no other configs
-  }
-
-  @Test
-  public void testMissingOutputDescriptorFields() {
-    String systemName = "eventHub";
-    String streamId = "input-stream";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-    try {
-      systemDescriptor.getOutputDescriptor(streamId, null, null, KVSerde.of(new StringSerde(), new IntegerSerde()));
-      fail("Should have thrown Config Exception");
-    } catch (ConfigException exception) {
-      assertEquals(String.format("Missing namespace and entity path Event Hubs output descriptor in " //
-          + "system: {%s}, stream: {%s}", systemName, streamId), exception.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsSystemDescriptor.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsSystemDescriptor.java
deleted file mode 100644
index 33bb1ba..0000000
--- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestEventHubsSystemDescriptor.java
+++ /dev/null
@@ -1,112 +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.system.eventhub;
-
-import java.util.Map;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-public class TestEventHubsSystemDescriptor {
-  @Test
-  public void testWithDescriptorOverrides() {
-    String systemName = "system-name";
-    String streamId1 = "input-stream1";
-    String streamId2 = "input-stream2";
-    String streamId3 = "output-stream1";
-    String streamId4 = "output-stream2";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName)
-        .withMaxEventCountPerPoll(1000)
-        .withNumClientThreads(5)
-        .withPartitioningMethod(PartitioningMethod.PARTITION_KEY_AS_PARTITION)
-        .withPrefetchCount(100)
-        .withReceiveQueueSize(500)
-        .withRuntimeInfoTimeout(60000)
-        .withSendKeys(false);
-
-    systemDescriptor.getInputDescriptor(streamId1, "entity-namespace1", "entity1", KVSerde.of(new StringSerde(),
-        new IntegerSerde()));
-    systemDescriptor.getInputDescriptor(streamId2, "entity-namespace2", "entity2", KVSerde.of(new StringSerde(),
-            new IntegerSerde()));
-    systemDescriptor.getOutputDescriptor(streamId3, "entity-namespace3", "entity3", KVSerde.of(new StringSerde(),
-            new IntegerSerde()));
-    systemDescriptor.getOutputDescriptor(streamId4, "entity-namespace4", "entity4", KVSerde.of(new StringSerde(),
-            new IntegerSerde()));
-
-    Map<String, String> generatedConfigs = systemDescriptor.toConfig();
-    assertEquals("org.apache.samza.system.eventhub.EventHubSystemFactory", generatedConfigs.get(String.format("systems.%s.samza.factory", systemName)));
-    assertEquals("1000", generatedConfigs.get(String.format(EventHubConfig.CONFIG_MAX_EVENT_COUNT_PER_POLL, systemName)));
-    assertEquals("5", generatedConfigs.get(String.format(EventHubConfig.CONFIG_SYSTEM_NUM_CLIENT_THREADS, systemName)));
-    assertEquals("PARTITION_KEY_AS_PARTITION", generatedConfigs.get(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName)));
-    assertEquals("100", generatedConfigs.get(String.format(EventHubConfig.CONFIG_PREFETCH_COUNT, systemName)));
-    assertEquals("500", generatedConfigs.get(String.format(EventHubConfig.CONFIG_CONSUMER_BUFFER_CAPACITY, systemName)));
-    assertEquals("60000", generatedConfigs.get(String.format(EventHubConfig.CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName)));
-    assertEquals("false", generatedConfigs.get(String.format(EventHubConfig.CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName)));
-    assertEquals(streamId1 + "," + streamId2 + "," + streamId3 + "," + streamId4, generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName)));
-  }
-
-  @Test
-  public void testWithoutDescriptorOverrides() {
-    String systemName = "eventHub";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-
-    Map<String, String> generatedConfigs = systemDescriptor.toConfig();
-    assertEquals("org.apache.samza.system.eventhub.EventHubSystemFactory", generatedConfigs.get(String.format("systems.%s.samza.factory", systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_MAX_EVENT_COUNT_PER_POLL, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_SYSTEM_NUM_CLIENT_THREADS, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_PREFETCH_COUNT, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_CONSUMER_BUFFER_CAPACITY, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName)));
-    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName)));
-    assertEquals(1, generatedConfigs.size());
-  }
-  @Test
-  public void testWithInputOutputStreams() {
-    String systemName = "system-name";
-    String streamId1 = "input-stream1";
-    String streamId2 = "input-stream2";
-    String streamId3 = "output-stream1";
-    String streamId4 = "output-stream2";
-
-    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
-
-    systemDescriptor.getInputDescriptor(streamId1, "entity-namespace1", "entity1", KVSerde.of(new StringSerde(),
-        new IntegerSerde()));
-    systemDescriptor.getInputDescriptor(streamId2, "entity-namespace2", "entity2", KVSerde.of(new StringSerde(),
-        new IntegerSerde()));
-    systemDescriptor.getOutputDescriptor(streamId3, "entity-namespace3", "entity3", KVSerde.of(new StringSerde(),
-        new IntegerSerde()));
-    systemDescriptor.getOutputDescriptor(streamId4, "entity-namespace4", "entity4", KVSerde.of(new StringSerde(),
-        new IntegerSerde()));
-
-    Map<String, String> generatedConfigs = systemDescriptor.toConfig();
-    assertEquals("org.apache.samza.system.eventhub.EventHubSystemFactory", generatedConfigs.get(String.format("systems.%s.samza.factory", systemName)));
-    assertEquals(streamId1 + "," + streamId2 + "," + streamId3 + "," + streamId4, generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName)));
-    assertEquals(2, generatedConfigs.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsInputDescriptor.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsInputDescriptor.java
new file mode 100644
index 0000000..1e6b368
--- /dev/null
+++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsInputDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.eventhub.descriptors;
+
+import java.util.Map;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.operators.KV;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.eventhub.EventHubConfig;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+
+public class TestEventHubsInputDescriptor {
+  @Test
+  public void testEntityConnectionConfigs() {
+    String systemName = "eventHub";
+    String streamId = "input-stream";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+
+    EventHubsInputDescriptor<KV<String, Integer>> inputDescriptor = systemDescriptor
+        .getInputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()))
+        .withSasKeyName("secretkey")
+        .withSasKey("sasToken-123")
+        .withConsumerGroup("$notdefault");
+
+    Map<String, String> generatedConfigs = inputDescriptor.toConfig();
+    assertEquals("eventHub", generatedConfigs.get("streams.input-stream.samza.system"));
+    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
+    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
+    assertEquals("secretkey", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
+    assertEquals("sasToken-123", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
+    assertEquals("$notdefault", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId)));
+  }
+
+  @Test
+  public void testWithoutEntityConnectionConfigs() {
+    String systemName = "eventHub";
+    String streamId = "input-stream";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+
+    EventHubsInputDescriptor<KV<String, Integer>> inputDescriptor = systemDescriptor
+        .getInputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()));
+
+    Map<String, String> generatedConfigs = inputDescriptor.toConfig();
+    assertEquals("eventHub", generatedConfigs.get("streams.input-stream.samza.system"));
+    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
+    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId)));
+    assertEquals(3, generatedConfigs.size()); // verify that there are no other configs
+  }
+
+  @Test
+  public void testMissingInputDescriptorFields() {
+    String systemName = "eventHub";
+    String streamId = "input-stream";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+    try {
+      systemDescriptor.getInputDescriptor(streamId, null, null, KVSerde.of(new StringSerde(), new IntegerSerde()));
+      fail("Should have thrown Config Exception");
+    } catch (ConfigException exception) {
+      assertEquals(String.format("Missing namespace and entity path Event Hubs input descriptor in " //
+          + "system: {%s}, stream: {%s}", systemName, streamId), exception.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsOutputDescriptor.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsOutputDescriptor.java
new file mode 100644
index 0000000..fa8ae56
--- /dev/null
+++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsOutputDescriptor.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.eventhub.descriptors;
+
+import java.util.Map;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.operators.KV;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.eventhub.EventHubConfig;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+public class TestEventHubsOutputDescriptor {
+  @Test
+  public void testEntityConnectionConfigs() {
+    String systemName = "eventHub";
+    String streamId = "output-stream";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+
+    EventHubsOutputDescriptor<KV<String, Integer>> outputDescriptor = systemDescriptor
+        .getOutputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()))
+        .withSasKeyName("secretkey")
+        .withSasKey("sasToken-123");
+
+    Map<String, String> generatedConfigs = outputDescriptor.toConfig();
+    assertEquals("eventHub", generatedConfigs.get("streams.output-stream.samza.system"));
+    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
+    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
+    assertEquals("secretkey", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
+    assertEquals("sasToken-123", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
+  }
+
+  @Test
+  public void testWithoutEntityConnectionConfigs() {
+    String systemName = "eventHub";
+    String streamId = "output-stream";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+
+    EventHubsOutputDescriptor<KV<String, Integer>> outputDescriptor = systemDescriptor
+        .getOutputDescriptor(streamId, "entity-namespace", "entity3", KVSerde.of(new StringSerde(), new IntegerSerde()));
+
+    Map<String, String> generatedConfigs = outputDescriptor.toConfig();
+    assertEquals("eventHub", generatedConfigs.get("streams.output-stream.samza.system"));
+    assertEquals("entity-namespace", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId)));
+    assertEquals("entity3", generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId)));
+    assertEquals(3, generatedConfigs.size()); // verify that there are no other configs
+  }
+
+  @Test
+  public void testMissingOutputDescriptorFields() {
+    String systemName = "eventHub";
+    String streamId = "input-stream";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+    try {
+      systemDescriptor.getOutputDescriptor(streamId, null, null, KVSerde.of(new StringSerde(), new IntegerSerde()));
+      fail("Should have thrown Config Exception");
+    } catch (ConfigException exception) {
+      assertEquals(String.format("Missing namespace and entity path Event Hubs output descriptor in " //
+          + "system: {%s}, stream: {%s}", systemName, streamId), exception.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsSystemDescriptor.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsSystemDescriptor.java
new file mode 100644
index 0000000..7f73bd9
--- /dev/null
+++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/descriptors/TestEventHubsSystemDescriptor.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.eventhub.descriptors;
+
+import java.util.Map;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.eventhub.EventHubConfig;
+import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestEventHubsSystemDescriptor {
+  @Test
+  public void testWithDescriptorOverrides() {
+    String systemName = "system-name";
+    String streamId1 = "input-stream1";
+    String streamId2 = "input-stream2";
+    String streamId3 = "output-stream1";
+    String streamId4 = "output-stream2";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName)
+        .withMaxEventCountPerPoll(1000)
+        .withNumClientThreads(5)
+        .withPartitioningMethod(PartitioningMethod.PARTITION_KEY_AS_PARTITION)
+        .withPrefetchCount(100)
+        .withReceiveQueueSize(500)
+        .withRuntimeInfoTimeout(60000)
+        .withSendKeys(false);
+
+    systemDescriptor.getInputDescriptor(streamId1, "entity-namespace1", "entity1", KVSerde.of(new StringSerde(),
+        new IntegerSerde()));
+    systemDescriptor.getInputDescriptor(streamId2, "entity-namespace2", "entity2", KVSerde.of(new StringSerde(),
+            new IntegerSerde()));
+    systemDescriptor.getOutputDescriptor(streamId3, "entity-namespace3", "entity3", KVSerde.of(new StringSerde(),
+            new IntegerSerde()));
+    systemDescriptor.getOutputDescriptor(streamId4, "entity-namespace4", "entity4", KVSerde.of(new StringSerde(),
+            new IntegerSerde()));
+
+    Map<String, String> generatedConfigs = systemDescriptor.toConfig();
+    assertEquals("org.apache.samza.system.eventhub.EventHubSystemFactory", generatedConfigs.get(String.format("systems.%s.samza.factory", systemName)));
+    assertEquals("1000", generatedConfigs.get(String.format(EventHubConfig.CONFIG_MAX_EVENT_COUNT_PER_POLL, systemName)));
+    assertEquals("5", generatedConfigs.get(String.format(EventHubConfig.CONFIG_SYSTEM_NUM_CLIENT_THREADS, systemName)));
+    assertEquals("PARTITION_KEY_AS_PARTITION", generatedConfigs.get(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName)));
+    assertEquals("100", generatedConfigs.get(String.format(EventHubConfig.CONFIG_PREFETCH_COUNT, systemName)));
+    assertEquals("500", generatedConfigs.get(String.format(EventHubConfig.CONFIG_CONSUMER_BUFFER_CAPACITY, systemName)));
+    assertEquals("60000", generatedConfigs.get(String.format(EventHubConfig.CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName)));
+    assertEquals("false", generatedConfigs.get(String.format(EventHubConfig.CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName)));
+    assertEquals(streamId1 + "," + streamId2 + "," + streamId3 + "," + streamId4, generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName)));
+  }
+
+  @Test
+  public void testWithoutDescriptorOverrides() {
+    String systemName = "eventHub";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+
+    Map<String, String> generatedConfigs = systemDescriptor.toConfig();
+    assertEquals("org.apache.samza.system.eventhub.EventHubSystemFactory", generatedConfigs.get(String.format("systems.%s.samza.factory", systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_MAX_EVENT_COUNT_PER_POLL, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_SYSTEM_NUM_CLIENT_THREADS, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_PREFETCH_COUNT, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_CONSUMER_BUFFER_CAPACITY, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName)));
+    assertNull(generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName)));
+    assertEquals(1, generatedConfigs.size());
+  }
+  @Test
+  public void testWithInputOutputStreams() {
+    String systemName = "system-name";
+    String streamId1 = "input-stream1";
+    String streamId2 = "input-stream2";
+    String streamId3 = "output-stream1";
+    String streamId4 = "output-stream2";
+
+    EventHubsSystemDescriptor systemDescriptor = new EventHubsSystemDescriptor(systemName);
+
+    systemDescriptor.getInputDescriptor(streamId1, "entity-namespace1", "entity1", KVSerde.of(new StringSerde(),
+        new IntegerSerde()));
+    systemDescriptor.getInputDescriptor(streamId2, "entity-namespace2", "entity2", KVSerde.of(new StringSerde(),
+        new IntegerSerde()));
+    systemDescriptor.getOutputDescriptor(streamId3, "entity-namespace3", "entity3", KVSerde.of(new StringSerde(),
+        new IntegerSerde()));
+    systemDescriptor.getOutputDescriptor(streamId4, "entity-namespace4", "entity4", KVSerde.of(new StringSerde(),
+        new IntegerSerde()));
+
+    Map<String, String> generatedConfigs = systemDescriptor.toConfig();
+    assertEquals("org.apache.samza.system.eventhub.EventHubSystemFactory", generatedConfigs.get(String.format("systems.%s.samza.factory", systemName)));
+    assertEquals(streamId1 + "," + streamId2 + "," + streamId3 + "," + streamId4, generatedConfigs.get(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName)));
+    assertEquals(2, generatedConfigs.size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 5416af5..0000000
--- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java
+++ /dev/null
@@ -1,298 +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.util.Collections;
-import java.util.HashMap;
-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.context.ApplicationContainerContext;
-import org.apache.samza.context.ApplicationContainerContextFactory;
-import org.apache.samza.context.ApplicationTaskContext;
-import org.apache.samza.context.ApplicationTaskContextFactory;
-import org.apache.samza.metrics.MetricsReporterFactory;
-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.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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * This is the base class that implements interface {@link ApplicationDescriptor}.
- * <p>
- * This base class contains the common objects that are used by both high-level and low-level API applications, such as
- * {@link Config}, {@link ApplicationContainerContextFactory}, {@link ApplicationTaskContextFactory}, and
- * {@link ProcessorLifecycleListenerFactory}.
- *
- * @param <S> the type of {@link ApplicationDescriptor} interface this implements. It has to be either
- *            {@link StreamApplicationDescriptor} or {@link TaskApplicationDescriptor}
- */
-public abstract class ApplicationDescriptorImpl<S extends ApplicationDescriptor>
-    implements ApplicationDescriptor<S> {
-  private static final Logger LOGGER = LoggerFactory.getLogger(ApplicationDescriptorImpl.class);
-
-  private final Class<? extends SamzaApplication> appClass;
-  private final Map<String, MetricsReporterFactory> reporterFactories = new LinkedHashMap<>();
-  // serdes used by input/output/intermediate streams, keyed by streamId
-  private final Map<String, KV<Serde, Serde>> streamSerdes = new HashMap<>();
-  // serdes used by tables, keyed by tableId
-  private final Map<String, KV<Serde, Serde>> tableSerdes = new HashMap<>();
-  final Config config;
-
-  private Optional<ApplicationContainerContextFactory<?>> applicationContainerContextFactoryOptional = Optional.empty();
-  private Optional<ApplicationTaskContextFactory<?>> applicationTaskContextFactoryOptional = Optional.empty();
-
-  // Default to no-op  ProcessorLifecycleListenerFactory
-  ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { };
-
-  ApplicationDescriptorImpl(SamzaApplication app, Config config) {
-    this.config = config;
-    this.appClass = app.getClass();
-  }
-
-  @Override
-  public Config getConfig() {
-    return config;
-  }
-
-  @Override
-  public S withApplicationContainerContextFactory(ApplicationContainerContextFactory<?> factory) {
-    this.applicationContainerContextFactoryOptional = Optional.of(factory);
-    return (S) this;
-  }
-
-  @Override
-  public S withApplicationTaskContextFactory(ApplicationTaskContextFactory<?> factory) {
-    this.applicationTaskContextFactoryOptional = Optional.of(factory);
-    return (S) this;
-  }
-
-  @Override
-  public S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory) {
-    this.listenerFactory = listenerFactory;
-    return (S) this;
-  }
-
-  @Override
-  public S withMetricsReporterFactories(Map<String, MetricsReporterFactory> reporterFactories) {
-    this.reporterFactories.clear();
-    this.reporterFactories.putAll(reporterFactories);
-    return (S) this;
-  }
-
-  /**
-   * Get the application class
-   *
-   * @return an implementation of {@link SamzaApplication}
-   */
-  public Class<? extends SamzaApplication> getAppClass() {
-    return appClass;
-  }
-
-  /**
-   * Get the {@link ApplicationContainerContextFactory} specified by the application.
-   *
-   * @return {@link ApplicationContainerContextFactory} if application specified it; empty otherwise
-   */
-  public Optional<ApplicationContainerContextFactory<ApplicationContainerContext>> getApplicationContainerContextFactory() {
-    @SuppressWarnings("unchecked") // ok because all context types are at least ApplicationContainerContext
-    Optional<ApplicationContainerContextFactory<ApplicationContainerContext>> factoryOptional =
-        (Optional) this.applicationContainerContextFactoryOptional;
-    return factoryOptional;
-  }
-
-  /**
-   * Get the {@link ApplicationTaskContextFactory} specified by the application.
-   *
-   * @return {@link ApplicationTaskContextFactory} if application specified it; empty otherwise
-   */
-  public Optional<ApplicationTaskContextFactory<ApplicationTaskContext>> getApplicationTaskContextFactory() {
-    @SuppressWarnings("unchecked") // ok because all context types are at least ApplicationTaskContext
-    Optional<ApplicationTaskContextFactory<ApplicationTaskContext>> factoryOptional =
-        (Optional) this.applicationTaskContextFactoryOptional;
-    return factoryOptional;
-  }
-
-  /**
-   * Get the {@link ProcessorLifecycleListenerFactory} associated with this application
-   *
-   * @return the {@link ProcessorLifecycleListenerFactory} in this application
-   */
-  public ProcessorLifecycleListenerFactory getProcessorLifecycleListenerFactory() {
-    return listenerFactory;
-  }
-
-  /**
-   * Get the {@link MetricsReporterFactory}s used in the application
-   *
-   * @return the map of {@link MetricsReporterFactory}s
-   */
-  public Map<String, MetricsReporterFactory> getMetricsReporterFactories() {
-    return Collections.unmodifiableMap(reporterFactories);
-  }
-
-  /**
-   * Get the default {@link SystemDescriptor} in this application
-   *
-   * @return the default {@link SystemDescriptor}
-   */
-  public Optional<SystemDescriptor> getDefaultSystemDescriptor() {
-    // default is not set
-    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<Serde, Serde> 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<Serde, Serde> 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<String, InputOperatorSpec> getInputOperators() {
-    return Collections.EMPTY_MAP;
-  }
-
-  /**
-   * Get all the {@link InputDescriptor}s to this application
-   *
-   * @return an immutable map of streamId to {@link InputDescriptor}
-   */
-  public abstract Map<String, InputDescriptor> getInputDescriptors();
-
-  /**
-   * Get all the {@link OutputDescriptor}s from this application
-   *
-   * @return an immutable map of streamId to {@link OutputDescriptor}
-   */
-  public abstract Map<String, OutputDescriptor> getOutputDescriptors();
-
-  /**
-   * Get all the broadcast streamIds from this application
-   *
-   * @return an immutable set of streamIds
-   */
-  public abstract Set<String> getBroadcastStreams();
-
-  /**
-   * Get all the {@link TableDescriptor}s in this application
-   *
-   * @return an immutable set of {@link TableDescriptor}s
-   */
-  public abstract Set<TableDescriptor> getTableDescriptors();
-
-  /**
-   * Get all the unique {@link SystemDescriptor}s in this application
-   *
-   * @return an immutable set of {@link SystemDescriptor}s
-   */
-  public abstract Set<SystemDescriptor> getSystemDescriptors();
-
-  /**
-   * Get all the unique input streamIds in this application
-   *
-   * @return an immutable set of input streamIds
-   */
-  public abstract Set<String> getInputStreamIds();
-
-  /**
-   * Get all the unique output streamIds in this application
-   *
-   * @return an immutable set of output streamIds
-   */
-  public abstract Set<String> getOutputStreamIds();
-
-  KV<Serde, Serde> getOrCreateStreamSerdes(String streamId, Serde serde) {
-    Serde keySerde, valueSerde;
-
-    KV<Serde, Serde> currentSerdePair = streamSerdes.get(streamId);
-
-    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);
-  }
-
-  KV<Serde, Serde> 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<Serde, Serde> 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

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java
deleted file mode 100644
index acf4bf7..0000000
--- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java
+++ /dev/null
@@ -1,51 +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 org.apache.samza.config.Config;
-
-
-/**
- * Util class to help creating {@link ApplicationDescriptorImpl} instance from {@link SamzaApplication} and {@link Config}
- */
-public class ApplicationDescriptorUtil {
-
-  private ApplicationDescriptorUtil() {
-
-  }
-
-  /**
-   * Create a new instance of {@link ApplicationDescriptorImpl} based on {@link SamzaApplication} and {@link 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<? extends ApplicationDescriptor> getAppDescriptor(SamzaApplication app, Config config) {
-    if (app instanceof StreamApplication) {
-      return new StreamApplicationDescriptorImpl((StreamApplication) app, 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.", app.getClass().getName()));
-  }
-
-}
\ No newline at end of file


[06/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableDescriptor.java
new file mode 100644
index 0000000..0187b2e
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableDescriptor.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.remote.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.table.TableSpec;
+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.retry.TableRetryPolicy;
+import org.apache.samza.table.utils.SerdeUtils;
+import org.apache.samza.util.EmbeddedTaggedRateLimiter;
+import org.apache.samza.util.RateLimiter;
+
+import com.google.common.base.Preconditions;
+
+
+/**
+ * Table descriptor for remote store backed tables
+ *
+ * @param <K> the type of the key
+ * @param <V> the type of the value
+ */
+public class RemoteTableDescriptor<K, V> extends BaseTableDescriptor<K, V, RemoteTableDescriptor<K, V>> {
+  /**
+   * Tag to be used for provision credits for rate limiting read operations from the remote table.
+   * Caller must pre-populate the credits with this tag when specifying a custom rate limiter instance
+   * through {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, TableRateLimiter.CreditFunction,
+   * TableRateLimiter.CreditFunction)}
+   */
+  public static final String RL_READ_TAG = "readTag";
+
+  /**
+   * Tag to be used for provision credits for rate limiting write operations into the remote table.
+   * Caller can optionally populate the credits with this tag when specifying a custom rate limiter instance
+   * through {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, TableRateLimiter.CreditFunction,
+   * TableRateLimiter.CreditFunction)} and it needs the write functionality.
+   */
+  public static final String RL_WRITE_TAG = "writeTag";
+
+  // Input support for a specific remote store (required)
+  private TableReadFunction<K, V> readFn;
+
+  // Output support for a specific remote store (optional)
+  private TableWriteFunction<K, V> writeFn;
+
+  // Rate limiter for client-side throttling;
+  // can either be constructed indirectly from rates or overridden by withRateLimiter()
+  private RateLimiter rateLimiter;
+
+  // Rates for constructing the default rate limiter when they are non-zero
+  private Map<String, Integer> tagCreditsMap = new HashMap<>();
+
+  private TableRateLimiter.CreditFunction<K, V> readCreditFn;
+  private TableRateLimiter.CreditFunction<K, V> writeCreditFn;
+
+  private TableRetryPolicy readRetryPolicy;
+  private TableRetryPolicy writeRetryPolicy;
+
+  // By default execute future callbacks on the native client threads
+  // ie. no additional thread pool for callbacks.
+  private int asyncCallbackPoolSize = -1;
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   */
+  public RemoteTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   * @param serde the serde for key and value
+   */
+  public RemoteTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
+  @Override
+  public TableSpec getTableSpec() {
+    validate();
+
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    generateTableSpecConfig(tableSpecConfig);
+
+    // Serialize and store reader/writer functions
+    tableSpecConfig.put(RemoteTableProvider.READ_FN, SerdeUtils.serialize("read function", readFn));
+
+    if (writeFn != null) {
+      tableSpecConfig.put(RemoteTableProvider.WRITE_FN, SerdeUtils.serialize("write function", writeFn));
+    }
+
+    // Serialize the rate limiter if specified
+    if (!tagCreditsMap.isEmpty()) {
+      rateLimiter = new EmbeddedTaggedRateLimiter(tagCreditsMap);
+    }
+
+    if (rateLimiter != null) {
+      tableSpecConfig.put(RemoteTableProvider.RATE_LIMITER, SerdeUtils.serialize("rate limiter", rateLimiter));
+    }
+
+    // Serialize the readCredit and writeCredit functions
+    if (readCreditFn != null) {
+      tableSpecConfig.put(RemoteTableProvider.READ_CREDIT_FN, SerdeUtils.serialize(
+          "read credit function", readCreditFn));
+    }
+
+    if (writeCreditFn != null) {
+      tableSpecConfig.put(RemoteTableProvider.WRITE_CREDIT_FN, SerdeUtils.serialize(
+          "write credit function", writeCreditFn));
+    }
+
+    if (readRetryPolicy != null) {
+      tableSpecConfig.put(RemoteTableProvider.READ_RETRY_POLICY, SerdeUtils.serialize(
+          "read retry policy", readRetryPolicy));
+    }
+
+    if (writeRetryPolicy != null) {
+      tableSpecConfig.put(RemoteTableProvider.WRITE_RETRY_POLICY, SerdeUtils.serialize(
+          "write retry policy", writeRetryPolicy));
+    }
+
+    tableSpecConfig.put(RemoteTableProvider.ASYNC_CALLBACK_POOL_SIZE, String.valueOf(asyncCallbackPoolSize));
+
+    return new TableSpec(tableId, serde, RemoteTableProviderFactory.class.getName(), tableSpecConfig);
+  }
+
+  /**
+   * Use specified TableReadFunction with remote table and a retry policy.
+   * @param readFn read function instance
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withReadFunction(TableReadFunction<K, V> readFn) {
+    Preconditions.checkNotNull(readFn, "null read function");
+    this.readFn = readFn;
+    return this;
+  }
+
+  /**
+   * Use specified TableWriteFunction with remote table and a retry policy.
+   * @param writeFn write function instance
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withWriteFunction(TableWriteFunction<K, V> writeFn) {
+    Preconditions.checkNotNull(writeFn, "null write function");
+    this.writeFn = writeFn;
+    return this;
+  }
+
+  /**
+   * Use specified TableReadFunction with remote table.
+   * @param readFn read function instance
+   * @param retryPolicy retry policy for the read function
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withReadFunction(TableReadFunction<K, V> readFn, TableRetryPolicy retryPolicy) {
+    Preconditions.checkNotNull(readFn, "null read function");
+    Preconditions.checkNotNull(retryPolicy, "null retry policy");
+    this.readFn = readFn;
+    this.readRetryPolicy = retryPolicy;
+    return this;
+  }
+
+  /**
+   * Use specified TableWriteFunction with remote table.
+   * @param writeFn write function instance
+   * @param retryPolicy retry policy for the write function
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withWriteFunction(TableWriteFunction<K, V> writeFn, TableRetryPolicy retryPolicy) {
+    Preconditions.checkNotNull(writeFn, "null write function");
+    Preconditions.checkNotNull(retryPolicy, "null retry policy");
+    this.writeFn = writeFn;
+    this.writeRetryPolicy = retryPolicy;
+    return this;
+  }
+
+  /**
+   * Specify a rate limiter along with credit functions to map a table record (as KV) to the amount
+   * of credits to be charged from the rate limiter for table read and write operations.
+   * This is an advanced API that provides greater flexibility to throttle each record in the table
+   * with different number of credits. For most common use-cases eg: limit the number of read/write
+   * operations, please instead use the {@link RemoteTableDescriptor#withReadRateLimit(int)} and
+   * {@link RemoteTableDescriptor#withWriteRateLimit(int)}.
+   *
+   * @param rateLimiter rate limiter instance to be used for throttling
+   * @param readCreditFn credit function for rate limiting read operations
+   * @param writeCreditFn credit function for rate limiting write operations
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withRateLimiter(RateLimiter rateLimiter,
+      TableRateLimiter.CreditFunction<K, V> readCreditFn,
+      TableRateLimiter.CreditFunction<K, V> writeCreditFn) {
+    Preconditions.checkNotNull(rateLimiter, "null read rate limiter");
+    this.rateLimiter = rateLimiter;
+    this.readCreditFn = readCreditFn;
+    this.writeCreditFn = writeCreditFn;
+    return this;
+  }
+
+  /**
+   * Specify the rate limit for table read operations. If the read rate limit is set with this method
+   * it is invalid to call {@link RemoteTableDescriptor#withRateLimiter(RateLimiter,
+   * TableRateLimiter.CreditFunction, TableRateLimiter.CreditFunction)}
+   * and vice versa.
+   * @param creditsPerSec rate limit for read operations; must be positive
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withReadRateLimit(int creditsPerSec) {
+    Preconditions.checkArgument(creditsPerSec > 0, "Max read rate must be a positive number.");
+    tagCreditsMap.put(RL_READ_TAG, creditsPerSec);
+    return this;
+  }
+
+  /**
+   * Specify the rate limit for table write operations. If the write rate limit is set with this method
+   * it is invalid to call {@link RemoteTableDescriptor#withRateLimiter(RateLimiter,
+   * TableRateLimiter.CreditFunction, TableRateLimiter.CreditFunction)}
+   * and vice versa.
+   * @param creditsPerSec rate limit for write operations; must be positive
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withWriteRateLimit(int creditsPerSec) {
+    Preconditions.checkArgument(creditsPerSec > 0, "Max write rate must be a positive number.");
+    tagCreditsMap.put(RL_WRITE_TAG, creditsPerSec);
+    return this;
+  }
+
+  /**
+   * Specify the size of the thread pool for the executor used to execute
+   * callbacks of CompletableFutures of async Table operations. By default, these
+   * futures are completed (called) by the threads of the native store client. Depending
+   * on the implementation of the native client, it may or may not allow executing long
+   * running operations in the callbacks. This config can be used to execute the callbacks
+   * from a separate executor to decouple from the native client. If configured, this
+   * thread pool is shared by all read and write operations.
+   * @param poolSize max number of threads in the executor for async callbacks
+   * @return this table descriptor instance
+   */
+  public RemoteTableDescriptor<K, V> withAsyncCallbackExecutorPoolSize(int poolSize) {
+    this.asyncCallbackPoolSize = poolSize;
+    return this;
+  }
+
+  @Override
+  protected void validate() {
+    super.validate();
+    Preconditions.checkNotNull(readFn, "TableReadFunction is required.");
+    Preconditions.checkArgument(rateLimiter == null || tagCreditsMap.isEmpty(),
+        "Only one of rateLimiter instance or read/write limits can be specified");
+    // Assume callback executor pool should have no more than 20 threads
+    Preconditions.checkArgument(asyncCallbackPoolSize <= 20,
+        "too many threads for async callback executor.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProvider.java
new file mode 100644
index 0000000..fdccc70
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProvider.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.remote.descriptors;
+
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.remote.RemoteReadWriteTable;
+import org.apache.samza.table.remote.RemoteReadableTable;
+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.retry.RetriableReadFunction;
+import org.apache.samza.table.retry.RetriableWriteFunction;
+import org.apache.samza.table.retry.TableRetryPolicy;
+import org.apache.samza.table.utils.descriptors.BaseTableProvider;
+import org.apache.samza.table.utils.SerdeUtils;
+import org.apache.samza.table.utils.TableMetricsUtil;
+import org.apache.samza.util.RateLimiter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+
+/**
+ * Provide for remote table instances
+ */
+public class RemoteTableProvider extends BaseTableProvider {
+
+  static final String READ_FN = "io.read.func";
+  static final String WRITE_FN = "io.write.func";
+  static final String RATE_LIMITER = "io.ratelimiter";
+  static final String READ_CREDIT_FN = "io.read.credit.func";
+  static final String WRITE_CREDIT_FN = "io.write.credit.func";
+  static final String ASYNC_CALLBACK_POOL_SIZE = "io.async.callback.pool.size";
+  static final String READ_RETRY_POLICY = "io.read.retry.policy";
+  static final String WRITE_RETRY_POLICY = "io.write.retry.policy";
+
+  private final boolean readOnly;
+  private final List<RemoteReadableTable<?, ?>> tables = new ArrayList<>();
+
+  /**
+   * Map of tableId -> executor service for async table IO and callbacks. The same executors
+   * are shared by both read/write operations such that tables of the same tableId all share
+   * the set same of executors globally whereas table itself is per-task.
+   */
+  private static Map<String, ExecutorService> tableExecutors = new ConcurrentHashMap<>();
+  private static Map<String, ExecutorService> callbackExecutors = new ConcurrentHashMap<>();
+  private static ScheduledExecutorService retryExecutor;
+
+  public RemoteTableProvider(TableSpec tableSpec) {
+    super(tableSpec);
+    this.readOnly = !tableSpec.getConfig().containsKey(WRITE_FN);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Table getTable() {
+    RemoteReadableTable table;
+    String tableId = tableSpec.getId();
+
+    TableReadFunction readFn = getReadFn();
+    RateLimiter rateLimiter = deserializeObject(RATE_LIMITER);
+    if (rateLimiter != null) {
+      rateLimiter.init(this.context);
+    }
+    TableRateLimiter.CreditFunction<?, ?> readCreditFn = deserializeObject(READ_CREDIT_FN);
+    TableRateLimiter readRateLimiter = new TableRateLimiter(tableSpec.getId(), rateLimiter, readCreditFn, RemoteTableDescriptor.RL_READ_TAG);
+
+    TableRateLimiter.CreditFunction<?, ?> writeCreditFn;
+    TableRateLimiter writeRateLimiter = null;
+
+    TableRetryPolicy readRetryPolicy = deserializeObject(READ_RETRY_POLICY);
+    TableRetryPolicy writeRetryPolicy = null;
+
+    if ((readRetryPolicy != null || writeRetryPolicy != null) && retryExecutor == null) {
+      retryExecutor = Executors.newSingleThreadScheduledExecutor(runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("table-retry-executor");
+          thread.setDaemon(true);
+          return thread;
+        });
+    }
+
+    if (readRetryPolicy != null) {
+      readFn = new RetriableReadFunction<>(readRetryPolicy, readFn, retryExecutor);
+    }
+
+    TableWriteFunction writeFn = getWriteFn();
+
+    boolean isRateLimited = readRateLimiter.isRateLimited();
+    if (!readOnly) {
+      writeCreditFn = deserializeObject(WRITE_CREDIT_FN);
+      writeRateLimiter = new TableRateLimiter(tableSpec.getId(), rateLimiter, writeCreditFn, RemoteTableDescriptor.RL_WRITE_TAG);
+      isRateLimited |= writeRateLimiter.isRateLimited();
+      writeRetryPolicy = deserializeObject(WRITE_RETRY_POLICY);
+      if (writeRetryPolicy != null) {
+        writeFn = new RetriableWriteFunction(writeRetryPolicy, writeFn, retryExecutor);
+      }
+    }
+
+    // Optional executor for future callback/completion. Shared by both read and write operations.
+    int callbackPoolSize = Integer.parseInt(tableSpec.getConfig().get(ASYNC_CALLBACK_POOL_SIZE));
+    if (callbackPoolSize > 0) {
+      callbackExecutors.computeIfAbsent(tableId, (arg) ->
+          Executors.newFixedThreadPool(callbackPoolSize, (runnable) -> {
+              Thread thread = new Thread(runnable);
+              thread.setName("table-" + tableId + "-async-callback-pool");
+              thread.setDaemon(true);
+              return thread;
+            }));
+    }
+
+    if (isRateLimited) {
+      tableExecutors.computeIfAbsent(tableId, (arg) ->
+          Executors.newSingleThreadExecutor(runnable -> {
+              Thread thread = new Thread(runnable);
+              thread.setName("table-" + tableId + "-async-executor");
+              thread.setDaemon(true);
+              return thread;
+            }));
+    }
+
+    if (readOnly) {
+      table = new RemoteReadableTable(tableSpec.getId(), readFn, readRateLimiter,
+          tableExecutors.get(tableId), callbackExecutors.get(tableId));
+    } else {
+      table = new RemoteReadWriteTable(tableSpec.getId(), readFn, writeFn, readRateLimiter,
+          writeRateLimiter, tableExecutors.get(tableId), callbackExecutors.get(tableId));
+    }
+
+    TableMetricsUtil metricsUtil = new TableMetricsUtil(this.context, table, tableId);
+    if (readRetryPolicy != null) {
+      ((RetriableReadFunction) readFn).setMetrics(metricsUtil);
+    }
+    if (writeRetryPolicy != null) {
+      ((RetriableWriteFunction) writeFn).setMetrics(metricsUtil);
+    }
+
+    table.init(this.context);
+    tables.add(table);
+    return table;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void close() {
+    tables.forEach(t -> t.close());
+    tableExecutors.values().forEach(e -> e.shutdown());
+    callbackExecutors.values().forEach(e -> e.shutdown());
+  }
+
+  private <T> T deserializeObject(String key) {
+    String entry = tableSpec.getConfig().getOrDefault(key, "");
+    if (entry.isEmpty()) {
+      return null;
+    }
+    return SerdeUtils.deserialize(key, entry);
+  }
+
+  private TableReadFunction<?, ?> getReadFn() {
+    TableReadFunction<?, ?> readFn = deserializeObject(READ_FN);
+    if (readFn != null) {
+      readFn.init(this.context);
+    }
+    return readFn;
+  }
+
+  private TableWriteFunction<?, ?> getWriteFn() {
+    TableWriteFunction<?, ?> writeFn = deserializeObject(WRITE_FN);
+    if (writeFn != null) {
+      writeFn.init(this.context);
+    }
+    return writeFn;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProviderFactory.java
new file mode 100644
index 0000000..4802265
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/remote/descriptors/RemoteTableProviderFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.table.remote.descriptors;
+
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
+import org.apache.samza.table.TableSpec;
+
+import com.google.common.base.Preconditions;
+
+
+/**
+ * Factory class for a remote table provider
+ */
+public class RemoteTableProviderFactory implements TableProviderFactory {
+  @Override
+  public TableProvider getTableProvider(TableSpec tableSpec) {
+    Preconditions.checkNotNull(tableSpec, "null table spec");
+    return new RemoteTableProvider(tableSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/retry/RetriableReadFunction.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/retry/RetriableReadFunction.java b/samza-core/src/main/java/org/apache/samza/table/retry/RetriableReadFunction.java
index 1adddc0..8f7aa7e 100644
--- a/samza-core/src/main/java/org/apache/samza/table/retry/RetriableReadFunction.java
+++ b/samza-core/src/main/java/org/apache/samza/table/retry/RetriableReadFunction.java
@@ -39,7 +39,7 @@ import static org.apache.samza.table.retry.FailsafeAdapter.failsafe;
 /**
  * Wrapper for a {@link TableReadFunction} instance to add common retry
  * support with a {@link TableRetryPolicy}. This wrapper is created by
- * {@link org.apache.samza.table.remote.RemoteTableProvider} when a retry
+ * {@link org.apache.samza.table.remote.descriptors.RemoteTableProvider} when a retry
  * policy is specified together with the {@link TableReadFunction}.
  *
  * Actual retry mechanism is provided by the failsafe library. Retry is

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/retry/RetriableWriteFunction.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/retry/RetriableWriteFunction.java b/samza-core/src/main/java/org/apache/samza/table/retry/RetriableWriteFunction.java
index 2f3f062..ee7959a 100644
--- a/samza-core/src/main/java/org/apache/samza/table/retry/RetriableWriteFunction.java
+++ b/samza-core/src/main/java/org/apache/samza/table/retry/RetriableWriteFunction.java
@@ -39,7 +39,7 @@ import static org.apache.samza.table.retry.FailsafeAdapter.failsafe;
 /**
  * Wrapper for a {@link TableWriteFunction} instance to add common retry
  * support with a {@link TableRetryPolicy}. This wrapper is created by
- * {@link org.apache.samza.table.remote.RemoteTableProvider} when a retry
+ * {@link org.apache.samza.table.remote.descriptors.RemoteTableProvider} when a retry
  * policy is specified together with the {@link TableWriteFunction}.
  *
  * Actual retry mechanism is provided by the failsafe library. Retry is

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/utils/BaseTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index dfbd835..0000000
--- a/samza-core/src/main/java/org/apache/samza/table/utils/BaseTableProvider.java
+++ /dev/null
@@ -1,73 +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.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.context.Context;
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableSpec;
-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 Context context;
-
-  public BaseTableProvider(TableSpec tableSpec) {
-    this.tableSpec = tableSpec;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void init(Context context) {
-    this.context = context;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public Map<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
-    Map<String, String> 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;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/table/utils/descriptors/BaseTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/utils/descriptors/BaseTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/utils/descriptors/BaseTableProvider.java
new file mode 100644
index 0000000..6f787d1
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/utils/descriptors/BaseTableProvider.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.table.utils.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JavaTableConfig;
+import org.apache.samza.context.Context;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.TableSpec;
+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 Context context;
+
+  public BaseTableProvider(TableSpec tableSpec) {
+    this.tableSpec = tableSpec;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void init(Context context) {
+    this.context = context;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Map<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
+    Map<String, String> 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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
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 218ba5d..87131d7 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
@@ -64,11 +64,12 @@ public class StreamOperatorTask implements StreamTask, InitableTask, WindowableT
    * Initializes this task during startup.
    * <p>
    * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}.
-   * 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 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.
+   * Users set the input and output streams and the task-wide context manager using
+   * {@link org.apache.samza.application.descriptors.StreamApplicationDescriptor} APIs, and the logical transforms
+   * using the {@link org.apache.samza.operators.MessageStream} APIs. After the
+   * {@link org.apache.samza.application.descriptors.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.
    *
    * @param context allows initializing and accessing contextual data of this StreamTask
    * @throws Exception in case of initialization errors

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java
----------------------------------------------------------------------
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 c312fac..b2297e1 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
@@ -21,10 +21,10 @@ 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.StreamApplicationDescriptorImpl;
-import org.apache.samza.application.TaskApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptorImpl;
 import org.apache.samza.config.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala
----------------------------------------------------------------------
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 929d6a4..ee6aff3 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,8 @@
 
 package org.apache.samza.job.local
 
-import org.apache.samza.application.{ApplicationDescriptorUtil, ApplicationUtil}
+import org.apache.samza.application.ApplicationUtil
+import org.apache.samza.application.descriptors.ApplicationDescriptorUtil
 import org.apache.samza.config.JobConfig._
 import org.apache.samza.config.ShellCommandConfig._
 import org.apache.samza.config.{Config, TaskConfigJava}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java
----------------------------------------------------------------------
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 ccd88b8..8b96c8a 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
@@ -18,6 +18,8 @@
  */
 package org.apache.samza.application;
 
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
+
 /**
  * Test class of {@link StreamApplication} for unit tests
  */

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java b/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java
index 9b590c4..ab91cee 100644
--- a/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java
+++ b/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java
@@ -25,6 +25,8 @@ 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.application.descriptors.TaskApplicationDescriptor;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptorImpl;
 import org.apache.samza.task.MockStreamTask;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index de16ef2..0000000
--- a/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java
+++ /dev/null
@@ -1,601 +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 com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.JobConfig;
-import org.apache.samza.context.ApplicationContainerContextFactory;
-import org.apache.samza.context.ApplicationTaskContextFactory;
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.data.TestMessageEnvelope;
-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.system.ExpandingInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.descriptors.base.system.TransformingInputDescriptorProvider;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.operators.functions.StreamExpander;
-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;
-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.table.TableSpec;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-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.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Unit test for {@link StreamApplicationDescriptorImpl}
- */
-public class TestStreamApplicationDescriptorImpl {
-
-  @Test
-  public void testConstructor() {
-    StreamApplication mockApp = mock(StreamApplication.class);
-    Config mockConfig = mock(Config.class);
-    StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(mockApp, mockConfig);
-    verify(mockApp).describe(appDesc);
-    assertEquals(mockConfig, appDesc.config);
-  }
-
-  @Test
-  public void testGetInputStreamWithValueSerde() {
-
-    String streamId = "test-stream-1";
-    Serde mockValueSerde = mock(Serde.class);
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mockValueSerde);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-      }, mock(Config.class));
-
-    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(streamId, inputOpSpec.getStreamId());
-    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
-    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
-  }
-
-  @Test
-  public void testGetInputStreamWithKeyValueSerde() {
-
-    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();
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mockKVSerde);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-      }, mock(Config.class));
-
-    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(streamId, inputOpSpec.getStreamId());
-    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
-    assertEquals(mockKeySerde, inputOpSpec.getKeySerde());
-    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testGetInputStreamWithNullSerde() {
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd = sd.getInputDescriptor("mockStreamId", null);
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-      }, mock(Config.class));
-  }
-
-  @Test
-  public void testGetInputStreamWithTransformFunction() {
-    String streamId = "test-stream-1";
-    Serde mockValueSerde = mock(Serde.class);
-    InputTransformer transformer = ime -> ime;
-    MockTransformingSystemDescriptor sd = new MockTransformingSystemDescriptor("mockSystem", transformer);
-    MockInputDescriptor isd = sd.getInputDescriptor(streamId, mockValueSerde);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-      }, mock(Config.class));
-
-    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(streamId, inputOpSpec.getStreamId());
-    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
-    assertEquals(transformer, inputOpSpec.getTransformer());
-  }
-
-  @Test
-  public void testGetInputStreamWithExpandingSystem() {
-    String streamId = "test-stream-1";
-    String expandedStreamId = "expanded-stream";
-    AtomicInteger expandCallCount = new AtomicInteger();
-    StreamExpander expander = (sg, isd) -> {
-      expandCallCount.incrementAndGet();
-      InputDescriptor expandedISD =
-          new GenericSystemDescriptor("expanded-system", "mockFactoryClass")
-              .getInputDescriptor(expandedStreamId, new IntegerSerde());
-
-      return sg.getInputStream(expandedISD);
-    };
-    MockExpandingSystemDescriptor sd = new MockExpandingSystemDescriptor("mock-system", expander);
-    MockInputDescriptor isd = sd.getInputDescriptor(streamId, new IntegerSerde());
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-      }, mock(Config.class));
-
-    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(expandedStreamId);
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(1, expandCallCount.get());
-    assertFalse(streamAppDesc.getInputOperators().containsKey(streamId));
-    assertFalse(streamAppDesc.getInputDescriptors().containsKey(streamId));
-    assertTrue(streamAppDesc.getInputDescriptors().containsKey(expandedStreamId));
-    assertEquals(expandedStreamId, inputOpSpec.getStreamId());
-  }
-
-  @Test
-  public void testGetInputStreamWithRelaxedTypes() {
-    String streamId = "test-stream-1";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mock(Serde.class));
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-      }, mock(Config.class));
-
-    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(streamId, inputOpSpec.getStreamId());
-    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
-  }
-
-  @Test
-  public void testMultipleGetInputStreams() {
-    String streamId1 = "test-stream-1";
-    String streamId2 = "test-stream-2";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd1 = sd.getInputDescriptor(streamId1, mock(Serde.class));
-    GenericInputDescriptor isd2 = sd.getInputDescriptor(streamId2, mock(Serde.class));
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd1);
-        appDesc.getInputStream(isd2);
-      }, mock(Config.class));
-
-    InputOperatorSpec inputOpSpec1 = streamAppDesc.getInputOperators().get(streamId1);
-    InputOperatorSpec inputOpSpec2 = streamAppDesc.getInputOperators().get(streamId2);
-
-    assertEquals(2, streamAppDesc.getInputOperators().size());
-    assertEquals(streamId1, inputOpSpec1.getStreamId());
-    assertEquals(streamId2, inputOpSpec2.getStreamId());
-    assertEquals(2, streamAppDesc.getInputDescriptors().size());
-    assertEquals(isd1, streamAppDesc.getInputDescriptors().get(streamId1));
-    assertEquals(isd2, streamAppDesc.getInputDescriptors().get(streamId2));
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testGetSameInputStreamTwice() {
-    String streamId = "test-stream-1";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd1 = sd.getInputDescriptor(streamId, mock(Serde.class));
-    GenericInputDescriptor isd2 = sd.getInputDescriptor(streamId, mock(Serde.class));
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd1);
-        // should throw exception
-        appDesc.getInputStream(isd2);
-      }, mock(Config.class));
-  }
-
-  @Test
-  public void testMultipleSystemDescriptorForSameSystemName() {
-    GenericSystemDescriptor sd1 = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericSystemDescriptor sd2 = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd1 = sd1.getInputDescriptor("test-stream-1", mock(Serde.class));
-    GenericInputDescriptor isd2 = sd2.getInputDescriptor("test-stream-2", mock(Serde.class));
-    GenericOutputDescriptor osd1 = sd2.getOutputDescriptor("test-stream-3", mock(Serde.class));
-
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd1);
-        try {
-          appDesc.getInputStream(isd2);
-          fail("Adding input stream with the same system name but different SystemDescriptor should have failed");
-        } catch (IllegalStateException e) { }
-
-        try {
-          appDesc.getOutputStream(osd1);
-          fail("adding output stream with the same system name but different SystemDescriptor should have failed");
-        } catch (IllegalStateException e) { }
-      }, mock(Config.class));
-
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        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");
-        } catch (IllegalStateException e) { }
-      }, mock(Config.class));
-  }
-
-  @Test
-  public void testGetOutputStreamWithKeyValueSerde() {
-    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();
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mockKVSerde);
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getOutputStream(osd);
-      }, mock(Config.class));
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId);
-    assertEquals(streamId, outputStreamImpl.getStreamId());
-    assertEquals(osd, streamAppDesc.getOutputDescriptors().get(streamId));
-    assertEquals(mockKeySerde, outputStreamImpl.getKeySerde());
-    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testGetOutputStreamWithNullSerde() {
-    String streamId = "test-stream-1";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, null);
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getOutputStream(osd);
-      }, mock(Config.class));
-  }
-
-  @Test
-  public void testGetOutputStreamWithValueSerde() {
-    String streamId = "test-stream-1";
-    Serde mockValueSerde = mock(Serde.class);
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mockValueSerde);
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getOutputStream(osd);
-      }, mock(Config.class));
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId);
-    assertEquals(streamId, outputStreamImpl.getStreamId());
-    assertEquals(osd, streamAppDesc.getOutputDescriptors().get(streamId));
-    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testSetDefaultSystemDescriptorAfterGettingInputStream() {
-    String streamId = "test-stream-1";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mock(Serde.class));
-
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getInputStream(isd);
-        appDesc.withDefaultSystem(sd); // should throw exception
-      }, mock(Config.class));
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testSetDefaultSystemDescriptorAfterGettingOutputStream() {
-    String streamId = "test-stream-1";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mock(Serde.class));
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getOutputStream(osd);
-        appDesc.withDefaultSystem(sd); // should throw exception
-      }, mock(Config.class));
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testSetDefaultSystemDescriptorAfterGettingIntermediateStream() {
-    String streamId = "test-stream-1";
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class));
-    streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false);
-    streamAppDesc.withDefaultSystem(mock(SystemDescriptor.class)); // should throw exception
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testGetSameOutputStreamTwice() {
-    String streamId = "test-stream-1";
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    GenericOutputDescriptor osd1 = sd.getOutputDescriptor(streamId, mock(Serde.class));
-    GenericOutputDescriptor osd2 = sd.getOutputDescriptor(streamId, mock(Serde.class));
-    new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getOutputStream(osd1);
-        appDesc.getOutputStream(osd2); // should throw exception
-      }, mock(Config.class));
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithValueSerde() {
-    String streamId = "stream-1";
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class));
-
-    Serde mockValueSerde = mock(Serde.class);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        streamAppDesc.getIntermediateStream(streamId, mockValueSerde, false);
-
-    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());
-    assertTrue(((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, ((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithKeyValueSerde() {
-    String streamId = "streamId";
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, 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();
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        streamAppDesc.getIntermediateStream(streamId, mockKVSerde, false);
-
-    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());
-    assertEquals(mockKeySerde, ((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde());
-    assertEquals(mockValueSerde, ((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithDefaultSystemDescriptor() {
-    Config mockConfig = mock(Config.class);
-    String streamId = "streamId";
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mock-system", "mock-system-factory");
-    streamAppDesc.withDefaultSystem(sd);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false);
-
-    assertEquals(streamAppDesc.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec());
-    assertEquals(streamAppDesc.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream());
-    assertEquals(streamId, intermediateStreamImpl.getStreamId());
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testGetIntermediateStreamWithNoSerde() {
-    Config mockConfig = mock(Config.class);
-    String streamId = "streamId";
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        streamAppDesc.getIntermediateStream(streamId, null, false); // should throw
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testGetSameIntermediateStreamTwice() {
-    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);
-  }
-
-  @Test
-  public void testGetNextOpIdIncrementsId() {
-    Config mockConfig = mock(Config.class);
-    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
-    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
-
-    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));
-  }
-
-  @Test(expected = SamzaException.class)
-  public void testGetNextOpIdRejectsDuplicates() {
-    Config mockConfig = mock(Config.class);
-    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
-    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
-    assertEquals("jobName-1234-join-customName", streamAppDesc.getNextOpId(OpCode.JOIN, "customName"));
-    streamAppDesc.getNextOpId(OpCode.JOIN, "customName"); // should throw
-  }
-
-  @Test
-  public void testOpIdValidation() {
-    Config mockConfig = mock(Config.class);
-    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
-    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
-
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
-
-    // null and empty userDefinedIDs should fall back to autogenerated IDs.
-    try {
-      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.");
-    }
-
-    List<String> validOpIds = ImmutableList.of("op_id", "op-id", "1000", "op_1", "OP_ID");
-    for (String validOpId: validOpIds) {
-      try {
-        streamAppDesc.getNextOpId(OpCode.FILTER, validOpId);
-      } catch (Exception e) {
-        fail("Received an exception with a valid operator ID: " + validOpId);
-      }
-    }
-
-    List<String> invalidOpIds = ImmutableList.of("op id", "op#id");
-    for (String invalidOpId: invalidOpIds) {
-      try {
-        streamAppDesc.getNextOpId(OpCode.FILTER, invalidOpId);
-        fail("Did not receive an exception with an invalid operator ID: " + invalidOpId);
-      } catch (SamzaException e) { }
-    }
-  }
-
-  @Test
-  public void testGetInputStreamPreservesInsertionOrder() {
-    Config mockConfig = mock(Config.class);
-
-    String testStreamId1 = "test-stream-1";
-    String testStreamId2 = "test-stream-2";
-    String testStreamId3 = "test-stream-3";
-
-    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
-    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)));
-      }, mockConfig);
-
-    List<InputOperatorSpec> inputSpecs = new ArrayList<>(streamAppDesc.getInputOperators().values());
-    assertEquals(inputSpecs.size(), 3);
-    assertEquals(inputSpecs.get(0).getStreamId(), testStreamId1);
-    assertEquals(inputSpecs.get(1).getStreamId(), testStreamId2);
-    assertEquals(inputSpecs.get(2).getStreamId(), testStreamId3);
-  }
-
-  @Test
-  public void testGetTable() throws Exception {
-    Config mockConfig = mock(Config.class);
-
-    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());
-    when(mockTableDescriptor.getSerde()).thenReturn(testTableSpec.getSerde());
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
-        appDesc.getTable(mockTableDescriptor);
-      }, mockConfig);
-    assertNotNull(streamAppDesc.getTables().get(testTableSpec.getId()));
-  }
-
-  @Test
-  public void testApplicationContainerContextFactory() {
-    ApplicationContainerContextFactory factory = mock(ApplicationContainerContextFactory.class);
-    StreamApplication testApp = appDesc -> appDesc.withApplicationContainerContextFactory(factory);
-    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.of(factory));
-  }
-
-  @Test
-  public void testNoApplicationContainerContextFactory() {
-    StreamApplication testApp = appDesc -> {
-    };
-    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.empty());
-  }
-
-  @Test
-  public void testApplicationTaskContextFactory() {
-    ApplicationTaskContextFactory factory = mock(ApplicationTaskContextFactory.class);
-    StreamApplication testApp = appDesc -> appDesc.withApplicationTaskContextFactory(factory);
-    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.of(factory));
-  }
-
-  @Test
-  public void testNoApplicationTaskContextFactory() {
-    StreamApplication testApp = appDesc -> {
-    };
-    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.empty());
-  }
-
-  @Test
-  public void testProcessorLifecycleListenerFactory() {
-    ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class);
-    StreamApplication testApp = appSpec -> appSpec.withProcessorLifecycleListenerFactory(mockFactory);
-    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 StreamApplicationDescriptorImpl(appDesc -> {
-        BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class);
-        when(mockTableDescriptor.getTableId()).thenReturn("my.table");
-        appDesc.getTable(mockTableDescriptor);
-      }, mockConfig);
-  }
-
-  class MockExpandingSystemDescriptor extends SystemDescriptor<MockExpandingSystemDescriptor> implements ExpandingInputDescriptorProvider<Integer> {
-    public MockExpandingSystemDescriptor(String systemName, StreamExpander expander) {
-      super(systemName, "factory.class", null, expander);
-    }
-
-    @Override
-    public MockInputDescriptor<Integer> getInputDescriptor(String streamId, Serde serde) {
-      return new MockInputDescriptor<>(streamId, this, serde);
-    }
-  }
-
-  class MockTransformingSystemDescriptor extends SystemDescriptor<MockTransformingSystemDescriptor> implements TransformingInputDescriptorProvider<Integer> {
-    public MockTransformingSystemDescriptor(String systemName, InputTransformer transformer) {
-      super(systemName, "factory.class", transformer, null);
-    }
-
-    @Override
-    public MockInputDescriptor<Integer> getInputDescriptor(String streamId, Serde serde) {
-      return new MockInputDescriptor<>(streamId, this, serde);
-    }
-  }
-
-  public class MockInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, MockInputDescriptor<StreamMessageType>> {
-    MockInputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde) {
-      super(streamId, serde, systemDescriptor, null);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index e79e25b..0000000
--- a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java
+++ /dev/null
@@ -1,172 +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.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Optional;
-import java.util.Set;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.ApplicationContainerContextFactory;
-import org.apache.samza.context.ApplicationTaskContextFactory;
-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.runtime.ProcessorLifecycleListenerFactory;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.task.TaskFactory;
-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.verify;
-import static org.mockito.Mockito.when;
-
-
-/**
- * Unit test for {@link TaskApplicationDescriptorImpl}
- */
-public class TestTaskApplicationDescriptorImpl {
-
-  private Config config = mock(Config.class);
-  private String defaultSystemName = "test-system";
-  private SystemDescriptor defaultSystemDescriptor = mock(SystemDescriptor.class);
-  private List<InputDescriptor> mockInputs = new ArrayList<InputDescriptor>() { {
-      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<OutputDescriptor> mockOutputs = new ArrayList<OutputDescriptor>() { {
-      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<TableDescriptor> mockTables = new HashSet<TableDescriptor>() { {
-      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);
-    } };
-
-  @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);
-    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(mockApp, config);
-    verify(mockApp).describe(appDesc);
-    assertEquals(config, appDesc.config);
-  }
-
-  @Test
-  public void testAddInputStreams() {
-    TaskApplication testApp = appDesc -> {
-      mockInputs.forEach(appDesc::addInputStream);
-    };
-    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
-    assertEquals(mockInputs.toArray(), appDesc.getInputDescriptors().values().toArray());
-  }
-
-  @Test
-  public void testAddOutputStreams() {
-    TaskApplication testApp = appDesc -> {
-      mockOutputs.forEach(appDesc::addOutputStream);
-    };
-    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
-    assertEquals(mockOutputs.toArray(), appDesc.getOutputDescriptors().values().toArray());
-  }
-
-  @Test
-  public void testAddTables() {
-    TaskApplication testApp = appDesc -> {
-      mockTables.forEach(appDesc::addTable);
-    };
-    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
-    assertEquals(mockTables, appDesc.getTableDescriptors());
-  }
-
-  @Test
-  public void testSetTaskFactory() {
-    TaskFactory mockTf = mock(TaskFactory.class);
-    TaskApplication testApp = appDesc -> appDesc.setTaskFactory(mockTf);
-    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
-    assertEquals(appDesc.getTaskFactory(), mockTf);
-  }
-
-  @Test
-  public void testApplicationContainerContextFactory() {
-    ApplicationContainerContextFactory factory = mock(ApplicationContainerContextFactory.class);
-    TaskApplication testApp = appDesc -> appDesc.withApplicationContainerContextFactory(factory);
-    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.of(factory));
-  }
-
-  @Test
-  public void testNoApplicationContainerContextFactory() {
-    TaskApplication testApp = appDesc -> {
-    };
-    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.empty());
-  }
-
-  @Test
-  public void testApplicationTaskContextFactory() {
-    ApplicationTaskContextFactory factory = mock(ApplicationTaskContextFactory.class);
-    TaskApplication testApp = appDesc -> appDesc.withApplicationTaskContextFactory(factory);
-    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.of(factory));
-  }
-
-  @Test
-  public void testNoApplicationTaskContextFactory() {
-    TaskApplication testApp = appDesc -> {
-    };
-    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
-    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.empty());
-  }
-
-  @Test
-  public void testProcessorLifecycleListener() {
-    ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class);
-    TaskApplication testApp = appDesc -> {
-      appDesc.withProcessorLifecycleListenerFactory(mockFactory);
-    };
-    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
-    assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory);
-  }
-}
\ No newline at end of file


[02/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseLocalStoreBackedTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 399f9fd..0000000
--- a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseLocalStoreBackedTableProvider.java
+++ /dev/null
@@ -1,149 +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 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.context.Context;
-import org.apache.samza.context.TaskContext;
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.util.NoOpMetricsRegistry;
-import org.junit.Test;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestBaseLocalStoreBackedTableProvider {
-
-  @Test
-  public void testInit() {
-    Context context = mock(Context.class);
-    TaskContext taskContext = mock(TaskContext.class);
-    when(context.getTaskContext()).thenReturn(taskContext);
-    when(taskContext.getStore(any())).thenReturn(mock(KeyValueStore.class));
-    when(taskContext.getTaskMetricsRegistry()).thenReturn(new NoOpMetricsRegistry());
-
-    TableSpec tableSpec = mock(TableSpec.class);
-    when(tableSpec.getId()).thenReturn("t1");
-
-    TableProvider tableProvider = createTableProvider(tableSpec);
-    tableProvider.init(context);
-    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<String, String> 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<String, String> 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<String, String> 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<String, String> jobConfig = new HashMap<>();
-    jobConfig.put(JobConfig.JOB_NAME(), "test-job");
-    jobConfig.put(JobConfig.JOB_ID(), "10");
-
-    TableProvider tableProvider = createTableProvider(tableSpec);
-    Map<String, String> 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<String, String> 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<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
-        return generateCommonStoreConfig(jobConfig, generatedConfig);
-      }
-    };
-  }
-
-  private BaseLocalStoreBackedTableDescriptor createTableDescriptor(String tableId) {
-    return new BaseLocalStoreBackedTableDescriptor(tableId) {
-      @Override
-      public TableSpec getTableSpec() {
-        validate();
-        Map<String, String> tableSpecConfig = new HashMap<>();
-        generateTableSpecConfig(tableSpecConfig);
-        return new TableSpec(tableId, serde, null, tableSpecConfig,
-            sideInputs, sideInputsProcessor);
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv/src/test/java/org/apache/samza/storage/kv/descriptors/TestBaseLocalStoreBackedTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/descriptors/TestBaseLocalStoreBackedTableProvider.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/descriptors/TestBaseLocalStoreBackedTableProvider.java
new file mode 100644
index 0000000..559f2e9
--- /dev/null
+++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/descriptors/TestBaseLocalStoreBackedTableProvider.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+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.context.Context;
+import org.apache.samza.context.TaskContext;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.util.NoOpMetricsRegistry;
+import org.junit.Test;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestBaseLocalStoreBackedTableProvider {
+
+  @Test
+  public void testInit() {
+    Context context = mock(Context.class);
+    TaskContext taskContext = mock(TaskContext.class);
+    when(context.getTaskContext()).thenReturn(taskContext);
+    when(taskContext.getStore(any())).thenReturn(mock(KeyValueStore.class));
+    when(taskContext.getTaskMetricsRegistry()).thenReturn(new NoOpMetricsRegistry());
+
+    TableSpec tableSpec = mock(TableSpec.class);
+    when(tableSpec.getId()).thenReturn("t1");
+
+    TableProvider tableProvider = createTableProvider(tableSpec);
+    tableProvider.init(context);
+    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<String, String> 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<String, String> 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<String, String> 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<String, String> jobConfig = new HashMap<>();
+    jobConfig.put(JobConfig.JOB_NAME(), "test-job");
+    jobConfig.put(JobConfig.JOB_ID(), "10");
+
+    TableProvider tableProvider = createTableProvider(tableSpec);
+    Map<String, String> 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<String, String> 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<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
+        return generateCommonStoreConfig(jobConfig, generatedConfig);
+      }
+    };
+  }
+
+  private BaseLocalStoreBackedTableDescriptor createTableDescriptor(String tableId) {
+    return new BaseLocalStoreBackedTableDescriptor(tableId) {
+      @Override
+      public TableSpec getTableSpec() {
+        validate();
+        Map<String, String> tableSpecConfig = new HashMap<>();
+        generateTableSpecConfig(tableSpecConfig);
+        return new TableSpec(tableId, serde, null, tableSpecConfig,
+            sideInputs, sideInputsProcessor);
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
index a1c1bdd..7faff17 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
@@ -21,7 +21,7 @@ package org.apache.samza.sql.impl;
 
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.sql.data.SamzaSqlCompositeKey;
@@ -29,7 +29,7 @@ import org.apache.samza.sql.data.SamzaSqlRelMessage;
 import org.apache.samza.sql.interfaces.SqlIOResolver;
 import org.apache.samza.sql.interfaces.SqlIOResolverFactory;
 import org.apache.samza.sql.interfaces.SqlIOConfig;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java
index 3a73e09..8636736 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java
@@ -29,7 +29,7 @@ import org.apache.commons.lang.Validate;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.StreamConfig;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.system.SystemStream;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java
----------------------------------------------------------------------
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 fd1a2a8..1caefe6 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
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Set;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.sql.dsl.SamzaSqlDslConverter;
 import org.apache.samza.sql.translator.QueryTranslator;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
index 435a2cc..94b2296 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
@@ -25,14 +25,14 @@ import java.util.Optional;
 import org.apache.calcite.rel.core.TableModify;
 import org.apache.commons.lang.Validate;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.context.Context;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.TableDescriptor;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
-import org.apache.samza.operators.descriptors.GenericOutputDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
----------------------------------------------------------------------
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 817f145..c9365cc 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,14 +31,11 @@ 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.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.context.Context;
 import org.apache.samza.operators.KV;
-import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TableDescriptor;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
-import org.apache.samza.operators.descriptors.GenericOutputDescriptor;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
 import org.apache.samza.sql.data.SamzaSqlExecutionContext;
@@ -50,7 +47,10 @@ import org.apache.samza.sql.planner.QueryPlanner;
 import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
 import org.apache.samza.sql.runner.SamzaSqlApplicationContext;
 import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
 import org.apache.samza.table.Table;
+import org.apache.samza.table.descriptors.TableDescriptor;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
----------------------------------------------------------------------
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 be94160..cc765bd 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,12 +23,12 @@ import java.util.List;
 import java.util.Map;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.commons.lang.Validate;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.context.Context;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java
----------------------------------------------------------------------
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 a7ab663..98cc92e 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,9 +32,9 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.SchemaPlus;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.sql.data.RexToJavaCompiler;
 import org.apache.samza.sql.data.SamzaSqlExecutionContext;
 import org.apache.samza.sql.interfaces.SamzaRelConverter;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java
----------------------------------------------------------------------
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 4c78b5a..14314c8 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
@@ -26,8 +26,8 @@ import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
@@ -36,13 +36,13 @@ import org.apache.samza.sql.data.SamzaSqlRelMessage;
 import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.sql.interfaces.SqlIOResolver;
 import org.apache.samza.sql.interfaces.SqlIOResolverFactory;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
 import org.apache.samza.table.ReadWriteTable;
 import org.apache.samza.table.Table;
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableProviderFactory;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
 import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.utils.BaseTableProvider;
+import org.apache.samza.table.utils.descriptors.BaseTableProvider;
 
 
 public class TestIOResolverFactory implements SqlIOResolverFactory {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java
----------------------------------------------------------------------
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 07ebe33..b9b0c96 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
@@ -24,7 +24,7 @@ import java.util.ArrayList;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.context.Context;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java
----------------------------------------------------------------------
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 f0a8a89..dcd7023 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,10 +33,10 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.operators.functions.StreamTableJoinFunction;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
@@ -49,7 +49,7 @@ import org.apache.samza.sql.data.RexToJavaCompiler;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
 import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.sql.interfaces.SqlIOResolver;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.internal.util.reflection.Whitebox;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java
----------------------------------------------------------------------
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 2ed7a00..68db1e4 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
@@ -31,7 +31,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.calcite.util.Pair;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.context.Context;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
----------------------------------------------------------------------
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 7a194db..cd81e0d 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
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.StreamConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
index a74993f..4943504 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
@@ -22,14 +22,13 @@ package org.apache.samza.sql.translator;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.operators.TableImpl;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
-import org.apache.samza.storage.kv.RocksDbTableProvider;
-import org.apache.samza.table.Table;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableProvider;
 import org.apache.samza.table.TableSpec;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java
----------------------------------------------------------------------
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 7d5e0d2..ba9c8b3 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 @@ package org.apache.samza.example;
 import java.time.Duration;
 import java.util.HashMap;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.triggers.Triggers;
@@ -33,9 +33,9 @@ import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java
----------------------------------------------------------------------
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 4ef2402..7721d44 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.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
@@ -29,9 +29,9 @@ import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
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 e991c4e..4923b7d 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.Collection;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.context.Context;
 import org.apache.samza.operators.KV;
@@ -36,9 +36,9 @@ import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.storage.kv.KeyValueStore;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/MergeExample.java
----------------------------------------------------------------------
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 fe018f3..ac0db36 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 @@ package org.apache.samza.example;
 import com.google.common.collect.ImmutableList;
 
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.runtime.ApplicationRunner;
@@ -31,9 +31,9 @@ import org.apache.samza.operators.KV;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 public class MergeExample implements StreamApplication {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
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 8d3812b..ea38984 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 @@ package org.apache.samza.example;
 
 import java.time.Duration;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.functions.JoinFunction;
@@ -29,9 +29,9 @@ import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
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 e2ebc93..1476c81 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.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
@@ -36,9 +36,9 @@ import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
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 8a0ca28..2cf3ac3 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 @@ package org.apache.samza.example;
 
 import java.time.Duration;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
@@ -32,9 +32,9 @@ import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java
----------------------------------------------------------------------
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 73dc10a..8f6c6f8 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,18 +18,18 @@
  */
 package org.apache.samza.example;
 
-import org.apache.samza.application.TaskApplicationDescriptor;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptor;
 import org.apache.samza.application.TaskApplication;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.descriptors.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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.StreamTask;
 import org.apache.samza.task.StreamTaskFactory;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/example/WindowExample.java
----------------------------------------------------------------------
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 2f4c19c..51089f7 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 @@ package org.apache.samza.example;
 
 import java.time.Duration;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OutputStream;
@@ -34,9 +34,9 @@ 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.system.kafka.KafkaInputDescriptor;
-import org.apache.samza.system.kafka.KafkaOutputDescriptor;
-import org.apache.samza.system.kafka.KafkaSystemDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.util.CommandLine;
 
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java b/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
index 42379f3..b47bf0a 100644
--- a/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
@@ -24,7 +24,7 @@ import java.time.Duration;
 import java.util.stream.Collectors;
 import java.util.List;
 import java.util.Map;
-import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
 import org.hamcrest.collection.IsIterableContainingInAnyOrder;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 
@@ -32,8 +32,8 @@ import static org.junit.Assert.assertThat;
 
 
 /**
- * Assertion utils on the content of a stream described by
- * {@link org.apache.samza.operators.descriptors.base.stream.StreamDescriptor}.
+ * Assertion utils on the content of a stream described by a
+ * {@link org.apache.samza.system.descriptors.StreamDescriptor}
  */
 public class StreamAssert {
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
----------------------------------------------------------------------
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 5cd47de..ba7128a 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
@@ -58,9 +58,9 @@ import org.apache.samza.system.SystemStreamPartition;
 import org.apache.samza.system.inmemory.InMemorySystemFactory;
 import org.apache.samza.task.AsyncStreamTask;
 import org.apache.samza.task.StreamTask;
-import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
-import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
-import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemorySystemDescriptor;
 import org.apache.samza.util.FileUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java
deleted file mode 100644
index 6065bf0..0000000
--- a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java
+++ /dev/null
@@ -1,42 +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.test.framework.system;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.serializers.NoOpSerde;
-
-/**
- * A descriptor for an in memory stream of messages that can either have single or multiple partitions.
- * <p>
- *  An instance of this descriptor may be obtained from an appropriately configured {@link InMemorySystemDescriptor}.
- * <p>
- * @param <StreamMessageType> type of messages in input stream
- */
-public class InMemoryInputDescriptor<StreamMessageType>
-    extends InputDescriptor<StreamMessageType, InMemoryInputDescriptor<StreamMessageType>> {
-  /**
-   * Constructs a new InMemoryInputDescriptor from specified components.
-   * @param systemDescriptor name of the system stream is associated with
-   * @param streamId name of the stream
-   */
-  InMemoryInputDescriptor(String streamId, InMemorySystemDescriptor systemDescriptor) {
-    super(streamId, new NoOpSerde<>(), systemDescriptor, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java
deleted file mode 100644
index 75fe7ae..0000000
--- a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java
+++ /dev/null
@@ -1,46 +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.test.framework.system;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.NoOpSerde;
-
-/**
- * A descriptor for an in memory output stream.
- * <p>
- * An instance of this descriptor may be obtained from an appropriately configured {@link InMemorySystemDescriptor}.
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- */
-public class InMemoryOutputDescriptor<StreamMessageType>
-    extends OutputDescriptor<StreamMessageType, InMemoryOutputDescriptor<StreamMessageType>> {
-
-  /**
-   * Constructs an {@link OutputDescriptor} instance.
-   * @param streamId id of the stream
-   * @param systemDescriptor system descriptor this stream descriptor was obtained from
-   */
-  InMemoryOutputDescriptor(String streamId, SystemDescriptor systemDescriptor) {
-    super(streamId, new NoOpSerde<>(), systemDescriptor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 77948f6..0000000
--- a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java
+++ /dev/null
@@ -1,109 +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.test.framework.system;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.samza.config.InMemorySystemConfig;
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SimpleInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.system.inmemory.InMemorySystemFactory;
-import org.apache.samza.config.JavaSystemConfig;
-
-/**
- * A descriptor for InMemorySystem.
- * System properties configured using a descriptor override corresponding properties provided in configuration.
- * <p>
- * Following system level configs are set by default
- * <ol>
- *   <li>"systems.%s.default.stream.samza.offset.default" = "oldest"</li>
- *   <li>"systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
- *   <li>"inmemory.scope = "Scope id generated to isolate the system in memory</li>
- * </ol>
- */
-public class InMemorySystemDescriptor extends SystemDescriptor<InMemorySystemDescriptor>
-    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
-  private static final String FACTORY_CLASS_NAME = InMemorySystemFactory.class.getName();
-  /**
-   * <p>
-   * 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.*"
-   * scope have the highest precedence.
-   *
-   * For this case, it generates following overridden configs
-   * <ol>
-   *      <li>"jobs.<job-name>.systems.%s.default.stream.samza.offset.default" = "oldest"</li>
-   *      <li>"jobs.<job-name>.systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
-   * </ol>
-   *
-   **/
-  private String inMemoryScope;
-
-  /**
-   * Constructs a new InMemorySystemDescriptor from specified components.
-   * <p>
-   * Every {@link InMemorySystemDescriptor} is configured to consume from the oldest offset, since stream is in memory and
-   * is used for testing purpose. System uses {@link InMemorySystemFactory} to initialize in memory streams.
-   * <p>
-   * @param systemName unique name of the system
-   */
-  public InMemorySystemDescriptor(String systemName) {
-    super(systemName, FACTORY_CLASS_NAME, null, null);
-    this.withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST);
-  }
-
-  @Override
-  public <StreamMessageType> InMemoryInputDescriptor<StreamMessageType> getInputDescriptor(
-      String streamId, Serde<StreamMessageType> serde) {
-    return new InMemoryInputDescriptor<StreamMessageType>(streamId, this);
-  }
-
-  @Override
-  public <StreamMessageType> InMemoryOutputDescriptor<StreamMessageType> getOutputDescriptor(
-      String streamId, Serde<StreamMessageType> serde) {
-    return new InMemoryOutputDescriptor<StreamMessageType>(streamId, this);
-  }
-
-  /**
-   * {@code inMemoryScope} defines the unique instance of InMemorySystem, that this system uses
-   * This method is framework use only, users are not supposed to use it
-   *
-   * @param inMemoryScope acts as a unique global identifier for this instance of InMemorySystem
-   * @return this system descriptor
-   */
-  public InMemorySystemDescriptor withInMemoryScope(String inMemoryScope) {
-    this.inMemoryScope = inMemoryScope;
-    return this;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    HashMap<String, String> configs = new HashMap<>(super.toConfig());
-    configs.put(InMemorySystemConfig.INMEMORY_SCOPE, this.inMemoryScope);
-    configs.put(String.format(JavaSystemConfig.SYSTEM_FACTORY_FORMAT, getSystemName()), FACTORY_CLASS_NAME);
-    return configs;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java
new file mode 100644
index 0000000..0e49550
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.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.test.framework.system.descriptors;
+
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.serializers.NoOpSerde;
+
+/**
+ * A descriptor for an in memory stream of messages that can either have single or multiple partitions.
+ * <p>
+ *  An instance of this descriptor may be obtained from an appropriately configured {@link InMemorySystemDescriptor}.
+ * <p>
+ * @param <StreamMessageType> type of messages in input stream
+ */
+public class InMemoryInputDescriptor<StreamMessageType>
+    extends InputDescriptor<StreamMessageType, InMemoryInputDescriptor<StreamMessageType>> {
+  /**
+   * Constructs a new InMemoryInputDescriptor from specified components.
+   * @param systemDescriptor name of the system stream is associated with
+   * @param streamId name of the stream
+   */
+  InMemoryInputDescriptor(String streamId, InMemorySystemDescriptor systemDescriptor) {
+    super(streamId, new NoOpSerde<>(), systemDescriptor, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
new file mode 100644
index 0000000..26c64f3
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
@@ -0,0 +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.test.framework.system.descriptors;
+
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.NoOpSerde;
+
+/**
+ * A descriptor for an in memory output stream.
+ * <p>
+ * An instance of this descriptor may be obtained from an appropriately configured {@link InMemorySystemDescriptor}.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ */
+public class InMemoryOutputDescriptor<StreamMessageType>
+    extends OutputDescriptor<StreamMessageType, InMemoryOutputDescriptor<StreamMessageType>> {
+
+  /**
+   * Constructs an {@link OutputDescriptor} instance.
+   * @param streamId id of the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   */
+  InMemoryOutputDescriptor(String streamId, SystemDescriptor systemDescriptor) {
+    super(streamId, new NoOpSerde<>(), systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemorySystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemorySystemDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemorySystemDescriptor.java
new file mode 100644
index 0000000..96a8aca
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemorySystemDescriptor.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.test.framework.system.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.config.InMemorySystemConfig;
+import org.apache.samza.system.descriptors.OutputDescriptorProvider;
+import org.apache.samza.system.descriptors.SimpleInputDescriptorProvider;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.inmemory.InMemorySystemFactory;
+import org.apache.samza.config.JavaSystemConfig;
+
+/**
+ * A descriptor for InMemorySystem.
+ * System properties configured using a descriptor override corresponding properties provided in configuration.
+ * <p>
+ * Following system level configs are set by default
+ * <ol>
+ *   <li>"systems.%s.default.stream.samza.offset.default" = "oldest"</li>
+ *   <li>"systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
+ *   <li>"inmemory.scope = "Scope id generated to isolate the system in memory</li>
+ * </ol>
+ */
+public class InMemorySystemDescriptor extends SystemDescriptor<InMemorySystemDescriptor>
+    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
+  private static final String FACTORY_CLASS_NAME = InMemorySystemFactory.class.getName();
+  /**
+   * <p>
+   * 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.*"
+   * scope have the highest precedence.
+   *
+   * For this case, it generates following overridden configs
+   * <ol>
+   *      <li>"jobs.<job-name>.systems.%s.default.stream.samza.offset.default" = "oldest"</li>
+   *      <li>"jobs.<job-name>.systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
+   * </ol>
+   *
+   **/
+  private String inMemoryScope;
+
+  /**
+   * Constructs a new InMemorySystemDescriptor from specified components.
+   * <p>
+   * Every {@link InMemorySystemDescriptor} is configured to consume from the oldest offset, since stream is in memory and
+   * is used for testing purpose. System uses {@link InMemorySystemFactory} to initialize in memory streams.
+   * <p>
+   * @param systemName unique name of the system
+   */
+  public InMemorySystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME, null, null);
+    this.withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST);
+  }
+
+  @Override
+  public <StreamMessageType> InMemoryInputDescriptor<StreamMessageType> getInputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new InMemoryInputDescriptor<StreamMessageType>(streamId, this);
+  }
+
+  @Override
+  public <StreamMessageType> InMemoryOutputDescriptor<StreamMessageType> getOutputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new InMemoryOutputDescriptor<StreamMessageType>(streamId, this);
+  }
+
+  /**
+   * {@code inMemoryScope} defines the unique instance of InMemorySystem, that this system uses
+   * This method is framework use only, users are not supposed to use it
+   *
+   * @param inMemoryScope acts as a unique global identifier for this instance of InMemorySystem
+   * @return this system descriptor
+   */
+  public InMemorySystemDescriptor withInMemoryScope(String inMemoryScope) {
+    this.inMemoryScope = inMemoryScope;
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    HashMap<String, String> configs = new HashMap<>(super.toConfig());
+    configs.put(InMemorySystemConfig.INMEMORY_SCOPE, this.inMemoryScope);
+    configs.put(String.format(JavaSystemConfig.SYSTEM_FACTORY_FORMAT, getSystemName()), FACTORY_CLASS_NAME);
+    return configs;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/main/java/org/apache/samza/test/integration/TestStandaloneIntegrationApplication.java
----------------------------------------------------------------------
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 1954cc3..2e51f6a 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,13 +19,13 @@
 package org.apache.samza.test.integration;
 
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.operators.KV;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
----------------------------------------------------------------------
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 4c8884d..672837b 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,16 +24,16 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
 import org.apache.samza.operators.KV;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.serializers.KVSerde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
----------------------------------------------------------------------
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 e0097bd..ba62691 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.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.samza.Partition;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
@@ -40,10 +40,10 @@ import org.apache.samza.container.SamzaContainer;
 import org.apache.samza.container.TaskInstance;
 import org.apache.samza.container.TaskName;
 import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.operators.KV;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
+import org.apache.samza.system.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.operators.impl.InputOperatorImpl;
 import org.apache.samza.operators.impl.OperatorImpl;
 import org.apache.samza.operators.impl.OperatorImplGraph;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
index 7696b62..e326f91 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
@@ -29,9 +29,9 @@ import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.operators.KV;
 import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
-import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
-import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemorySystemDescriptor;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/framework/BroadcastAssertApp.java
----------------------------------------------------------------------
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 4caf266..ef17a22 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,12 +21,12 @@ package org.apache.samza.test.framework;
 
 import java.util.Arrays;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.system.kafka.KafkaInputDescriptor;
-import org.apache.samza.system.kafka.KafkaSystemDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.test.operator.data.PageView;
 
 public class BroadcastAssertApp implements StreamApplication {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java
index 7deb4d7..649c032 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java
@@ -22,7 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import org.apache.samza.application.TaskApplication;
-import org.apache.samza.application.TaskApplicationDescriptor;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;
@@ -31,8 +31,8 @@ import org.apache.samza.config.ZkConfig;
 import org.apache.samza.job.ApplicationStatus;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.kafka.KafkaInputDescriptor;
-import org.apache.samza.system.kafka.KafkaSystemDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.task.ClosableTask;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.StreamTask;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java
----------------------------------------------------------------------
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 6188381..a442140 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
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Random;
 import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OutputStream;
@@ -33,22 +33,23 @@ import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.storage.kv.RocksDbTableDescriptor;
+import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
 import org.apache.samza.system.OutgoingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.table.Table;
 import org.apache.samza.test.controlmessages.TestData;
-import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
-import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
-import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemorySystemDescriptor;
 import org.apache.samza.test.table.PageViewToProfileJoinFunction;
 import org.apache.samza.test.table.TestTableData;
 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 {
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
index 003b200..aa9e107 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
@@ -28,35 +28,36 @@ import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.application.TaskApplication;
-import org.apache.samza.application.TaskApplicationDescriptor;
 import org.apache.samza.context.Context;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptor;
 import org.apache.samza.operators.KV;
 import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.storage.kv.inmemory.InMemoryTableDescriptor;
+import org.apache.samza.storage.kv.inmemory.descriptors.InMemoryTableDescriptor;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.OutgoingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
-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.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.table.ReadWriteTable;
 import org.apache.samza.task.InitableTask;
 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.test.framework.system.InMemoryInputDescriptor;
-import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
-import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.descriptors.InMemorySystemDescriptor;
 import org.apache.samza.test.table.TestTableData;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 import org.junit.Assert;
 import org.junit.Test;
-import static org.apache.samza.test.table.TestTableData.Profile;
-import static org.apache.samza.test.table.TestTableData.PageView;
+
 import static org.apache.samza.test.table.TestTableData.EnrichedPageView;
+import static org.apache.samza.test.table.TestTableData.PageView;
+import static org.apache.samza.test.table.TestTableData.Profile;
 
 
 public class StreamTaskIntegrationTest {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java b/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java
index 1644a0f..20f18ee 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java
@@ -25,14 +25,14 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.Scheduler;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.functions.ScheduledFunction;
 import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.system.kafka.KafkaInputDescriptor;
-import org.apache.samza.system.kafka.KafkaSystemDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.test.operator.data.PageView;
 
 public class TestSchedulingApp implements StreamApplication {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java
----------------------------------------------------------------------
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 c63c11f..dda31ea 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.time.Duration;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
@@ -35,8 +35,8 @@ import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.system.OutgoingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.kafka.KafkaInputDescriptor;
-import org.apache.samza.system.kafka.KafkaSystemDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.test.operator.data.AdClick;
 import org.apache.samza.test.operator.data.PageView;


[03/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableProvider.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableProvider.java
new file mode 100644
index 0000000..0b48e6b
--- /dev/null
+++ b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableProvider.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.storage.kv.inmemory.descriptors;
+
+import java.util.HashMap;
+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;
+import org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.TableSpec;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+
+public class TestInMemoryTableProvider {
+  @Test
+  public void testGenerateConfig() {
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    tableSpecConfig.put("inmemory.c1", "c1-value");
+    tableSpecConfig.put("inmemory.c2", "c2-value");
+    tableSpecConfig.put("c3", "c3-value");
+    tableSpecConfig.put("c4", "c4-value");
+
+    TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()),
+        "my-table-provider-factory", tableSpecConfig);
+
+    Map<String, String> 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<String, String> 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")));
+    Assert.assertEquals(
+        InMemoryKeyValueStorageEngineFactory.class.getName(),
+        tableConfig.get(String.format(StorageConfig.FACTORY(), "t1")));
+    Assert.assertEquals("c1-value", tableConfig.get("stores.t1.c1"));
+    Assert.assertEquals("c2-value", tableConfig.get("stores.t1.c2"));
+    Assert.assertEquals("c3-value", tableConfig.get("tables.t1.c3"));
+    Assert.assertEquals("c4-value", tableConfig.get("tables.t1.c4"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 50bc2c2..0000000
--- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java
+++ /dev/null
@@ -1,339 +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.serializers.KVSerde;
-import org.apache.samza.table.TableSpec;
-
-
-/**
- * Table descriptor for RocksDb backed tables
- *
- * @param <K> the type of the key
- * @param <V> the type of the value
- */
-public class RocksDbTableDescriptor<K, V> extends BaseLocalStoreBackedTableDescriptor<K, V, RocksDbTableDescriptor<K, V>> {
-
-  static final public String WRITE_BATCH_SIZE = "write.batch.size";
-  static final public String OBJECT_CACHE_SIZE = "object.cache.size";
-  static final public String CONTAINER_CACHE_SIZE_BYTES = "container.cache.size.bytes";
-  static final public String CONTAINER_WRITE_BUFFER_SIZE_BYTES = "container.write.buffer.size.bytes";
-  static final public String ROCKSDB_COMPRESSION = "rocksdb.compression";
-  static final public String ROCKSDB_BLOCK_SIZE_BYTES = "rocksdb.block.size.bytes";
-  static final public String ROCKSDB_TTL_MS = "rocksdb.ttl.ms";
-  static final public String ROCKSDB_COMPACTION_STYLE = "rocksdb.compaction.style";
-  static final public String ROCKSDB_NUM_WRITE_BUFFERS = "rocksdb.num.write.buffers";
-  static final public String ROCKSDB_MAX_LOG_FILE_SIZE_BYTES = "rocksdb.max.log.file.size.bytes";
-  static final public String ROCKSDB_KEEP_LOG_FILE_NUM = "rocksdb.keep.log.file.num";
-
-  private Integer writeBatchSize;
-  private Integer objectCacheSize;
-  private Integer cacheSize;
-  private Integer writeBufferSize;
-  private Integer blockSize;
-  private Integer ttl;
-  private Integer numWriteBuffers;
-  private Integer maxLogFileSize;
-  private Integer numLogFilesToKeep;
-  private String compressionType;
-  private String compactionStyle;
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   */
-  public RocksDbTableDescriptor(String tableId) {
-    super(tableId);
-  }
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   * @param serde the serde for key and value
-   */
-  public RocksDbTableDescriptor(String tableId, KVSerde<K, V> serde) {
-    super(tableId, serde);
-  }
-
-  /**
-   * For better write performance, the storage engine buffers writes and applies them to the
-   * underlying store in a batch. If the same key is written multiple times in quick succession,
-   * this buffer also deduplicates writes to the same key. This property is set to the number
-   * of key/value pairs that should be kept in this in-memory buffer, per task instance.
-   * The number cannot be greater than {@link #withObjectCacheSize}.
-   * <p>
-   * Default value is 500.
-   * <p>
-   * Refer to <code>stores.store-name.write.batch.size</code> in Samza configuration guide
-   *
-   * @param writeBatchSize write batch size
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor withWriteBatchSize(int writeBatchSize) {
-    this.writeBatchSize = writeBatchSize;
-    return this;
-  }
-
-  /**
-   * Samza maintains an additional cache in front of RocksDB for frequently-accessed objects.
-   * This cache contains deserialized objects (avoiding the deserialization overhead on cache
-   * hits), in contrast to the RocksDB block cache ({@link #withCacheSize}), which caches
-   * serialized objects. This property determines the number of objects to keep in Samza's
-   * cache, per task instance. This same cache is also used for write buffering
-   * (see {@link #withWriteBatchSize}). A value of 0 disables all caching and batching.
-   * <p>
-   * Default value is 1,000.
-   * <p>
-   * Refer to <code>stores.store-name.object.cache.size</code> in Samza configuration guide
-   *
-   * @param objectCacheSize the object cache size
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor withObjectCacheSize(int objectCacheSize) {
-    this.objectCacheSize = objectCacheSize;
-    return this;
-  }
-
-  /**
-   * The size of RocksDB's block cache in bytes, per container. If there are several task
-   * instances within one container, each is given a proportional share of this cache.
-   * Note that this is an off-heap memory allocation, so the container's total memory
-   * use is the maximum JVM heap size plus the size of this cache.
-   * <p>
-   * Default value is 104,857,600.
-   * <p>
-   * Refer to <code>stores.store-name.container.cache.size.bytes</code> in Samza configuration guide
-   *
-   * @param cacheSize the cache size in bytes
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withCacheSize(int cacheSize) {
-    this.cacheSize = cacheSize;
-    return this;
-  }
-
-  /**
-   * The amount of memory (in bytes) that RocksDB uses for buffering writes before they are
-   * written to disk, per container. If there are several task instances within one container,
-   * each is given a proportional share of this buffer. This setting also determines the
-   * size of RocksDB's segment files.
-   * <p>
-   * Default value is 33,554,432.
-   * <p>
-   * Refer to <code>stores.store-name.container.write.buffer.size.bytes</code> in Samza configuration guide
-   *
-   * @param writeBufferSize the write buffer size in bytes
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withWriteBufferSize(int writeBufferSize) {
-    this.writeBufferSize = writeBufferSize;
-    return this;
-  }
-
-  /**
-   * Controls whether RocksDB should compress data on disk and in the block cache.
-   * The following values are valid:
-   * <ul>
-   *   <li><b>snappy</b> Compress data using the <a href="https://code.google.com/p/snappy/">Snappy</a> codec.
-   *   <li><b>bzip2</b> Compress data using the <a href="http://en.wikipedia.org/wiki/Bzip2">bzip2</a> codec.
-   *   <li><b>zlib</b> Compress data using the <a href="http://en.wikipedia.org/wiki/Zlib">zlib</a> codec.
-   *   <li><b>lz4</b> Compress data using the <a href="https://code.google.com/p/lz4/">lz4</a> codec.
-   *   <li><b>lz4hc</b> Compress data using the <a href="https://code.google.com/p/lz4/">lz4hc</a> (high compression) codec.
-   *   <li><b>none</b> Do not compress data.
-   * </ul>
-   * <p>
-   * Default value is snappy.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.compression</code> in Samza configuration guide
-   *
-   * @param compressionType the compression type
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withCompressionType(String compressionType) {
-    this.compressionType = compressionType;
-    return this;
-  }
-
-  /**
-   * If compression is enabled, RocksDB groups approximately this many uncompressed
-   * bytes into one compressed block. You probably don't need to change this property.
-   * <p>
-   * Default value is 4,096.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.block.size.bytes</code> in Samza configuration guide
-   *
-   * @param blockSize the block size in bytes
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withBlockSize(int blockSize) {
-    this.blockSize = blockSize;
-    return this;
-  }
-
-  /**
-   * The time-to-live of the store. Please note it's not a strict TTL limit (removed
-   * only after compaction). Please use caution opening a database with and without
-   * TTL, as it might corrupt the database. Please make sure to read the
-   * <a href="https://github.com/facebook/rocksdb/wiki/Time-to-Live">constraints</a>
-   * before using.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.ttl.ms</code> in Samza configuration guide
-   *
-   * @param ttl the time to live in milliseconds
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withTtl(int ttl) {
-    this.ttl = ttl;
-    return this;
-  }
-
-  /**
-   * This property controls the compaction style that RocksDB will employ when compacting
-   * its levels. The following values are valid:
-   * <ul>
-   *   <li><b>universal</b> Use <a href="https://github.com/facebook/rocksdb/wiki/Universal-Compaction">universal</a> compaction.
-   *   <li><b>fifo</b> Use <a href="https://github.com/facebook/rocksdb/wiki/FIFO-compaction-style">FIFO</a> compaction.
-   *   <li><b>level</b> Use RocksDB's standard leveled compaction.
-   * </ul>
-   * <p>
-   * Default value is universal.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.compaction.style</code> in Samza configuration guide
-   *
-   * @param compactionStyle the compaction style
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withCompactionStyle(String compactionStyle) {
-    this.compactionStyle = compactionStyle;
-    return this;
-  }
-
-  /**
-   * Configures the
-   * <a href="https://github.com/facebook/rocksdb/wiki/Basic-Operations#write-buffer">
-   * number of write buffers</a> that a RocksDB store uses. This allows RocksDB
-   * to continue taking writes to other buffers even while a given write buffer is being
-   * flushed to disk.
-   * <p>
-   * Default value is 3.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.num.write.buffers</code> in Samza configuration guide
-   *
-   * @param numWriteBuffers the number of write buffers
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withNumWriteBuffers(int numWriteBuffers) {
-    this.numWriteBuffers = numWriteBuffers;
-    return this;
-  }
-
-  /**
-   * The maximum size in bytes of the RocksDB LOG file before it is rotated.
-   * <p>
-   * Default value is 67,108,864.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.max.log.file.size.bytes</code> in Samza configuration guide
-   *
-   * @param maxLogFileSize the maximal log file size in bytes
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withMaxLogFileSize(int maxLogFileSize) {
-    this.maxLogFileSize = maxLogFileSize;
-    return this;
-  }
-
-  /**
-   * The number of RocksDB LOG files (including rotated LOG.old.* files) to keep.
-   * <p>
-   * Default value is 2.
-   * <p>
-   * Refer to <code>stores.store-name.rocksdb.keep.log.file.num</code> in Samza configuration guide
-   *
-   * @param numLogFilesToKeep the number of log files to keep
-   * @return this table descriptor instance
-   */
-  public RocksDbTableDescriptor<K, V> withNumLogFilesToKeep(int numLogFilesToKeep) {
-    this.numLogFilesToKeep = numLogFilesToKeep;
-    return this;
-  }
-
-  /**
-   * Create a table spec based on this table description
-   * @return the table spec
-   */
-  @Override
-  public TableSpec getTableSpec() {
-
-    validate();
-
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    generateTableSpecConfig(tableSpecConfig);
-
-    return new TableSpec(tableId, serde, RocksDbTableProviderFactory.class.getName(), tableSpecConfig,
-        sideInputs, sideInputsProcessor);
-  }
-
-  @Override
-  protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
-
-    super.generateTableSpecConfig(tableSpecConfig);
-
-    if (writeBatchSize != null) {
-      addRocksDbConfig(tableSpecConfig, WRITE_BATCH_SIZE, writeBatchSize.toString());
-    }
-    if (objectCacheSize != null) {
-      addRocksDbConfig(tableSpecConfig, OBJECT_CACHE_SIZE, objectCacheSize.toString());
-    }
-    if (cacheSize != null) {
-      addRocksDbConfig(tableSpecConfig, CONTAINER_CACHE_SIZE_BYTES, cacheSize.toString());
-    }
-    if (writeBufferSize != null) {
-      addRocksDbConfig(tableSpecConfig, CONTAINER_WRITE_BUFFER_SIZE_BYTES, writeBufferSize.toString());
-    }
-    if (compressionType != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_COMPRESSION, compressionType);
-    }
-    if (blockSize != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_BLOCK_SIZE_BYTES, blockSize.toString());
-    }
-    if (ttl != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_TTL_MS, ttl.toString());
-    }
-    if (compactionStyle != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_COMPACTION_STYLE, compactionStyle);
-    }
-    if (numWriteBuffers != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_NUM_WRITE_BUFFERS, numWriteBuffers.toString());
-    }
-    if (maxLogFileSize != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_MAX_LOG_FILE_SIZE_BYTES, maxLogFileSize.toString());
-    }
-    if (numLogFilesToKeep != null) {
-      addRocksDbConfig(tableSpecConfig, ROCKSDB_KEEP_LOG_FILE_NUM, numLogFilesToKeep.toString());
-    }
-  }
-
-  private void addRocksDbConfig(Map<String, String> map, String key, String value) {
-    map.put("rocksdb." + key, value);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index df60a5a..0000000
--- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java
+++ /dev/null
@@ -1,73 +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.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;
-
-
-/**
- * Table provider for tables backed by RocksDb.
- */
-public class RocksDbTableProvider extends BaseLocalStoreBackedTableProvider {
-
-  public RocksDbTableProvider(TableSpec tableSpec) {
-    super(tableSpec);
-  }
-
-  @Override
-  public Map<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
-
-    Map<String, String> tableConfig = new HashMap<>();
-
-    // Store factory configuration
-    tableConfig.put(String.format(
-        StorageConfig.FACTORY(), tableSpec.getId()),
-        RocksDbKeyValueStorageEngineFactory.class.getName());
-
-    // Common store configuration
-    tableConfig.putAll(generateCommonStoreConfig(jobConfig, generatedConfig));
-
-    // Rest of the configuration
-    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");
-
-    logger.info("Generated configuration for table " + tableSpec.getId());
-
-    return tableConfig;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java
deleted file mode 100644
index dbe0f97..0000000
--- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java
+++ /dev/null
@@ -1,31 +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 org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableProviderFactory;
-import org.apache.samza.table.TableSpec;
-
-
-public class RocksDbTableProviderFactory implements TableProviderFactory {
-  @Override
-  public TableProvider getTableProvider(TableSpec tableSpec) {
-    return new RocksDbTableProvider(tableSpec);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableDescriptor.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableDescriptor.java
new file mode 100644
index 0000000..a8ede78
--- /dev/null
+++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableDescriptor.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.storage.kv.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Table descriptor for RocksDb backed tables
+ *
+ * @param <K> the type of the key
+ * @param <V> the type of the value
+ */
+public class RocksDbTableDescriptor<K, V> extends BaseLocalStoreBackedTableDescriptor<K, V, RocksDbTableDescriptor<K, V>> {
+
+  static final public String WRITE_BATCH_SIZE = "write.batch.size";
+  static final public String OBJECT_CACHE_SIZE = "object.cache.size";
+  static final public String CONTAINER_CACHE_SIZE_BYTES = "container.cache.size.bytes";
+  static final public String CONTAINER_WRITE_BUFFER_SIZE_BYTES = "container.write.buffer.size.bytes";
+  static final public String ROCKSDB_COMPRESSION = "rocksdb.compression";
+  static final public String ROCKSDB_BLOCK_SIZE_BYTES = "rocksdb.block.size.bytes";
+  static final public String ROCKSDB_TTL_MS = "rocksdb.ttl.ms";
+  static final public String ROCKSDB_COMPACTION_STYLE = "rocksdb.compaction.style";
+  static final public String ROCKSDB_NUM_WRITE_BUFFERS = "rocksdb.num.write.buffers";
+  static final public String ROCKSDB_MAX_LOG_FILE_SIZE_BYTES = "rocksdb.max.log.file.size.bytes";
+  static final public String ROCKSDB_KEEP_LOG_FILE_NUM = "rocksdb.keep.log.file.num";
+
+  private Integer writeBatchSize;
+  private Integer objectCacheSize;
+  private Integer cacheSize;
+  private Integer writeBufferSize;
+  private Integer blockSize;
+  private Integer ttl;
+  private Integer numWriteBuffers;
+  private Integer maxLogFileSize;
+  private Integer numLogFilesToKeep;
+  private String compressionType;
+  private String compactionStyle;
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   */
+  public RocksDbTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   * @param serde the serde for key and value
+   */
+  public RocksDbTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
+  /**
+   * For better write performance, the storage engine buffers writes and applies them to the
+   * underlying store in a batch. If the same key is written multiple times in quick succession,
+   * this buffer also deduplicates writes to the same key. This property is set to the number
+   * of key/value pairs that should be kept in this in-memory buffer, per task instance.
+   * The number cannot be greater than {@link #withObjectCacheSize}.
+   * <p>
+   * Default value is 500.
+   * <p>
+   * Refer to <code>stores.store-name.write.batch.size</code> in Samza configuration guide
+   *
+   * @param writeBatchSize write batch size
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor withWriteBatchSize(int writeBatchSize) {
+    this.writeBatchSize = writeBatchSize;
+    return this;
+  }
+
+  /**
+   * Samza maintains an additional cache in front of RocksDB for frequently-accessed objects.
+   * This cache contains deserialized objects (avoiding the deserialization overhead on cache
+   * hits), in contrast to the RocksDB block cache ({@link #withCacheSize}), which caches
+   * serialized objects. This property determines the number of objects to keep in Samza's
+   * cache, per task instance. This same cache is also used for write buffering
+   * (see {@link #withWriteBatchSize}). A value of 0 disables all caching and batching.
+   * <p>
+   * Default value is 1,000.
+   * <p>
+   * Refer to <code>stores.store-name.object.cache.size</code> in Samza configuration guide
+   *
+   * @param objectCacheSize the object cache size
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor withObjectCacheSize(int objectCacheSize) {
+    this.objectCacheSize = objectCacheSize;
+    return this;
+  }
+
+  /**
+   * The size of RocksDB's block cache in bytes, per container. If there are several task
+   * instances within one container, each is given a proportional share of this cache.
+   * Note that this is an off-heap memory allocation, so the container's total memory
+   * use is the maximum JVM heap size plus the size of this cache.
+   * <p>
+   * Default value is 104,857,600.
+   * <p>
+   * Refer to <code>stores.store-name.container.cache.size.bytes</code> in Samza configuration guide
+   *
+   * @param cacheSize the cache size in bytes
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withCacheSize(int cacheSize) {
+    this.cacheSize = cacheSize;
+    return this;
+  }
+
+  /**
+   * The amount of memory (in bytes) that RocksDB uses for buffering writes before they are
+   * written to disk, per container. If there are several task instances within one container,
+   * each is given a proportional share of this buffer. This setting also determines the
+   * size of RocksDB's segment files.
+   * <p>
+   * Default value is 33,554,432.
+   * <p>
+   * Refer to <code>stores.store-name.container.write.buffer.size.bytes</code> in Samza configuration guide
+   *
+   * @param writeBufferSize the write buffer size in bytes
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withWriteBufferSize(int writeBufferSize) {
+    this.writeBufferSize = writeBufferSize;
+    return this;
+  }
+
+  /**
+   * Controls whether RocksDB should compress data on disk and in the block cache.
+   * The following values are valid:
+   * <ul>
+   *   <li><b>snappy</b> Compress data using the <a href="https://code.google.com/p/snappy/">Snappy</a> codec.
+   *   <li><b>bzip2</b> Compress data using the <a href="http://en.wikipedia.org/wiki/Bzip2">bzip2</a> codec.
+   *   <li><b>zlib</b> Compress data using the <a href="http://en.wikipedia.org/wiki/Zlib">zlib</a> codec.
+   *   <li><b>lz4</b> Compress data using the <a href="https://code.google.com/p/lz4/">lz4</a> codec.
+   *   <li><b>lz4hc</b> Compress data using the <a href="https://code.google.com/p/lz4/">lz4hc</a> (high compression) codec.
+   *   <li><b>none</b> Do not compress data.
+   * </ul>
+   * <p>
+   * Default value is snappy.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.compression</code> in Samza configuration guide
+   *
+   * @param compressionType the compression type
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withCompressionType(String compressionType) {
+    this.compressionType = compressionType;
+    return this;
+  }
+
+  /**
+   * If compression is enabled, RocksDB groups approximately this many uncompressed
+   * bytes into one compressed block. You probably don't need to change this property.
+   * <p>
+   * Default value is 4,096.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.block.size.bytes</code> in Samza configuration guide
+   *
+   * @param blockSize the block size in bytes
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withBlockSize(int blockSize) {
+    this.blockSize = blockSize;
+    return this;
+  }
+
+  /**
+   * The time-to-live of the store. Please note it's not a strict TTL limit (removed
+   * only after compaction). Please use caution opening a database with and without
+   * TTL, as it might corrupt the database. Please make sure to read the
+   * <a href="https://github.com/facebook/rocksdb/wiki/Time-to-Live">constraints</a>
+   * before using.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.ttl.ms</code> in Samza configuration guide
+   *
+   * @param ttl the time to live in milliseconds
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withTtl(int ttl) {
+    this.ttl = ttl;
+    return this;
+  }
+
+  /**
+   * This property controls the compaction style that RocksDB will employ when compacting
+   * its levels. The following values are valid:
+   * <ul>
+   *   <li><b>universal</b> Use <a href="https://github.com/facebook/rocksdb/wiki/Universal-Compaction">universal</a> compaction.
+   *   <li><b>fifo</b> Use <a href="https://github.com/facebook/rocksdb/wiki/FIFO-compaction-style">FIFO</a> compaction.
+   *   <li><b>level</b> Use RocksDB's standard leveled compaction.
+   * </ul>
+   * <p>
+   * Default value is universal.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.compaction.style</code> in Samza configuration guide
+   *
+   * @param compactionStyle the compaction style
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withCompactionStyle(String compactionStyle) {
+    this.compactionStyle = compactionStyle;
+    return this;
+  }
+
+  /**
+   * Configures the
+   * <a href="https://github.com/facebook/rocksdb/wiki/Basic-Operations#write-buffer">
+   * number of write buffers</a> that a RocksDB store uses. This allows RocksDB
+   * to continue taking writes to other buffers even while a given write buffer is being
+   * flushed to disk.
+   * <p>
+   * Default value is 3.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.num.write.buffers</code> in Samza configuration guide
+   *
+   * @param numWriteBuffers the number of write buffers
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withNumWriteBuffers(int numWriteBuffers) {
+    this.numWriteBuffers = numWriteBuffers;
+    return this;
+  }
+
+  /**
+   * The maximum size in bytes of the RocksDB LOG file before it is rotated.
+   * <p>
+   * Default value is 67,108,864.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.max.log.file.size.bytes</code> in Samza configuration guide
+   *
+   * @param maxLogFileSize the maximal log file size in bytes
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withMaxLogFileSize(int maxLogFileSize) {
+    this.maxLogFileSize = maxLogFileSize;
+    return this;
+  }
+
+  /**
+   * The number of RocksDB LOG files (including rotated LOG.old.* files) to keep.
+   * <p>
+   * Default value is 2.
+   * <p>
+   * Refer to <code>stores.store-name.rocksdb.keep.log.file.num</code> in Samza configuration guide
+   *
+   * @param numLogFilesToKeep the number of log files to keep
+   * @return this table descriptor instance
+   */
+  public RocksDbTableDescriptor<K, V> withNumLogFilesToKeep(int numLogFilesToKeep) {
+    this.numLogFilesToKeep = numLogFilesToKeep;
+    return this;
+  }
+
+  /**
+   * Create a table spec based on this table description
+   * @return the table spec
+   */
+  @Override
+  public TableSpec getTableSpec() {
+
+    validate();
+
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    generateTableSpecConfig(tableSpecConfig);
+
+    return new TableSpec(tableId, serde, RocksDbTableProviderFactory.class.getName(), tableSpecConfig,
+        sideInputs, sideInputsProcessor);
+  }
+
+  @Override
+  protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
+
+    super.generateTableSpecConfig(tableSpecConfig);
+
+    if (writeBatchSize != null) {
+      addRocksDbConfig(tableSpecConfig, WRITE_BATCH_SIZE, writeBatchSize.toString());
+    }
+    if (objectCacheSize != null) {
+      addRocksDbConfig(tableSpecConfig, OBJECT_CACHE_SIZE, objectCacheSize.toString());
+    }
+    if (cacheSize != null) {
+      addRocksDbConfig(tableSpecConfig, CONTAINER_CACHE_SIZE_BYTES, cacheSize.toString());
+    }
+    if (writeBufferSize != null) {
+      addRocksDbConfig(tableSpecConfig, CONTAINER_WRITE_BUFFER_SIZE_BYTES, writeBufferSize.toString());
+    }
+    if (compressionType != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_COMPRESSION, compressionType);
+    }
+    if (blockSize != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_BLOCK_SIZE_BYTES, blockSize.toString());
+    }
+    if (ttl != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_TTL_MS, ttl.toString());
+    }
+    if (compactionStyle != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_COMPACTION_STYLE, compactionStyle);
+    }
+    if (numWriteBuffers != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_NUM_WRITE_BUFFERS, numWriteBuffers.toString());
+    }
+    if (maxLogFileSize != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_MAX_LOG_FILE_SIZE_BYTES, maxLogFileSize.toString());
+    }
+    if (numLogFilesToKeep != null) {
+      addRocksDbConfig(tableSpecConfig, ROCKSDB_KEEP_LOG_FILE_NUM, numLogFilesToKeep.toString());
+    }
+  }
+
+  private void addRocksDbConfig(Map<String, String> map, String key, String value) {
+    map.put("rocksdb." + key, value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProvider.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProvider.java
new file mode 100644
index 0000000..e0c3355
--- /dev/null
+++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProvider.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.storage.kv.descriptors;
+
+import java.util.HashMap;
+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.storage.kv.RocksDbKeyValueStorageEngineFactory;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Table provider for tables backed by RocksDb.
+ */
+public class RocksDbTableProvider extends BaseLocalStoreBackedTableProvider {
+
+  public RocksDbTableProvider(TableSpec tableSpec) {
+    super(tableSpec);
+  }
+
+  @Override
+  public Map<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
+
+    Map<String, String> tableConfig = new HashMap<>();
+
+    // Store factory configuration
+    tableConfig.put(String.format(
+        StorageConfig.FACTORY(), tableSpec.getId()),
+        RocksDbKeyValueStorageEngineFactory.class.getName());
+
+    // Common store configuration
+    tableConfig.putAll(generateCommonStoreConfig(jobConfig, generatedConfig));
+
+    // Rest of the configuration
+    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");
+
+    logger.info("Generated configuration for table " + tableSpec.getId());
+
+    return tableConfig;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProviderFactory.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProviderFactory.java
new file mode 100644
index 0000000..74e74db
--- /dev/null
+++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/descriptors/RocksDbTableProviderFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.storage.kv.descriptors;
+
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
+import org.apache.samza.table.TableSpec;
+
+
+public class RocksDbTableProviderFactory implements TableProviderFactory {
+  @Override
+  public TableProvider getTableProvider(TableSpec tableSpec) {
+    return new RocksDbTableProvider(tableSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index cd7e85c..0000000
--- a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.storage.kv;
-
-import junit.framework.Assert;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.table.TableSpec;
-import org.junit.Test;
-
-
-public class TestRocksDbTableDescriptor {
-
-  @Test
-  public void testMinimal() {
-    new RocksDbTableDescriptor<Integer, String>("1")
-        .validate();
-  }
-
-  @Test
-  public void testSerde() {
-    TableSpec tableSpec = new RocksDbTableDescriptor("1",
-            KVSerde.of(new IntegerSerde(), new StringSerde()))
-        .getTableSpec();
-    Assert.assertNotNull(tableSpec.getSerde());
-    Assert.assertEquals(tableSpec.getSerde().getKeySerde().getClass(), IntegerSerde.class);
-    Assert.assertEquals(tableSpec.getSerde().getValueSerde().getClass(), StringSerde.class);
-  }
-
-  @Test
-  public void testTableSpec() {
-
-    TableSpec tableSpec = new RocksDbTableDescriptor<Integer, String>("1",
-            KVSerde.of(new IntegerSerde(), new StringSerde()))
-        .withBlockSize(1)
-        .withCacheSize(2)
-        .withCompactionStyle("fifo")
-        .withCompressionType("snappy")
-        .withMaxLogFileSize(3)
-        .withNumLogFilesToKeep(4)
-        .withNumWriteBuffers(5)
-        .withObjectCacheSize(6)
-        .withTtl(7)
-        .withWriteBatchSize(8)
-        .withWriteBufferSize(9)
-        .withConfig("rocksdb.abc", "xyz")
-        .getTableSpec();
-
-    Assert.assertNotNull(tableSpec.getSerde());
-    Assert.assertNotNull(tableSpec.getSerde().getKeySerde());
-    Assert.assertNotNull(tableSpec.getSerde().getValueSerde());
-    Assert.assertEquals("1", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_BLOCK_SIZE_BYTES));
-    Assert.assertEquals("2", getConfig(tableSpec, RocksDbTableDescriptor.CONTAINER_CACHE_SIZE_BYTES));
-    Assert.assertEquals("3", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_MAX_LOG_FILE_SIZE_BYTES));
-    Assert.assertEquals("4", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_KEEP_LOG_FILE_NUM));
-    Assert.assertEquals("5", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_NUM_WRITE_BUFFERS));
-    Assert.assertEquals("6", getConfig(tableSpec, RocksDbTableDescriptor.OBJECT_CACHE_SIZE));
-    Assert.assertEquals("7", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_TTL_MS));
-    Assert.assertEquals("8", getConfig(tableSpec, RocksDbTableDescriptor.WRITE_BATCH_SIZE));
-    Assert.assertEquals("9", getConfig(tableSpec, RocksDbTableDescriptor.CONTAINER_WRITE_BUFFER_SIZE_BYTES));
-    Assert.assertEquals("snappy", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPRESSION));
-    Assert.assertEquals("fifo", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPACTION_STYLE));
-    Assert.assertEquals("xyz", getConfig(tableSpec, "abc"));
-    Assert.assertEquals("false", tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_ENABLE_CHANGELOG));
-  }
-
-  @Test
-  public void testTableSpecWithChangelogEnabled() {
-
-    TableSpec tableSpec = new RocksDbTableDescriptor("1", 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) {
-    return tableSpec.getConfig().get("rocksdb." + key);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 8ce061c..0000000
--- a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java
+++ /dev/null
@@ -1,67 +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.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;
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableSpec;
-import org.junit.Test;
-
-import junit.framework.Assert;
-
-
-public class TestRocksDbTableProvider {
-  @Test
-  public void testGenerateConfig() {
-
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    tableSpecConfig.put("rocksdb.c1", "c1-value");
-    tableSpecConfig.put("rocksdb.c2", "c2-value");
-    tableSpecConfig.put("c3", "c3-value");
-    tableSpecConfig.put("c4", "c4-value");
-
-    TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()),
-        "my-table-provider-factory", tableSpecConfig);
-
-    Map<String, String> 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<String, String> 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")));
-    Assert.assertEquals(
-        RocksDbKeyValueStorageEngineFactory.class.getName(),
-        tableConfig.get(String.format(StorageConfig.FACTORY(), "t1")));
-    Assert.assertEquals("c1-value", tableConfig.get("stores.t1.c1"));
-    Assert.assertEquals("c2-value", tableConfig.get("stores.t1.c2"));
-    Assert.assertEquals("c3-value", tableConfig.get("tables.t1.c3"));
-    Assert.assertEquals("c4-value", tableConfig.get("tables.t1.c4"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableDescriptor.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableDescriptor.java
new file mode 100644
index 0000000..86efea5
--- /dev/null
+++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableDescriptor.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.storage.kv.descriptors;
+
+import junit.framework.Assert;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.table.TableSpec;
+import org.junit.Test;
+
+
+public class TestRocksDbTableDescriptor {
+
+  @Test
+  public void testMinimal() {
+    new RocksDbTableDescriptor<Integer, String>("1")
+        .validate();
+  }
+
+  @Test
+  public void testSerde() {
+    TableSpec tableSpec = new RocksDbTableDescriptor("1",
+            KVSerde.of(new IntegerSerde(), new StringSerde()))
+        .getTableSpec();
+    Assert.assertNotNull(tableSpec.getSerde());
+    Assert.assertEquals(tableSpec.getSerde().getKeySerde().getClass(), IntegerSerde.class);
+    Assert.assertEquals(tableSpec.getSerde().getValueSerde().getClass(), StringSerde.class);
+  }
+
+  @Test
+  public void testTableSpec() {
+
+    TableSpec tableSpec = new RocksDbTableDescriptor<Integer, String>("1",
+            KVSerde.of(new IntegerSerde(), new StringSerde()))
+        .withBlockSize(1)
+        .withCacheSize(2)
+        .withCompactionStyle("fifo")
+        .withCompressionType("snappy")
+        .withMaxLogFileSize(3)
+        .withNumLogFilesToKeep(4)
+        .withNumWriteBuffers(5)
+        .withObjectCacheSize(6)
+        .withTtl(7)
+        .withWriteBatchSize(8)
+        .withWriteBufferSize(9)
+        .withConfig("rocksdb.abc", "xyz")
+        .getTableSpec();
+
+    Assert.assertNotNull(tableSpec.getSerde());
+    Assert.assertNotNull(tableSpec.getSerde().getKeySerde());
+    Assert.assertNotNull(tableSpec.getSerde().getValueSerde());
+    Assert.assertEquals("1", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_BLOCK_SIZE_BYTES));
+    Assert.assertEquals("2", getConfig(tableSpec, RocksDbTableDescriptor.CONTAINER_CACHE_SIZE_BYTES));
+    Assert.assertEquals("3", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_MAX_LOG_FILE_SIZE_BYTES));
+    Assert.assertEquals("4", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_KEEP_LOG_FILE_NUM));
+    Assert.assertEquals("5", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_NUM_WRITE_BUFFERS));
+    Assert.assertEquals("6", getConfig(tableSpec, RocksDbTableDescriptor.OBJECT_CACHE_SIZE));
+    Assert.assertEquals("7", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_TTL_MS));
+    Assert.assertEquals("8", getConfig(tableSpec, RocksDbTableDescriptor.WRITE_BATCH_SIZE));
+    Assert.assertEquals("9", getConfig(tableSpec, RocksDbTableDescriptor.CONTAINER_WRITE_BUFFER_SIZE_BYTES));
+    Assert.assertEquals("snappy", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPRESSION));
+    Assert.assertEquals("fifo", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPACTION_STYLE));
+    Assert.assertEquals("xyz", getConfig(tableSpec, "abc"));
+    Assert.assertEquals("false", tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_ENABLE_CHANGELOG));
+  }
+
+  @Test
+  public void testTableSpecWithChangelogEnabled() {
+
+    TableSpec tableSpec = new RocksDbTableDescriptor("1", 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) {
+    return tableSpec.getConfig().get("rocksdb." + key);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableProvider.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableProvider.java
new file mode 100644
index 0000000..5e4601d
--- /dev/null
+++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/descriptors/TestRocksDbTableProvider.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.kv.descriptors;
+
+import java.util.HashMap;
+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;
+import org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.TableSpec;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+
+public class TestRocksDbTableProvider {
+  @Test
+  public void testGenerateConfig() {
+
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    tableSpecConfig.put("rocksdb.c1", "c1-value");
+    tableSpecConfig.put("rocksdb.c2", "c2-value");
+    tableSpecConfig.put("c3", "c3-value");
+    tableSpecConfig.put("c4", "c4-value");
+
+    TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()),
+        "my-table-provider-factory", tableSpecConfig);
+
+    Map<String, String> 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<String, String> 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")));
+    Assert.assertEquals(
+        RocksDbKeyValueStorageEngineFactory.class.getName(),
+        tableConfig.get(String.format(StorageConfig.FACTORY(), "t1")));
+    Assert.assertEquals("c1-value", tableConfig.get("stores.t1.c1"));
+    Assert.assertEquals("c2-value", tableConfig.get("stores.t1.c2"));
+    Assert.assertEquals("c3-value", tableConfig.get("tables.t1.c3"));
+    Assert.assertEquals("c4-value", tableConfig.get("tables.t1.c4"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 84e5fbe..0000000
--- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java
+++ /dev/null
@@ -1,168 +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 com.google.common.base.Preconditions;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.storage.SideInputsProcessor;
-
-
-/**
- * Table descriptor for store backed tables.
- *
- * @param <K> the type of the key in this table
- * @param <V> the type of the value in this table
- * @param <D> the type of the concrete table descriptor
- */
-abstract public class BaseLocalStoreBackedTableDescriptor<K, V, D extends BaseLocalStoreBackedTableDescriptor<K, V, D>>
-    extends BaseTableDescriptor<K, V, D> {
-
-  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<String> sideInputs;
-  protected SideInputsProcessor sideInputsProcessor;
-  protected boolean enableChangelog;
-  protected String changelogStream;
-  protected Integer changelogReplicationFactor;
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   */
-  public BaseLocalStoreBackedTableDescriptor(String tableId) {
-    super(tableId);
-  }
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   * @param serde the serde for key and value
-   */
-  public BaseLocalStoreBackedTableDescriptor(String tableId, KVSerde<K, V> serde) {
-    super(tableId, serde);
-  }
-
-  public D withSideInputs(List<String> sideInputs) {
-    this.sideInputs = sideInputs;
-    // Disable changelog
-    this.enableChangelog = false;
-    this.changelogStream = null;
-    this.changelogReplicationFactor = null;
-    return (D) this;
-  }
-
-  public D withSideInputsProcessor(SideInputsProcessor sideInputsProcessor) {
-    this.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 <code>stores.store-name.changelog</code> in Samza configuration guide
-   *
-   * @return this table descriptor instance
-   */
-  public D withChangelogEnabled() {
-    this.enableChangelog = true;
-    return (D) this;
-  }
-
-  /**
-   * Samza stores are local to a container. If the container fails, the contents of
-   * the store are lost. To prevent loss of data, you need to set this property to
-   * configure a changelog stream: Samza then ensures that writes to the store are
-   * replicated to this stream, and the store is restored from this stream after a
-   * failure. The value of this property is given in the form system-name.stream-name.
-   * The "system-name" part is optional. If it is omitted you must specify the system
-   * in <code>job.changelog.system</code> config. Any output stream can be used as
-   * changelog, but you must ensure that only one job ever writes to a given changelog
-   * stream (each instance of a job and each store needs its own changelog stream).
-   * <p>
-   * Refer to <code>stores.store-name.changelog</code> 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;
-  }
-
-  /**
-   * The property defines the number of replicas to use for the change log stream.
-   * <p>
-   * Default value is <code>stores.default.changelog.replication.factor</code>.
-   * <p>
-   * Refer to <code>stores.store-name.changelog.replication.factor</code> 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<String, String> 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));
-      }
-    }
-  }
-
-  /**
-   * Validate that this table descriptor is constructed properly
-   */
-  protected void validate() {
-    super.validate();
-    if (sideInputs != null || sideInputsProcessor != null) {
-      Preconditions.checkArgument(sideInputs != null && !sideInputs.isEmpty() && sideInputsProcessor != null,
-          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));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index e56c977..0000000
--- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java
+++ /dev/null
@@ -1,147 +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 com.google.common.base.Preconditions;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-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.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.context.Context;
-import org.apache.samza.table.ReadableTable;
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.utils.BaseTableProvider;
-import org.apache.samza.table.utils.SerdeUtils;
-
-
-/**
- * Base class for tables backed by Samza local stores. The backing stores are
- * injected during initialization of the table. Since the lifecycle
- * of the underlying stores are already managed by Samza container,
- * the table provider will not manage the lifecycle of the backing
- * stores.
- */
-abstract public class BaseLocalStoreBackedTableProvider extends BaseTableProvider {
-  public static final Pattern SYSTEM_STREAM_NAME_PATTERN = Pattern.compile("[\\d\\w-_.]+");
-
-  protected KeyValueStore kvStore;
-
-  public BaseLocalStoreBackedTableProvider(TableSpec tableSpec) {
-    super(tableSpec);
-  }
-
-  @Override
-  public void init(Context context) {
-    super.init(context);
-
-    Preconditions.checkNotNull(this.context, "Must specify context for local tables.");
-
-    kvStore = (KeyValueStore) this.context.getTaskContext().getStore(tableSpec.getId());
-
-    if (kvStore == null) {
-      throw new SamzaException(String.format(
-          "Backing store for table %s was not injected by SamzaContainer", tableSpec.getId()));
-    }
-
-    logger.info("Initialized backing store for table " + tableSpec.getId());
-  }
-
-  @Override
-  public Table getTable() {
-    if (kvStore == null) {
-      throw new SamzaException("Store not initialized for table " + tableSpec.getId());
-    }
-    ReadableTable table = new LocalStoreBackedReadWriteTable(tableSpec.getId(), kvStore);
-    table.init(this.context);
-    return table;
-  }
-
-  protected Map<String, String> generateCommonStoreConfig(Config jobConfig, Map<String, String> generatedConfig) {
-
-    Map<String, String> storeConfig = new HashMap<>();
-
-    // 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<String, String> 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);
-
-    String valueSerde = tableConfig.getValueSerde(tableSpec.getId());
-    storeConfig.put(String.format(StorageConfig.MSG_SERDE(), tableSpec.getId()), valueSerde);
-
-    List<String> sideInputs = tableSpec.getSideInputs();
-    if (sideInputs != null && !sideInputs.isEmpty()) {
-      sideInputs.forEach(si -> Preconditions.checkState(isValidSystemStreamName(si), String.format(
-          "Side input stream %s doesn't confirm to pattern %s", si, SYSTEM_STREAM_NAME_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", mergedJobConfig.getName().get(), mergedJobConfig.getJobId(),
-            tableSpec.getId());
-      }
-
-      Preconditions.checkState(isValidSystemStreamName(changelogStream), String.format(
-          "Changelog stream %s doesn't confirm to pattern %s", changelogStream, SYSTEM_STREAM_NAME_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;
-  }
-
-  @Override
-  public void close() {
-    logger.info("Shutting down table provider for table " + tableSpec.getId());
-  }
-
-  private boolean isValidSystemStreamName(String name) {
-    return StringUtils.isNotBlank(name) && SYSTEM_STREAM_NAME_PATTERN.matcher(name).matches();
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableDescriptor.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableDescriptor.java
new file mode 100644
index 0000000..7957fd3
--- /dev/null
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableDescriptor.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.storage.SideInputsProcessor;
+
+
+/**
+ * Table descriptor for store backed tables.
+ *
+ * @param <K> the type of the key in this table
+ * @param <V> the type of the value in this table
+ * @param <D> the type of the concrete table descriptor
+ */
+abstract public class BaseLocalStoreBackedTableDescriptor<K, V, D extends BaseLocalStoreBackedTableDescriptor<K, V, D>>
+    extends BaseTableDescriptor<K, V, D> {
+
+  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<String> sideInputs;
+  protected SideInputsProcessor sideInputsProcessor;
+  protected boolean enableChangelog;
+  protected String changelogStream;
+  protected Integer changelogReplicationFactor;
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   */
+  public BaseLocalStoreBackedTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   * @param serde the serde for key and value
+   */
+  public BaseLocalStoreBackedTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
+  public D withSideInputs(List<String> sideInputs) {
+    this.sideInputs = sideInputs;
+    // Disable changelog
+    this.enableChangelog = false;
+    this.changelogStream = null;
+    this.changelogReplicationFactor = null;
+    return (D) this;
+  }
+
+  public D withSideInputsProcessor(SideInputsProcessor sideInputsProcessor) {
+    this.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 <code>stores.store-name.changelog</code> in Samza configuration guide
+   *
+   * @return this table descriptor instance
+   */
+  public D withChangelogEnabled() {
+    this.enableChangelog = true;
+    return (D) this;
+  }
+
+  /**
+   * Samza stores are local to a container. If the container fails, the contents of
+   * the store are lost. To prevent loss of data, you need to set this property to
+   * configure a changelog stream: Samza then ensures that writes to the store are
+   * replicated to this stream, and the store is restored from this stream after a
+   * failure. The value of this property is given in the form system-name.stream-name.
+   * The "system-name" part is optional. If it is omitted you must specify the system
+   * in <code>job.changelog.system</code> config. Any output stream can be used as
+   * changelog, but you must ensure that only one job ever writes to a given changelog
+   * stream (each instance of a job and each store needs its own changelog stream).
+   * <p>
+   * Refer to <code>stores.store-name.changelog</code> 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;
+  }
+
+  /**
+   * The property defines the number of replicas to use for the change log stream.
+   * <p>
+   * Default value is <code>stores.default.changelog.replication.factor</code>.
+   * <p>
+   * Refer to <code>stores.store-name.changelog.replication.factor</code> 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<String, String> 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));
+      }
+    }
+  }
+
+  /**
+   * Validate that this table descriptor is constructed properly
+   */
+  protected void validate() {
+    super.validate();
+    if (sideInputs != null || sideInputsProcessor != null) {
+      Preconditions.checkArgument(sideInputs != null && !sideInputs.isEmpty() && sideInputsProcessor != null,
+          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));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableProvider.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableProvider.java
new file mode 100644
index 0000000..f5dd71b
--- /dev/null
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/descriptors/BaseLocalStoreBackedTableProvider.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.descriptors;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.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.context.Context;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.storage.kv.LocalStoreBackedReadWriteTable;
+import org.apache.samza.table.ReadableTable;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.utils.descriptors.BaseTableProvider;
+import org.apache.samza.table.utils.SerdeUtils;
+
+
+/**
+ * Base class for tables backed by Samza local stores. The backing stores are
+ * injected during initialization of the table. Since the lifecycle
+ * of the underlying stores are already managed by Samza container,
+ * the table provider will not manage the lifecycle of the backing
+ * stores.
+ */
+abstract public class BaseLocalStoreBackedTableProvider extends BaseTableProvider {
+  public static final Pattern SYSTEM_STREAM_NAME_PATTERN = Pattern.compile("[\\d\\w-_.]+");
+
+  protected KeyValueStore kvStore;
+
+  public BaseLocalStoreBackedTableProvider(TableSpec tableSpec) {
+    super(tableSpec);
+  }
+
+  @Override
+  public void init(Context context) {
+    super.init(context);
+
+    Preconditions.checkNotNull(this.context, "Must specify context for local tables.");
+
+    kvStore = (KeyValueStore) this.context.getTaskContext().getStore(tableSpec.getId());
+
+    if (kvStore == null) {
+      throw new SamzaException(String.format(
+          "Backing store for table %s was not injected by SamzaContainer", tableSpec.getId()));
+    }
+
+    logger.info("Initialized backing store for table " + tableSpec.getId());
+  }
+
+  @Override
+  public Table getTable() {
+    if (kvStore == null) {
+      throw new SamzaException("Store not initialized for table " + tableSpec.getId());
+    }
+    ReadableTable table = new LocalStoreBackedReadWriteTable(tableSpec.getId(), kvStore);
+    table.init(this.context);
+    return table;
+  }
+
+  protected Map<String, String> generateCommonStoreConfig(Config jobConfig, Map<String, String> generatedConfig) {
+
+    Map<String, String> storeConfig = new HashMap<>();
+
+    // 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<String, String> 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);
+
+    String valueSerde = tableConfig.getValueSerde(tableSpec.getId());
+    storeConfig.put(String.format(StorageConfig.MSG_SERDE(), tableSpec.getId()), valueSerde);
+
+    List<String> sideInputs = tableSpec.getSideInputs();
+    if (sideInputs != null && !sideInputs.isEmpty()) {
+      sideInputs.forEach(si -> Preconditions.checkState(isValidSystemStreamName(si), String.format(
+          "Side input stream %s doesn't confirm to pattern %s", si, SYSTEM_STREAM_NAME_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", mergedJobConfig.getName().get(), mergedJobConfig.getJobId(),
+            tableSpec.getId());
+      }
+
+      Preconditions.checkState(isValidSystemStreamName(changelogStream), String.format(
+          "Changelog stream %s doesn't confirm to pattern %s", changelogStream, SYSTEM_STREAM_NAME_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;
+  }
+
+  @Override
+  public void close() {
+    logger.info("Shutting down table provider for table " + tableSpec.getId());
+  }
+
+  private boolean isValidSystemStreamName(String name) {
+    return StringUtils.isNotBlank(name) && SYSTEM_STREAM_NAME_PATTERN.matcher(name).matches();
+  }
+}


[04/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaInputDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaInputDescriptor.java
deleted file mode 100644
index d3fe252..0000000
--- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaInputDescriptor.java
+++ /dev/null
@@ -1,108 +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.system.kafka;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for a kafka input stream.
- * <p>
- * An instance of this descriptor may be obtained from an appropriately configured {@link KafkaSystemDescriptor}.
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- */
-public class KafkaInputDescriptor<StreamMessageType>
-    extends InputDescriptor<StreamMessageType, KafkaInputDescriptor<StreamMessageType>> {
-  private static final String CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY = "systems.%s.streams.%s.consumer.auto.offset.reset";
-  private static final String CONSUMER_FETCH_MESSAGE_MAX_BYTES_CONFIG_KEY = "systems.%s.streams.%s.consumer.fetch.message.max.bytes";
-
-  private Optional<String> consumerAutoOffsetResetOptional = Optional.empty();
-  private Optional<Long> consumerFetchMessageMaxBytesOptional = Optional.empty();
-
-  KafkaInputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde, InputTransformer transformer) {
-    super(streamId, serde, systemDescriptor, transformer);
-  }
-
-  /**
-   * This setting determines what happens if a consumer attempts to read an offset for this topic that is outside of
-   * the current valid range. This could happen if the topic does not exist, or if a checkpoint is older than the
-   * maximum message history retained by the brokers. This property is not to be confused with
-   * {@link InputDescriptor#withOffsetDefault}, which determines what happens if there is no checkpoint.
-   * <p>
-   * The following are valid values for auto.offset.reset:
-   * <ul>
-   *   <li>smallest: Start consuming at the smallest (oldest) offset available on the broker
-   *   (process as much message history as available).
-   *   <li>largest: Start consuming at the largest (newest) offset available on the broker
-   *   (skip any messages published while the job was not running).
-   *   <li>anything else: Throw an exception and refuse to start up the job.
-   * </ul>
-   * <p>
-   * Note: This property may be set at a system level using {@link KafkaSystemDescriptor#withConsumerAutoOffsetReset}
-   *
-   * @param consumerAutoOffsetReset consumer auto offset reset policy for the input
-   * @return this input descriptor
-   */
-  public KafkaInputDescriptor<StreamMessageType> withConsumerAutoOffsetReset(String consumerAutoOffsetReset) {
-    this.consumerAutoOffsetResetOptional = Optional.of(StringUtils.stripToNull(consumerAutoOffsetReset));
-    return this;
-  }
-
-  /**
-   * The number of bytes of messages to attempt to fetch for each topic-partition for this topic in each fetch request.
-   * These bytes will be read into memory for each partition, so this helps control the memory used by the consumer.
-   * The fetch request size must be at least as large as the maximum message size the server allows or else it is
-   * possible for the producer to send messages larger than the consumer can fetch.
-   * <p>
-   * Note: This property may be set at a system level using {@link KafkaSystemDescriptor#withConsumerFetchMessageMaxBytes}
-   *
-   * @param fetchMessageMaxBytes number of bytes of messages to attempt to fetch for each topic-partition
-   *                             in each fetch request
-   * @return this input descriptor
-   */
-  public KafkaInputDescriptor<StreamMessageType> withConsumerFetchMessageMaxBytes(long fetchMessageMaxBytes) {
-    this.consumerFetchMessageMaxBytesOptional = Optional.of(fetchMessageMaxBytes);
-    return this;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    HashMap<String, String> configs = new HashMap<>(super.toConfig());
-    // Note: Kafka configuration needs the topic's physical name, not the stream-id.
-    // We won't have that here if user only specified it in configs, or if it got rewritten
-    // by the planner to something different than what's in this descriptor.
-    String streamName = getPhysicalName().orElse(getStreamId());
-    String systemName = getSystemName();
-
-    consumerAutoOffsetResetOptional.ifPresent(autoOffsetReset ->
-        configs.put(String.format(CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY, systemName, streamName), autoOffsetReset));
-    consumerFetchMessageMaxBytesOptional.ifPresent(fetchMessageMaxBytes ->
-        configs.put(String.format(CONSUMER_FETCH_MESSAGE_MAX_BYTES_CONFIG_KEY, systemName, streamName), Long.toString(fetchMessageMaxBytes)));
-    return configs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaOutputDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaOutputDescriptor.java
deleted file mode 100644
index 1142276..0000000
--- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaOutputDescriptor.java
+++ /dev/null
@@ -1,39 +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.system.kafka;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for a kafka output stream.
- * <p>
- * An instance of this descriptor may be obtained from an appropriately configured {@link KafkaSystemDescriptor}.
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- */
-public class KafkaOutputDescriptor<StreamMessageType>
-    extends OutputDescriptor<StreamMessageType, KafkaOutputDescriptor<StreamMessageType>> {
-  KafkaOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
-    super(streamId, serde, systemDescriptor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemDescriptor.java
deleted file mode 100644
index e88ed70..0000000
--- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemDescriptor.java
+++ /dev/null
@@ -1,245 +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.system.kafka;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.config.KafkaConfig;
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SimpleInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-
-/**
- * A descriptor for a Kafka system.
- * <p>
- * System properties configured using a descriptor override corresponding properties provided in configuration.
- */
-@SuppressWarnings("unchecked")
-public class KafkaSystemDescriptor extends SystemDescriptor<KafkaSystemDescriptor>
-    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
-  private static final String FACTORY_CLASS_NAME = KafkaSystemFactory.class.getName();
-  private static final String CONSUMER_ZK_CONNECT_CONFIG_KEY = "systems.%s.consumer.zookeeper.connect";
-  private static final String CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY = "systems.%s.consumer.auto.offset.reset";
-  private static final String CONSUMER_FETCH_THRESHOLD_CONFIG_KEY = KafkaConfig.CONSUMER_FETCH_THRESHOLD();
-  private static final String CONSUMER_FETCH_THRESHOLD_BYTES_CONFIG_KEY = KafkaConfig.CONSUMER_FETCH_THRESHOLD_BYTES();
-  private static final String CONSUMER_FETCH_MESSAGE_MAX_BYTES_KEY = "systems.%s.consumer.fetch.message.max.bytes";
-  private static final String CONSUMER_CONFIGS_CONFIG_KEY = "systems.%s.consumer.%s";
-  private static final String PRODUCER_BOOTSTRAP_SERVERS_CONFIG_KEY = "systems.%s.producer.bootstrap.servers";
-  private static final String PRODUCER_CONFIGS_CONFIG_KEY = "systems.%s.producer.%s";
-
-  private List<String> consumerZkConnect = Collections.emptyList();
-  private Optional<String> consumerAutoOffsetResetOptional = Optional.empty();
-  private Optional<Integer> consumerFetchThresholdOptional = Optional.empty();
-  private Optional<Long> consumerFetchThresholdBytesOptional = Optional.empty();
-  private Optional<Long> consumerFetchMessageMaxBytesOptional = Optional.empty();
-  private Map<String, String> consumerConfigs = Collections.emptyMap();
-  private List<String> producerBootstrapServers = Collections.emptyList();
-  private Map<String, String> producerConfigs = Collections.emptyMap();
-
-  /**
-   * Constructs a {@link KafkaSystemDescriptor} instance with no system level serde.
-   * Serdes must be provided explicitly at stream level when getting input or output descriptors.
-   *
-   * @param systemName name of this system
-   */
-  public KafkaSystemDescriptor(String systemName) {
-    super(systemName, FACTORY_CLASS_NAME, null, null);
-  }
-
-  @Override
-  public <StreamMessageType> KafkaInputDescriptor<StreamMessageType> getInputDescriptor(String streamId, Serde<StreamMessageType> serde) {
-    return new KafkaInputDescriptor<>(streamId, this, serde, null);
-  }
-
-  @Override
-  public <StreamMessageType> KafkaOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
-    return new KafkaOutputDescriptor<>(streamId, this, serde);
-  }
-
-  /**
-   * The hostname and port of one or more Zookeeper nodes where information about the Kafka cluster can be found.
-   * This is given as a list of hostname:port pairs, such as
-   * {@code ImmutableList.of("zk1.example.com:2181", "zk2.example.com:2181", "zk3.example.com:2181")}.
-   * If the cluster information is at some sub-path of the Zookeeper namespace, you need to include the path at the
-   * end of the list of hostnames, for example:
-   * {@code ImmutableList.of("zk1.example.com:2181", "zk2.example.com:2181/clusters/my-kafka")}.
-   *
-   * @param consumerZkConnect Zookeeper connection information for the system
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withConsumerZkConnect(List<String> consumerZkConnect) {
-    this.consumerZkConnect = consumerZkConnect;
-    return this;
-  }
-
-  /**
-   * This setting determines what happens if a consumer attempts to read an offset that is outside of the current
-   * valid range. This could happen if the topic does not exist, or if a checkpoint is older than the maximum message
-   * history retained by the brokers. This property is not to be confused with {@link InputDescriptor#withOffsetDefault},
-   * which determines what happens if there is no checkpoint.
-   * <p>
-   * The following are valid values for auto.offset.reset:
-   * <ul>
-   *   <li>smallest: Start consuming at the smallest (oldest) offset available on the broker
-   *   (process as much message history as available).
-   *   <li>largest: Start consuming at the largest (newest) offset available on the broker
-   *   (skip any messages published while the job was not running).
-   *   <li>anything else: Throw an exception and refuse to start up the job.
-   * </ul>
-   * <p>
-   * Note: This property may be set at a topic level using {@link KafkaInputDescriptor#withConsumerAutoOffsetReset}
-   *
-   * @param consumerAutoOffsetReset consumer auto offset reset policy for the system
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withConsumerAutoOffsetReset(String consumerAutoOffsetReset) {
-    this.consumerAutoOffsetResetOptional = Optional.of(StringUtils.stripToNull(consumerAutoOffsetReset));
-    return this;
-  }
-
-  /**
-   * When consuming streams from Kafka, a Samza container maintains an in-memory buffer for incoming messages in
-   * order to increase throughput (the stream task can continue processing buffered messages while new messages are
-   * fetched from Kafka). This parameter determines the number of messages we aim to buffer across all stream partitions
-   * consumed by a container. For example, if a container consumes 50 partitions, it will try to buffer 1000
-   * messages per partition by default. When the number of buffered messages falls below that threshold, Samza
-   * fetches more messages from the Kafka broker to replenish the buffer. Increasing this parameter can increase
-   * a job's processing throughput, but also increases the amount of memory used.
-   *
-   * @param fetchThreshold number of incoming messages to buffer in-memory
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withSamzaFetchThreshold(int fetchThreshold) {
-    this.consumerFetchThresholdOptional = Optional.of(fetchThreshold);
-    return this;
-  }
-
-  /**
-   * When consuming streams from Kafka, a Samza container maintains an in-memory buffer for incoming messages in
-   * order to increase throughput (the stream task can continue processing buffered messages while new messages are
-   * fetched from Kafka). This parameter determines the total size of messages we aim to buffer across all stream
-   * partitions consumed by a container based on bytes. Defines how many bytes to use for the buffered prefetch
-   * messages for job as a whole. The bytes for a single system/stream/partition are computed based on this.
-   * This fetches the entire messages, hence this bytes limit is a soft one, and the actual usage can be
-   * the bytes limit + size of max message in the partition for a given stream. If the value of this property
-   * is &gt; 0 then this takes precedence over systems.system-name.samza.fetch.threshold.
-   * <p>
-   * For example, if fetchThresholdBytes is set to 100000 bytes, and there are 50 SystemStreamPartitions registered,
-   * then the per-partition threshold is (100000 / 2) / 50 = 1000 bytes. As this is a soft limit, the actual usage
-   * can be 1000 bytes + size of max message. As soon as a SystemStreamPartition's buffered messages bytes drops
-   * below 1000, a fetch request will be executed to get more data for it. Increasing this parameter will decrease
-   * the latency between when a queue is drained of messages and when new messages are enqueued, but also leads
-   * to an increase in memory usage since more messages will be held in memory. The default value is -1,
-   * which means this is not used.
-   *
-   * @param fetchThresholdBytes number of bytes for incoming messages to buffer in-memory
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withSamzaFetchThresholdBytes(long fetchThresholdBytes) {
-    this.consumerFetchThresholdBytesOptional = Optional.of(fetchThresholdBytes);
-    return this;
-  }
-
-  /**
-   * The number of bytes of messages to attempt to fetch for each topic-partition in each fetch request.
-   * These bytes will be read into memory for each partition, so this helps control the memory used by the consumer.
-   * The fetch request size must be at least as large as the maximum message size the server allows or else it is
-   * possible for the producer to send messages larger than the consumer can fetch.
-   * <p>
-   * Note: This property may be set at a topic level using {@link KafkaInputDescriptor#withConsumerFetchMessageMaxBytes}
-   *
-   * @param fetchMessageMaxBytes number of bytes of messages to attempt to fetch for each topic-partition
-   *                             in each fetch request
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withConsumerFetchMessageMaxBytes(long fetchMessageMaxBytes) {
-    this.consumerFetchMessageMaxBytesOptional = Optional.of(fetchMessageMaxBytes);
-    return this;
-  }
-
-  /**
-   * Any Kafka consumer configuration can be included here. For example, to change the socket timeout,
-   * you can set socket.timeout.ms. (There is no need to configure group.id or client.id, as they are automatically
-   * configured by Samza. Also, there is no need to set auto.commit.enable because Samza has its own
-   * checkpointing mechanism.)
-   *
-   * @param consumerConfigs additional consumer configuration
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withConsumerConfigs(Map<String, String> consumerConfigs) {
-    this.consumerConfigs = consumerConfigs;
-    return this;
-  }
-
-  /**
-   * A list of network endpoints where the Kafka brokers are running. This is given as a list of hostname:port pairs,
-   * for example {@code ImmutableList.of("kafka1.example.com:9092", "kafka2.example.com:9092", "kafka3.example.com:9092")}.
-   * It's not necessary to list every single Kafka node in the cluster: Samza uses this property in order to discover
-   * which topics and partitions are hosted on which broker. This property is needed even if you are only consuming
-   * from Kafka, and not writing to it, because Samza uses it to discover metadata about streams being consumed.
-   *
-   * @param producerBootstrapServers network endpoints where the kafka brokers are running
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withProducerBootstrapServers(List<String> producerBootstrapServers) {
-    this.producerBootstrapServers = producerBootstrapServers;
-    return this;
-  }
-
-  /**
-   * Any Kafka producer configuration can be included here. For example, to change the request timeout,
-   * you can set timeout.ms. (There is no need to configure client.id as it is automatically configured by Samza.)
-   *
-   * @param producerConfigs additional producer configuration
-   * @return this system descriptor
-   */
-  public KafkaSystemDescriptor withProducerConfigs(Map<String, String> producerConfigs) {
-    this.producerConfigs = producerConfigs;
-    return this;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    Map<String, String> configs = new HashMap<>(super.toConfig());
-    if(!consumerZkConnect.isEmpty()) {
-      configs.put(String.format(CONSUMER_ZK_CONNECT_CONFIG_KEY, getSystemName()), String.join(",", consumerZkConnect));
-    }
-    consumerAutoOffsetResetOptional.ifPresent(consumerAutoOffsetReset ->
-        configs.put(String.format(CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY, getSystemName()), consumerAutoOffsetReset));
-    consumerFetchThresholdOptional.ifPresent(consumerFetchThreshold ->
-        configs.put(String.format(CONSUMER_FETCH_THRESHOLD_CONFIG_KEY, getSystemName()), Integer.toString(consumerFetchThreshold)));
-    consumerFetchThresholdBytesOptional.ifPresent(consumerFetchThresholdBytes ->
-        configs.put(String.format(CONSUMER_FETCH_THRESHOLD_BYTES_CONFIG_KEY, getSystemName()), Long.toString(consumerFetchThresholdBytes)));
-    consumerFetchMessageMaxBytesOptional.ifPresent(consumerFetchMessageMaxBytes ->
-        configs.put(String.format(CONSUMER_FETCH_MESSAGE_MAX_BYTES_KEY, getSystemName()), Long.toString(consumerFetchMessageMaxBytes)));
-    consumerConfigs.forEach((key, value) -> configs.put(String.format(CONSUMER_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
-    if (!producerBootstrapServers.isEmpty()) {
-      configs.put(String.format(PRODUCER_BOOTSTRAP_SERVERS_CONFIG_KEY, getSystemName()), String.join(",", producerBootstrapServers));
-    }
-    producerConfigs.forEach((key, value) -> configs.put(String.format(PRODUCER_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
-    return configs;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java
new file mode 100644
index 0000000..1896fd3
--- /dev/null
+++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.kafka.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.serializers.Serde;
+
+/**
+ * A descriptor for a kafka input stream.
+ * <p>
+ * An instance of this descriptor may be obtained from an appropriately configured {@link KafkaSystemDescriptor}.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ */
+public class KafkaInputDescriptor<StreamMessageType>
+    extends InputDescriptor<StreamMessageType, KafkaInputDescriptor<StreamMessageType>> {
+  private static final String CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY = "systems.%s.streams.%s.consumer.auto.offset.reset";
+  private static final String CONSUMER_FETCH_MESSAGE_MAX_BYTES_CONFIG_KEY = "systems.%s.streams.%s.consumer.fetch.message.max.bytes";
+
+  private Optional<String> consumerAutoOffsetResetOptional = Optional.empty();
+  private Optional<Long> consumerFetchMessageMaxBytesOptional = Optional.empty();
+
+  KafkaInputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde, InputTransformer transformer) {
+    super(streamId, serde, systemDescriptor, transformer);
+  }
+
+  /**
+   * This setting determines what happens if a consumer attempts to read an offset for this topic that is outside of
+   * the current valid range. This could happen if the topic does not exist, or if a checkpoint is older than the
+   * maximum message history retained by the brokers. This property is not to be confused with
+   * {@link InputDescriptor#withOffsetDefault}, which determines what happens if there is no checkpoint.
+   * <p>
+   * The following are valid values for auto.offset.reset:
+   * <ul>
+   *   <li>smallest: Start consuming at the smallest (oldest) offset available on the broker
+   *   (process as much message history as available).
+   *   <li>largest: Start consuming at the largest (newest) offset available on the broker
+   *   (skip any messages published while the job was not running).
+   *   <li>anything else: Throw an exception and refuse to start up the job.
+   * </ul>
+   * <p>
+   * Note: This property may be set at a system level using {@link KafkaSystemDescriptor#withConsumerAutoOffsetReset}
+   *
+   * @param consumerAutoOffsetReset consumer auto offset reset policy for the input
+   * @return this input descriptor
+   */
+  public KafkaInputDescriptor<StreamMessageType> withConsumerAutoOffsetReset(String consumerAutoOffsetReset) {
+    this.consumerAutoOffsetResetOptional = Optional.of(StringUtils.stripToNull(consumerAutoOffsetReset));
+    return this;
+  }
+
+  /**
+   * The number of bytes of messages to attempt to fetch for each topic-partition for this topic in each fetch request.
+   * These bytes will be read into memory for each partition, so this helps control the memory used by the consumer.
+   * The fetch request size must be at least as large as the maximum message size the server allows or else it is
+   * possible for the producer to send messages larger than the consumer can fetch.
+   * <p>
+   * Note: This property may be set at a system level using {@link KafkaSystemDescriptor#withConsumerFetchMessageMaxBytes}
+   *
+   * @param fetchMessageMaxBytes number of bytes of messages to attempt to fetch for each topic-partition
+   *                             in each fetch request
+   * @return this input descriptor
+   */
+  public KafkaInputDescriptor<StreamMessageType> withConsumerFetchMessageMaxBytes(long fetchMessageMaxBytes) {
+    this.consumerFetchMessageMaxBytesOptional = Optional.of(fetchMessageMaxBytes);
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    HashMap<String, String> configs = new HashMap<>(super.toConfig());
+    // Note: Kafka configuration needs the topic's physical name, not the stream-id.
+    // We won't have that here if user only specified it in configs, or if it got rewritten
+    // by the planner to something different than what's in this descriptor.
+    String streamName = getPhysicalName().orElse(getStreamId());
+    String systemName = getSystemName();
+
+    consumerAutoOffsetResetOptional.ifPresent(autoOffsetReset ->
+        configs.put(String.format(CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY, systemName, streamName), autoOffsetReset));
+    consumerFetchMessageMaxBytesOptional.ifPresent(fetchMessageMaxBytes ->
+        configs.put(String.format(CONSUMER_FETCH_MESSAGE_MAX_BYTES_CONFIG_KEY, systemName, streamName), Long.toString(fetchMessageMaxBytes)));
+    return configs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java
new file mode 100644
index 0000000..0ec5ce7
--- /dev/null
+++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.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.system.kafka.descriptors;
+
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+
+/**
+ * A descriptor for a kafka output stream.
+ * <p>
+ * An instance of this descriptor may be obtained from an appropriately configured {@link KafkaSystemDescriptor}.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ */
+public class KafkaOutputDescriptor<StreamMessageType>
+    extends OutputDescriptor<StreamMessageType, KafkaOutputDescriptor<StreamMessageType>> {
+  KafkaOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
+    super(streamId, serde, systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java
new file mode 100644
index 0000000..0c6eaeb
--- /dev/null
+++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.kafka.descriptors;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.config.KafkaConfig;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptorProvider;
+import org.apache.samza.system.descriptors.SimpleInputDescriptorProvider;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.kafka.KafkaSystemFactory;
+
+
+/**
+ * A descriptor for a Kafka system.
+ * <p>
+ * System properties configured using a descriptor override corresponding properties provided in configuration.
+ */
+@SuppressWarnings("unchecked")
+public class KafkaSystemDescriptor extends SystemDescriptor<KafkaSystemDescriptor>
+    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
+  private static final String FACTORY_CLASS_NAME = KafkaSystemFactory.class.getName();
+  private static final String CONSUMER_ZK_CONNECT_CONFIG_KEY = "systems.%s.consumer.zookeeper.connect";
+  private static final String CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY = "systems.%s.consumer.auto.offset.reset";
+  private static final String CONSUMER_FETCH_THRESHOLD_CONFIG_KEY = KafkaConfig.CONSUMER_FETCH_THRESHOLD();
+  private static final String CONSUMER_FETCH_THRESHOLD_BYTES_CONFIG_KEY = KafkaConfig.CONSUMER_FETCH_THRESHOLD_BYTES();
+  private static final String CONSUMER_FETCH_MESSAGE_MAX_BYTES_KEY = "systems.%s.consumer.fetch.message.max.bytes";
+  private static final String CONSUMER_CONFIGS_CONFIG_KEY = "systems.%s.consumer.%s";
+  private static final String PRODUCER_BOOTSTRAP_SERVERS_CONFIG_KEY = "systems.%s.producer.bootstrap.servers";
+  private static final String PRODUCER_CONFIGS_CONFIG_KEY = "systems.%s.producer.%s";
+
+  private List<String> consumerZkConnect = Collections.emptyList();
+  private Optional<String> consumerAutoOffsetResetOptional = Optional.empty();
+  private Optional<Integer> consumerFetchThresholdOptional = Optional.empty();
+  private Optional<Long> consumerFetchThresholdBytesOptional = Optional.empty();
+  private Optional<Long> consumerFetchMessageMaxBytesOptional = Optional.empty();
+  private Map<String, String> consumerConfigs = Collections.emptyMap();
+  private List<String> producerBootstrapServers = Collections.emptyList();
+  private Map<String, String> producerConfigs = Collections.emptyMap();
+
+  /**
+   * Constructs a {@link KafkaSystemDescriptor} instance with no system level serde.
+   * Serdes must be provided explicitly at stream level when getting input or output descriptors.
+   *
+   * @param systemName name of this system
+   */
+  public KafkaSystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME, null, null);
+  }
+
+  @Override
+  public <StreamMessageType> KafkaInputDescriptor<StreamMessageType> getInputDescriptor(String streamId, Serde<StreamMessageType> serde) {
+    return new KafkaInputDescriptor<>(streamId, this, serde, null);
+  }
+
+  @Override
+  public <StreamMessageType> KafkaOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
+    return new KafkaOutputDescriptor<>(streamId, this, serde);
+  }
+
+  /**
+   * The hostname and port of one or more Zookeeper nodes where information about the Kafka cluster can be found.
+   * This is given as a list of hostname:port pairs, such as
+   * {@code ImmutableList.of("zk1.example.com:2181", "zk2.example.com:2181", "zk3.example.com:2181")}.
+   * If the cluster information is at some sub-path of the Zookeeper namespace, you need to include the path at the
+   * end of the list of hostnames, for example:
+   * {@code ImmutableList.of("zk1.example.com:2181", "zk2.example.com:2181/clusters/my-kafka")}.
+   *
+   * @param consumerZkConnect Zookeeper connection information for the system
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withConsumerZkConnect(List<String> consumerZkConnect) {
+    this.consumerZkConnect = consumerZkConnect;
+    return this;
+  }
+
+  /**
+   * This setting determines what happens if a consumer attempts to read an offset that is outside of the current
+   * valid range. This could happen if the topic does not exist, or if a checkpoint is older than the maximum message
+   * history retained by the brokers. This property is not to be confused with {@link InputDescriptor#withOffsetDefault},
+   * which determines what happens if there is no checkpoint.
+   * <p>
+   * The following are valid values for auto.offset.reset:
+   * <ul>
+   *   <li>smallest: Start consuming at the smallest (oldest) offset available on the broker
+   *   (process as much message history as available).
+   *   <li>largest: Start consuming at the largest (newest) offset available on the broker
+   *   (skip any messages published while the job was not running).
+   *   <li>anything else: Throw an exception and refuse to start up the job.
+   * </ul>
+   * <p>
+   * Note: This property may be set at a topic level using {@link KafkaInputDescriptor#withConsumerAutoOffsetReset}
+   *
+   * @param consumerAutoOffsetReset consumer auto offset reset policy for the system
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withConsumerAutoOffsetReset(String consumerAutoOffsetReset) {
+    this.consumerAutoOffsetResetOptional = Optional.of(StringUtils.stripToNull(consumerAutoOffsetReset));
+    return this;
+  }
+
+  /**
+   * When consuming streams from Kafka, a Samza container maintains an in-memory buffer for incoming messages in
+   * order to increase throughput (the stream task can continue processing buffered messages while new messages are
+   * fetched from Kafka). This parameter determines the number of messages we aim to buffer across all stream partitions
+   * consumed by a container. For example, if a container consumes 50 partitions, it will try to buffer 1000
+   * messages per partition by default. When the number of buffered messages falls below that threshold, Samza
+   * fetches more messages from the Kafka broker to replenish the buffer. Increasing this parameter can increase
+   * a job's processing throughput, but also increases the amount of memory used.
+   *
+   * @param fetchThreshold number of incoming messages to buffer in-memory
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withSamzaFetchThreshold(int fetchThreshold) {
+    this.consumerFetchThresholdOptional = Optional.of(fetchThreshold);
+    return this;
+  }
+
+  /**
+   * When consuming streams from Kafka, a Samza container maintains an in-memory buffer for incoming messages in
+   * order to increase throughput (the stream task can continue processing buffered messages while new messages are
+   * fetched from Kafka). This parameter determines the total size of messages we aim to buffer across all stream
+   * partitions consumed by a container based on bytes. Defines how many bytes to use for the buffered prefetch
+   * messages for job as a whole. The bytes for a single system/stream/partition are computed based on this.
+   * This fetches the entire messages, hence this bytes limit is a soft one, and the actual usage can be
+   * the bytes limit + size of max message in the partition for a given stream. If the value of this property
+   * is &gt; 0 then this takes precedence over systems.system-name.samza.fetch.threshold.
+   * <p>
+   * For example, if fetchThresholdBytes is set to 100000 bytes, and there are 50 SystemStreamPartitions registered,
+   * then the per-partition threshold is (100000 / 2) / 50 = 1000 bytes. As this is a soft limit, the actual usage
+   * can be 1000 bytes + size of max message. As soon as a SystemStreamPartition's buffered messages bytes drops
+   * below 1000, a fetch request will be executed to get more data for it. Increasing this parameter will decrease
+   * the latency between when a queue is drained of messages and when new messages are enqueued, but also leads
+   * to an increase in memory usage since more messages will be held in memory. The default value is -1,
+   * which means this is not used.
+   *
+   * @param fetchThresholdBytes number of bytes for incoming messages to buffer in-memory
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withSamzaFetchThresholdBytes(long fetchThresholdBytes) {
+    this.consumerFetchThresholdBytesOptional = Optional.of(fetchThresholdBytes);
+    return this;
+  }
+
+  /**
+   * The number of bytes of messages to attempt to fetch for each topic-partition in each fetch request.
+   * These bytes will be read into memory for each partition, so this helps control the memory used by the consumer.
+   * The fetch request size must be at least as large as the maximum message size the server allows or else it is
+   * possible for the producer to send messages larger than the consumer can fetch.
+   * <p>
+   * Note: This property may be set at a topic level using {@link KafkaInputDescriptor#withConsumerFetchMessageMaxBytes}
+   *
+   * @param fetchMessageMaxBytes number of bytes of messages to attempt to fetch for each topic-partition
+   *                             in each fetch request
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withConsumerFetchMessageMaxBytes(long fetchMessageMaxBytes) {
+    this.consumerFetchMessageMaxBytesOptional = Optional.of(fetchMessageMaxBytes);
+    return this;
+  }
+
+  /**
+   * Any Kafka consumer configuration can be included here. For example, to change the socket timeout,
+   * you can set socket.timeout.ms. (There is no need to configure group.id or client.id, as they are automatically
+   * configured by Samza. Also, there is no need to set auto.commit.enable because Samza has its own
+   * checkpointing mechanism.)
+   *
+   * @param consumerConfigs additional consumer configuration
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withConsumerConfigs(Map<String, String> consumerConfigs) {
+    this.consumerConfigs = consumerConfigs;
+    return this;
+  }
+
+  /**
+   * A list of network endpoints where the Kafka brokers are running. This is given as a list of hostname:port pairs,
+   * for example {@code ImmutableList.of("kafka1.example.com:9092", "kafka2.example.com:9092", "kafka3.example.com:9092")}.
+   * It's not necessary to list every single Kafka node in the cluster: Samza uses this property in order to discover
+   * which topics and partitions are hosted on which broker. This property is needed even if you are only consuming
+   * from Kafka, and not writing to it, because Samza uses it to discover metadata about streams being consumed.
+   *
+   * @param producerBootstrapServers network endpoints where the kafka brokers are running
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withProducerBootstrapServers(List<String> producerBootstrapServers) {
+    this.producerBootstrapServers = producerBootstrapServers;
+    return this;
+  }
+
+  /**
+   * Any Kafka producer configuration can be included here. For example, to change the request timeout,
+   * you can set timeout.ms. (There is no need to configure client.id as it is automatically configured by Samza.)
+   *
+   * @param producerConfigs additional producer configuration
+   * @return this system descriptor
+   */
+  public KafkaSystemDescriptor withProducerConfigs(Map<String, String> producerConfigs) {
+    this.producerConfigs = producerConfigs;
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    Map<String, String> configs = new HashMap<>(super.toConfig());
+    if(!consumerZkConnect.isEmpty()) {
+      configs.put(String.format(CONSUMER_ZK_CONNECT_CONFIG_KEY, getSystemName()), String.join(",", consumerZkConnect));
+    }
+    consumerAutoOffsetResetOptional.ifPresent(consumerAutoOffsetReset ->
+        configs.put(String.format(CONSUMER_AUTO_OFFSET_RESET_CONFIG_KEY, getSystemName()), consumerAutoOffsetReset));
+    consumerFetchThresholdOptional.ifPresent(consumerFetchThreshold ->
+        configs.put(String.format(CONSUMER_FETCH_THRESHOLD_CONFIG_KEY, getSystemName()), Integer.toString(consumerFetchThreshold)));
+    consumerFetchThresholdBytesOptional.ifPresent(consumerFetchThresholdBytes ->
+        configs.put(String.format(CONSUMER_FETCH_THRESHOLD_BYTES_CONFIG_KEY, getSystemName()), Long.toString(consumerFetchThresholdBytes)));
+    consumerFetchMessageMaxBytesOptional.ifPresent(consumerFetchMessageMaxBytes ->
+        configs.put(String.format(CONSUMER_FETCH_MESSAGE_MAX_BYTES_KEY, getSystemName()), Long.toString(consumerFetchMessageMaxBytes)));
+    consumerConfigs.forEach((key, value) -> configs.put(String.format(CONSUMER_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
+    if (!producerBootstrapServers.isEmpty()) {
+      configs.put(String.format(PRODUCER_BOOTSTRAP_SERVERS_CONFIG_KEY, getSystemName()), String.join(",", producerBootstrapServers));
+    }
+    producerConfigs.forEach((key, value) -> configs.put(String.format(PRODUCER_CONFIGS_CONFIG_KEY, getSystemName(), key), value));
+    return configs;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaInputDescriptor.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaInputDescriptor.java
deleted file mode 100644
index 689f9c9..0000000
--- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaInputDescriptor.java
+++ /dev/null
@@ -1,66 +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.system.kafka;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Map;
-import org.apache.samza.SamzaException;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.descriptors.GenericSystemDescriptor;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestKafkaInputDescriptor {
-  @Test
-  public void testISDConfigsWithOverrides() {
-    KafkaSystemDescriptor sd = new KafkaSystemDescriptor("kafka");
-
-    KafkaInputDescriptor<KV<String, Integer>> isd =
-        sd.getInputDescriptor("input-stream", KVSerde.of(new StringSerde(), new IntegerSerde()))
-            .withConsumerAutoOffsetReset("largest")
-            .withConsumerFetchMessageMaxBytes(1024 * 1024);
-
-    Map<String, String> generatedConfigs = isd.toConfig();;
-    assertEquals("kafka", generatedConfigs.get("streams.input-stream.samza.system"));
-    assertEquals("largest", generatedConfigs.get("systems.kafka.streams.input-stream.consumer.auto.offset.reset"));
-    assertEquals("1048576", generatedConfigs.get("systems.kafka.streams.input-stream.consumer.fetch.message.max.bytes"));
-  }
-
-  @Test
-  public void testISDConfigsWithDefaults() {
-    KafkaSystemDescriptor sd = new KafkaSystemDescriptor("kafka")
-        .withConsumerZkConnect(ImmutableList.of("localhost:123"))
-        .withProducerBootstrapServers(ImmutableList.of("localhost:567", "localhost:890"));
-
-    KafkaInputDescriptor<KV<String, Integer>> isd =
-        sd.getInputDescriptor("input-stream", KVSerde.of(new StringSerde(), new IntegerSerde()));
-
-    Map<String, String> generatedConfigs = isd.toConfig();
-    assertEquals("kafka", generatedConfigs.get("streams.input-stream.samza.system"));
-    assertEquals(1, generatedConfigs.size()); // verify that there are no other configs
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminWithMock.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminWithMock.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminWithMock.java
index 6a03198..3e75468 100644
--- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminWithMock.java
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminWithMock.java
@@ -52,7 +52,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import static org.apache.samza.system.kafka.KafkaSystemDescriptor.*;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemDescriptor.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemDescriptor.java
deleted file mode 100644
index e1ddac0..0000000
--- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemDescriptor.java
+++ /dev/null
@@ -1,69 +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.system.kafka;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Map;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestKafkaSystemDescriptor {
-  @Test
-  public void testSDConfigsWithOverrides() {
-    KafkaSystemDescriptor sd =
-        new KafkaSystemDescriptor("kafka")
-            .withConsumerZkConnect(ImmutableList.of("localhost:1234"))
-            .withProducerBootstrapServers(ImmutableList.of("localhost:567", "localhost:890"))
-            .withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST)
-            .withConsumerAutoOffsetReset("smallest")
-            .withConsumerFetchMessageMaxBytes(1024*1024)
-            .withSamzaFetchThreshold(10000)
-            .withSamzaFetchThresholdBytes(1024 * 1024)
-            .withConsumerConfigs(ImmutableMap.of("custom-consumer-config-key", "custom-consumer-config-value"))
-            .withProducerConfigs(ImmutableMap.of("custom-producer-config-key", "custom-producer-config-value"))
-            .withDefaultStreamConfigs(ImmutableMap.of("custom-stream-config-key", "custom-stream-config-value"));
-
-    Map<String, String> generatedConfigs = sd.toConfig();
-    assertEquals("org.apache.samza.system.kafka.KafkaSystemFactory", generatedConfigs.get("systems.kafka.samza.factory"));
-    assertEquals("localhost:1234", generatedConfigs.get("systems.kafka.consumer.zookeeper.connect"));
-    assertEquals("localhost:567,localhost:890", generatedConfigs.get("systems.kafka.producer.bootstrap.servers"));
-    assertEquals("smallest", generatedConfigs.get("systems.kafka.consumer.auto.offset.reset"));
-    assertEquals("1048576", generatedConfigs.get("systems.kafka.consumer.fetch.message.max.bytes"));
-    assertEquals("10000", generatedConfigs.get("systems.kafka.samza.fetch.threshold"));
-    assertEquals("1048576", generatedConfigs.get("systems.kafka.samza.fetch.threshold.bytes"));
-    assertEquals("custom-consumer-config-value", generatedConfigs.get("systems.kafka.consumer.custom-consumer-config-key"));
-    assertEquals("custom-producer-config-value", generatedConfigs.get("systems.kafka.producer.custom-producer-config-key"));
-    assertEquals("custom-stream-config-value", generatedConfigs.get("systems.kafka.default.stream.custom-stream-config-key"));
-    assertEquals("oldest", generatedConfigs.get("systems.kafka.default.stream.samza.offset.default"));
-    assertEquals(11, generatedConfigs.size());
-  }
-
-  @Test
-  public void testSDConfigsWithoutOverrides() {
-    KafkaSystemDescriptor sd = new KafkaSystemDescriptor("kafka");
-
-    Map<String, String> generatedConfigs = sd.toConfig();
-    assertEquals("org.apache.samza.system.kafka.KafkaSystemFactory", generatedConfigs.get("systems.kafka.samza.factory"));
-    assertEquals(1, generatedConfigs.size()); // verify that there are no other configs
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaInputDescriptor.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaInputDescriptor.java
new file mode 100644
index 0000000..5bce72d
--- /dev/null
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaInputDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.kafka.descriptors;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.Map;
+import org.apache.samza.operators.KV;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestKafkaInputDescriptor {
+  @Test
+  public void testISDConfigsWithOverrides() {
+    KafkaSystemDescriptor sd = new KafkaSystemDescriptor("kafka");
+
+    KafkaInputDescriptor<KV<String, Integer>> isd =
+        sd.getInputDescriptor("input-stream", KVSerde.of(new StringSerde(), new IntegerSerde()))
+            .withConsumerAutoOffsetReset("largest")
+            .withConsumerFetchMessageMaxBytes(1024 * 1024);
+
+    Map<String, String> generatedConfigs = isd.toConfig();;
+    assertEquals("kafka", generatedConfigs.get("streams.input-stream.samza.system"));
+    assertEquals("largest", generatedConfigs.get("systems.kafka.streams.input-stream.consumer.auto.offset.reset"));
+    assertEquals("1048576", generatedConfigs.get("systems.kafka.streams.input-stream.consumer.fetch.message.max.bytes"));
+  }
+
+  @Test
+  public void testISDConfigsWithDefaults() {
+    KafkaSystemDescriptor sd = new KafkaSystemDescriptor("kafka")
+        .withConsumerZkConnect(ImmutableList.of("localhost:123"))
+        .withProducerBootstrapServers(ImmutableList.of("localhost:567", "localhost:890"));
+
+    KafkaInputDescriptor<KV<String, Integer>> isd =
+        sd.getInputDescriptor("input-stream", KVSerde.of(new StringSerde(), new IntegerSerde()));
+
+    Map<String, String> generatedConfigs = isd.toConfig();
+    assertEquals("kafka", generatedConfigs.get("streams.input-stream.samza.system"));
+    assertEquals(1, generatedConfigs.size()); // verify that there are no other configs
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaSystemDescriptor.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaSystemDescriptor.java
new file mode 100644
index 0000000..31469f8
--- /dev/null
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/descriptors/TestKafkaSystemDescriptor.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.kafka.descriptors;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestKafkaSystemDescriptor {
+  @Test
+  public void testSDConfigsWithOverrides() {
+    KafkaSystemDescriptor sd =
+        new KafkaSystemDescriptor("kafka")
+            .withConsumerZkConnect(ImmutableList.of("localhost:1234"))
+            .withProducerBootstrapServers(ImmutableList.of("localhost:567", "localhost:890"))
+            .withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST)
+            .withConsumerAutoOffsetReset("smallest")
+            .withConsumerFetchMessageMaxBytes(1024*1024)
+            .withSamzaFetchThreshold(10000)
+            .withSamzaFetchThresholdBytes(1024 * 1024)
+            .withConsumerConfigs(ImmutableMap.of("custom-consumer-config-key", "custom-consumer-config-value"))
+            .withProducerConfigs(ImmutableMap.of("custom-producer-config-key", "custom-producer-config-value"))
+            .withDefaultStreamConfigs(ImmutableMap.of("custom-stream-config-key", "custom-stream-config-value"));
+
+    Map<String, String> generatedConfigs = sd.toConfig();
+    assertEquals("org.apache.samza.system.kafka.KafkaSystemFactory", generatedConfigs.get("systems.kafka.samza.factory"));
+    assertEquals("localhost:1234", generatedConfigs.get("systems.kafka.consumer.zookeeper.connect"));
+    assertEquals("localhost:567,localhost:890", generatedConfigs.get("systems.kafka.producer.bootstrap.servers"));
+    assertEquals("smallest", generatedConfigs.get("systems.kafka.consumer.auto.offset.reset"));
+    assertEquals("1048576", generatedConfigs.get("systems.kafka.consumer.fetch.message.max.bytes"));
+    assertEquals("10000", generatedConfigs.get("systems.kafka.samza.fetch.threshold"));
+    assertEquals("1048576", generatedConfigs.get("systems.kafka.samza.fetch.threshold.bytes"));
+    assertEquals("custom-consumer-config-value", generatedConfigs.get("systems.kafka.consumer.custom-consumer-config-key"));
+    assertEquals("custom-producer-config-value", generatedConfigs.get("systems.kafka.producer.custom-producer-config-key"));
+    assertEquals("custom-stream-config-value", generatedConfigs.get("systems.kafka.default.stream.custom-stream-config-key"));
+    assertEquals("oldest", generatedConfigs.get("systems.kafka.default.stream.samza.offset.default"));
+    assertEquals(11, generatedConfigs.size());
+  }
+
+  @Test
+  public void testSDConfigsWithoutOverrides() {
+    KafkaSystemDescriptor sd = new KafkaSystemDescriptor("kafka");
+
+    Map<String, String> generatedConfigs = sd.toConfig();
+    assertEquals("org.apache.samza.system.kafka.KafkaSystemFactory", generatedConfigs.get("systems.kafka.samza.factory"));
+    assertEquals(1, generatedConfigs.size()); // verify that there are no other configs
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index d052c99..0000000
--- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java
+++ /dev/null
@@ -1,74 +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.inmemory;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.storage.kv.BaseLocalStoreBackedTableDescriptor;
-import org.apache.samza.table.TableSpec;
-
-
-/**
- * Table descriptor for in-memory tables
- *
- * @param <K> the type of the key
- * @param <V> the type of the value
- */
-public class InMemoryTableDescriptor<K, V> extends BaseLocalStoreBackedTableDescriptor<K, V, InMemoryTableDescriptor<K, V>> {
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   */
-  public InMemoryTableDescriptor(String tableId) {
-    super(tableId);
-  }
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   * @param serde the serde for key and value
-   */
-  public InMemoryTableDescriptor(String tableId, KVSerde<K, V> serde) {
-    super(tableId, serde);
-  }
-
-  @Override
-  protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
-    super.generateTableSpecConfig(tableSpecConfig);
-  }
-
-  @Override
-  public TableSpec getTableSpec() {
-
-    validate();
-
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    generateTableSpecConfig(tableSpecConfig);
-
-    return new TableSpec(tableId, serde, InMemoryTableProviderFactory.class.getName(), tableSpecConfig,
-        sideInputs, sideInputsProcessor);
-  }
-
-  private void addInMemoryConfig(Map<String, String> map, String key, String value) {
-    map.put("inmemory." + key, value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 46406e5..0000000
--- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.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.storage.kv.inmemory;
-
-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;
-import org.apache.samza.table.TableSpec;
-
-
-/**
- * Table provider of an in-memory table
- */
-public class InMemoryTableProvider extends BaseLocalStoreBackedTableProvider {
-
-  public InMemoryTableProvider(TableSpec tableSpec) {
-    super(tableSpec);
-  }
-
-  @Override
-  public Map<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
-
-    Map<String, String> tableConfig = new HashMap<>();
-
-    // Store factory configuration
-    tableConfig.put(String.format(
-        StorageConfig.FACTORY(), tableSpec.getId()),
-        InMemoryKeyValueStorageEngineFactory.class.getName());
-
-    // Common store configuration
-    tableConfig.putAll(generateCommonStoreConfig(jobConfig, generatedConfig));
-
-    // Rest of the configuration
-    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());
-
-    return tableConfig;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java
deleted file mode 100644
index f05982a..0000000
--- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java
+++ /dev/null
@@ -1,33 +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.inmemory;
-
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableProviderFactory;
-import org.apache.samza.table.TableSpec;
-
-/**
- * Factory class for an in-memory table provider
- */
-public class InMemoryTableProviderFactory implements TableProviderFactory {
-  @Override
-  public TableProvider getTableProvider(TableSpec tableSpec) {
-    return new InMemoryTableProvider(tableSpec);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableDescriptor.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableDescriptor.java
new file mode 100644
index 0000000..c2d08ca
--- /dev/null
+++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.storage.kv.inmemory.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.storage.kv.descriptors.BaseLocalStoreBackedTableDescriptor;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Table descriptor for in-memory tables
+ *
+ * @param <K> the type of the key
+ * @param <V> the type of the value
+ */
+public class InMemoryTableDescriptor<K, V> extends BaseLocalStoreBackedTableDescriptor<K, V, InMemoryTableDescriptor<K, V>> {
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   */
+  public InMemoryTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
+   * @param serde the serde for key and value
+   */
+  public InMemoryTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
+  @Override
+  protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
+    super.generateTableSpecConfig(tableSpecConfig);
+  }
+
+  @Override
+  public TableSpec getTableSpec() {
+
+    validate();
+
+    Map<String, String> tableSpecConfig = new HashMap<>();
+    generateTableSpecConfig(tableSpecConfig);
+
+    return new TableSpec(tableId, serde, InMemoryTableProviderFactory.class.getName(), tableSpecConfig,
+        sideInputs, sideInputsProcessor);
+  }
+
+  private void addInMemoryConfig(Map<String, String> map, String key, String value) {
+    map.put("inmemory." + key, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProvider.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProvider.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProvider.java
new file mode 100644
index 0000000..448a316
--- /dev/null
+++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProvider.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.storage.kv.inmemory.descriptors;
+
+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.descriptors.BaseLocalStoreBackedTableProvider;
+import org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Table provider of an in-memory table
+ */
+public class InMemoryTableProvider extends BaseLocalStoreBackedTableProvider {
+
+  public InMemoryTableProvider(TableSpec tableSpec) {
+    super(tableSpec);
+  }
+
+  @Override
+  public Map<String, String> generateConfig(Config jobConfig, Map<String, String> generatedConfig) {
+
+    Map<String, String> tableConfig = new HashMap<>();
+
+    // Store factory configuration
+    tableConfig.put(String.format(
+        StorageConfig.FACTORY(), tableSpec.getId()),
+        InMemoryKeyValueStorageEngineFactory.class.getName());
+
+    // Common store configuration
+    tableConfig.putAll(generateCommonStoreConfig(jobConfig, generatedConfig));
+
+    // Rest of the configuration
+    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());
+
+    return tableConfig;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProviderFactory.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProviderFactory.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProviderFactory.java
new file mode 100644
index 0000000..51d5e40
--- /dev/null
+++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/descriptors/InMemoryTableProviderFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.storage.kv.inmemory.descriptors;
+
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
+import org.apache.samza.table.TableSpec;
+
+/**
+ * Factory class for an in-memory table provider
+ */
+public class InMemoryTableProviderFactory implements TableProviderFactory {
+  @Override
+  public TableProvider getTableProvider(TableSpec tableSpec) {
+    return new InMemoryTableProvider(tableSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
deleted file mode 100644
index 89bd058..0000000
--- a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
+++ /dev/null
@@ -1,48 +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.inmemory;
-
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.table.TableSpec;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class TestInMemoryTableDescriptor {
-  @Test
-  public void testTableSpec() {
-
-    TableSpec tableSpec = new InMemoryTableDescriptor("1",
-            KVSerde.of(new IntegerSerde(), new StringSerde()))
-        .withConfig("inmemory.abc", "xyz")
-        .getTableSpec();
-
-    Assert.assertNotNull(tableSpec.getSerde());
-    Assert.assertNotNull(tableSpec.getSerde().getKeySerde());
-    Assert.assertNotNull(tableSpec.getSerde().getValueSerde());
-    Assert.assertEquals("xyz", getConfig(tableSpec, "abc"));
-  }
-
-  private String getConfig(TableSpec tableSpec, String key) {
-    return tableSpec.getConfig().get("inmemory." + key);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 2145b68..0000000
--- a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java
+++ /dev/null
@@ -1,66 +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.inmemory;
-
-import java.util.HashMap;
-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;
-import org.apache.samza.table.TableProvider;
-import org.apache.samza.table.TableSpec;
-import org.junit.Test;
-
-import junit.framework.Assert;
-
-
-public class TestInMemoryTableProvider {
-  @Test
-  public void testGenerateConfig() {
-    Map<String, String> tableSpecConfig = new HashMap<>();
-    tableSpecConfig.put("inmemory.c1", "c1-value");
-    tableSpecConfig.put("inmemory.c2", "c2-value");
-    tableSpecConfig.put("c3", "c3-value");
-    tableSpecConfig.put("c4", "c4-value");
-
-    TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()),
-        "my-table-provider-factory", tableSpecConfig);
-
-    Map<String, String> 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<String, String> 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")));
-    Assert.assertEquals(
-        InMemoryKeyValueStorageEngineFactory.class.getName(),
-        tableConfig.get(String.format(StorageConfig.FACTORY(), "t1")));
-    Assert.assertEquals("c1-value", tableConfig.get("stores.t1.c1"));
-    Assert.assertEquals("c2-value", tableConfig.get("stores.t1.c2"));
-    Assert.assertEquals("c3-value", tableConfig.get("tables.t1.c3"));
-    Assert.assertEquals("c4-value", tableConfig.get("tables.t1.c4"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableDescriptor.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableDescriptor.java
new file mode 100644
index 0000000..33e3c35
--- /dev/null
+++ b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/descriptors/TestInMemoryTableDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.kv.inmemory.descriptors;
+
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.table.TableSpec;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestInMemoryTableDescriptor {
+  @Test
+  public void testTableSpec() {
+
+    TableSpec tableSpec = new InMemoryTableDescriptor("1",
+            KVSerde.of(new IntegerSerde(), new StringSerde()))
+        .withConfig("inmemory.abc", "xyz")
+        .getTableSpec();
+
+    Assert.assertNotNull(tableSpec.getSerde());
+    Assert.assertNotNull(tableSpec.getSerde().getKeySerde());
+    Assert.assertNotNull(tableSpec.getSerde().getValueSerde());
+    Assert.assertEquals("xyz", getConfig(tableSpec, "abc"));
+  }
+
+  private String getConfig(TableSpec tableSpec, String key) {
+    return tableSpec.getConfig().get("inmemory." + key);
+  }
+}


[12/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
Consolidating package names for System, Stream, Application and Table descriptors.

Everything in this PR is either:
1. A package name change and a corresponding file move.
2. Javadoc changes to use FQN in link tags to fix checkstyle complaints about unused imports, and corresponding fixes to make them fit within line width. No change in contents.
3. In a couple of places, changing method visibility to public with VisibleForTesting annotations for accessing them from Tests (RemoteReadWriteTable.java, RemoteReadableTable.java)

Author: Prateek Maheshwari <pm...@apache.org>

Reviewers: Bharath Kumarasubramanian <bk...@linkedin.com>, Jagadish Venkatraman <vj...@gmail.com>, Yi Pan <ni...@gmail.com>

Closes #720 from prateekm/descriptor-package-cleanup


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/74675cea
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/74675cea
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/74675cea

Branch: refs/heads/master
Commit: 74675cea55d163bf18bf16c8619355009af2300c
Parents: 9a5094d
Author: Prateek Maheshwari <pm...@apache.org>
Authored: Fri Oct 12 18:34:34 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Fri Oct 12 18:34:34 2018 -0700

----------------------------------------------------------------------
 docs/startup/quick-start/versioned/index.md     |   2 +-
 .../application/ApplicationDescriptor.java      |  94 ---
 .../samza/application/SamzaApplication.java     |   1 +
 .../samza/application/StreamApplication.java    |   1 +
 .../StreamApplicationDescriptor.java            | 107 ----
 .../samza/application/TaskApplication.java      |   1 +
 .../application/TaskApplicationDescriptor.java  |  64 --
 .../descriptors/ApplicationDescriptor.java      |  96 +++
 .../StreamApplicationDescriptor.java            | 107 ++++
 .../descriptors/TaskApplicationDescriptor.java  |  65 ++
 .../ApplicationContainerContextFactory.java     |   3 +-
 .../context/ApplicationTaskContextFactory.java  |   3 +-
 .../apache/samza/operators/MessageStream.java   |   5 +-
 .../apache/samza/operators/TableDescriptor.java |  64 --
 .../descriptors/GenericInputDescriptor.java     |  48 --
 .../descriptors/GenericOutputDescriptor.java    |  48 --
 .../descriptors/GenericSystemDescriptor.java    |  61 --
 .../base/stream/InputDescriptor.java            | 183 ------
 .../base/stream/OutputDescriptor.java           |  44 --
 .../base/stream/StreamDescriptor.java           | 136 -----
 .../ExpandingInputDescriptorProvider.java       |  44 --
 .../base/system/OutputDescriptorProvider.java   |  48 --
 .../system/SimpleInputDescriptorProvider.java   |  43 --
 .../base/system/SystemDescriptor.java           | 177 ------
 .../TransformingInputDescriptorProvider.java    |  44 --
 .../operators/functions/ClosableFunction.java   |  12 +-
 .../operators/functions/InitableFunction.java   |   9 +-
 .../operators/functions/InputTransformer.java   |  45 --
 .../operators/functions/StreamExpander.java     |  58 --
 .../ExpandingInputDescriptorProvider.java       |  43 ++
 .../descriptors/GenericInputDescriptor.java     |  47 ++
 .../descriptors/GenericOutputDescriptor.java    |  47 ++
 .../descriptors/GenericSystemDescriptor.java    |  58 ++
 .../system/descriptors/InputDescriptor.java     | 181 ++++++
 .../system/descriptors/InputTransformer.java    |  47 ++
 .../system/descriptors/OutputDescriptor.java    |  43 ++
 .../descriptors/OutputDescriptorProvider.java   |  47 ++
 .../SimpleInputDescriptorProvider.java          |  42 ++
 .../system/descriptors/StreamDescriptor.java    | 135 +++++
 .../system/descriptors/StreamExpander.java      |  57 ++
 .../system/descriptors/SystemDescriptor.java    | 175 ++++++
 .../TransformingInputDescriptorProvider.java    |  43 ++
 .../samza/table/TableDescriptorsProvider.java   |   2 +-
 .../org/apache/samza/table/TableProvider.java   |  61 --
 .../samza/table/TableProviderFactory.java       |  35 --
 .../java/org/apache/samza/table/TableSpec.java  |   5 +-
 .../table/descriptors/TableDescriptor.java      |  64 ++
 .../samza/table/descriptors/TableProvider.java  |  62 ++
 .../table/descriptors/TableProviderFactory.java |  36 ++
 .../TestExpandingInputDescriptor.java           |  59 --
 .../descriptors/TestGenericInputDescriptor.java | 123 ----
 .../TestGenericSystemDescriptor.java            |  63 --
 .../descriptors/TestSimpleInputDescriptor.java  |  63 --
 .../TestTransformingInputDescriptor.java        |  64 --
 .../ExampleExpandingInputDescriptor.java        |  30 -
 .../ExampleExpandingOutputDescriptor.java       |  29 -
 .../ExampleExpandingSystemDescriptor.java       |  49 --
 .../serde/ExampleSimpleInputDescriptor.java     |  30 -
 .../serde/ExampleSimpleOutputDescriptor.java    |  29 -
 .../serde/ExampleSimpleSystemDescriptor.java    |  43 --
 .../ExampleTransformingInputDescriptor.java     |  30 -
 .../ExampleTransformingOutputDescriptor.java    |  29 -
 .../ExampleTransformingSystemDescriptor.java    |  43 --
 .../TestExpandingInputDescriptor.java           |  59 ++
 .../descriptors/TestGenericInputDescriptor.java | 123 ++++
 .../TestGenericSystemDescriptor.java            |  63 ++
 .../descriptors/TestSimpleInputDescriptor.java  |  63 ++
 .../TestTransformingInputDescriptor.java        |  64 ++
 .../ExampleExpandingInputDescriptor.java        |  30 +
 .../ExampleExpandingOutputDescriptor.java       |  29 +
 .../ExampleExpandingSystemDescriptor.java       |  49 ++
 .../serde/ExampleSimpleInputDescriptor.java     |  30 +
 .../serde/ExampleSimpleOutputDescriptor.java    |  29 +
 .../serde/ExampleSimpleSystemDescriptor.java    |  43 ++
 .../ExampleTransformingInputDescriptor.java     |  30 +
 .../ExampleTransformingOutputDescriptor.java    |  29 +
 .../ExampleTransformingSystemDescriptor.java    |  43 ++
 .../eventhub/EventHubsInputDescriptor.java      | 121 ----
 .../eventhub/EventHubsOutputDescriptor.java     | 104 ----
 .../eventhub/EventHubsSystemDescriptor.java     | 217 -------
 .../descriptors/EventHubsInputDescriptor.java   | 122 ++++
 .../descriptors/EventHubsOutputDescriptor.java  | 105 ++++
 .../descriptors/EventHubsSystemDescriptor.java  | 219 +++++++
 .../eventhub/TestEventHubsInputDescriptor.java  |  91 ---
 .../eventhub/TestEventHubsOutputDescriptor.java |  88 ---
 .../eventhub/TestEventHubsSystemDescriptor.java | 112 ----
 .../TestEventHubsInputDescriptor.java           |  92 +++
 .../TestEventHubsOutputDescriptor.java          |  89 +++
 .../TestEventHubsSystemDescriptor.java          | 113 ++++
 .../application/ApplicationDescriptorImpl.java  | 298 ---------
 .../application/ApplicationDescriptorUtil.java  |  51 --
 .../application/LegacyTaskApplication.java      |   1 +
 .../StreamApplicationDescriptorImpl.java        | 366 -----------
 .../TaskApplicationDescriptorImpl.java          | 143 -----
 .../descriptors/ApplicationDescriptorImpl.java  | 300 +++++++++
 .../descriptors/ApplicationDescriptorUtil.java  |  54 ++
 .../StreamApplicationDescriptorImpl.java        | 367 +++++++++++
 .../TaskApplicationDescriptorImpl.java          | 144 +++++
 .../apache/samza/config/JavaTableConfig.java    |   4 +-
 .../samza/execution/ExecutionPlanner.java       |   6 +-
 .../org/apache/samza/execution/JobGraph.java    |   4 +-
 .../samza/execution/JobGraphJsonGenerator.java  |   4 +-
 .../org/apache/samza/execution/JobNode.java     |   4 +-
 .../org/apache/samza/execution/JobPlanner.java  |   4 +-
 .../apache/samza/execution/LocalJobPlanner.java |   4 +-
 .../samza/execution/RemoteJobPlanner.java       |   4 +-
 .../samza/operators/BaseTableDescriptor.java    | 110 ----
 .../samza/operators/MessageStreamImpl.java      |   2 +-
 .../samza/operators/OperatorSpecGraph.java      |   2 +-
 .../descriptors/DelegatingSystemDescriptor.java |  64 --
 .../samza/operators/impl/InputOperatorImpl.java |   2 +-
 .../samza/operators/spec/InputOperatorSpec.java |   2 +-
 .../samza/operators/spec/OperatorSpec.java      |   3 +-
 .../samza/operators/spec/OperatorSpecs.java     |   2 +-
 .../stream/IntermediateMessageStreamImpl.java   |   2 +-
 .../samza/runtime/LocalApplicationRunner.java   |   6 +-
 .../samza/runtime/LocalContainerRunner.java     |   6 +-
 .../samza/runtime/RemoteApplicationRunner.java  |   6 +-
 .../descriptors/DelegatingSystemDescriptor.java |  61 ++
 .../samza/table/TableConfigGenerator.java       |   6 +-
 .../org/apache/samza/table/TableManager.java    |   2 +
 .../table/caching/CachingTableDescriptor.java   | 164 -----
 .../table/caching/CachingTableProvider.java     | 104 ----
 .../caching/CachingTableProviderFactory.java    |  34 --
 .../descriptors/CachingTableDescriptor.java     | 166 +++++
 .../descriptors/CachingTableProvider.java       | 105 ++++
 .../CachingTableProviderFactory.java            |  34 ++
 .../guava/GuavaCacheTableDescriptor.java        |  75 ---
 .../caching/guava/GuavaCacheTableProvider.java  |  59 --
 .../guava/GuavaCacheTableProviderFactory.java   |  34 --
 .../descriptors/GuavaCacheTableDescriptor.java  |  75 +++
 .../descriptors/GuavaCacheTableProvider.java    |  60 ++
 .../GuavaCacheTableProviderFactory.java         |  34 ++
 .../descriptors/BaseHybridTableDescriptor.java  |  48 ++
 .../table/descriptors/BaseTableDescriptor.java  | 110 ++++
 .../table/hybrid/BaseHybridTableDescriptor.java |  50 --
 .../table/remote/RemoteReadWriteTable.java      |  15 +-
 .../samza/table/remote/RemoteReadableTable.java |  26 +-
 .../table/remote/RemoteTableDescriptor.java     | 275 ---------
 .../samza/table/remote/RemoteTableProvider.java | 200 ------
 .../remote/RemoteTableProviderFactory.java      |  38 --
 .../descriptors/RemoteTableDescriptor.java      | 278 +++++++++
 .../remote/descriptors/RemoteTableProvider.java | 202 +++++++
 .../descriptors/RemoteTableProviderFactory.java |  38 ++
 .../table/retry/RetriableReadFunction.java      |   2 +-
 .../table/retry/RetriableWriteFunction.java     |   2 +-
 .../samza/table/utils/BaseTableProvider.java    |  73 ---
 .../utils/descriptors/BaseTableProvider.java    |  73 +++
 .../apache/samza/task/StreamOperatorTask.java   |  11 +-
 .../org/apache/samza/task/TaskFactoryUtil.java  |   8 +-
 .../samza/job/local/ThreadJobFactory.scala      |   3 +-
 .../application/MockStreamApplication.java      |   2 +
 .../samza/application/TestApplicationUtil.java  |   2 +
 .../TestStreamApplicationDescriptorImpl.java    | 601 ------------------
 .../TestTaskApplicationDescriptorImpl.java      | 172 ------
 .../TestStreamApplicationDescriptorImpl.java    | 602 +++++++++++++++++++
 .../TestTaskApplicationDescriptorImpl.java      | 173 ++++++
 .../execution/ExecutionPlannerTestBase.java     |  10 +-
 .../samza/execution/TestExecutionPlanner.java   |  22 +-
 .../apache/samza/execution/TestJobGraph.java    |   2 +-
 .../execution/TestJobGraphJsonGenerator.java    |   8 +-
 .../TestJobNodeConfigurationGenerator.java      |  14 +-
 .../samza/execution/TestLocalJobPlanner.java    |   6 +-
 .../samza/execution/TestRemoteJobPlanner.java   |   6 +-
 .../samza/operators/TestJoinOperator.java       |   6 +-
 .../samza/operators/TestMessageStreamImpl.java  |   2 +-
 .../samza/operators/TestOperatorSpecGraph.java  |   2 +-
 .../operators/impl/TestOperatorImplGraph.java   |   8 +-
 .../operators/impl/TestWindowOperator.java      |   6 +-
 .../spec/TestPartitionByOperatorSpec.java       |   6 +-
 .../runtime/TestLocalApplicationRunner.java     |   6 +-
 .../apache/samza/table/TestTableManager.java    |   2 +
 .../samza/table/caching/TestCachingTable.java   |  10 +-
 .../table/remote/TestRemoteTableDescriptor.java | 236 --------
 .../descriptors/TestRemoteTableDescriptor.java  | 239 ++++++++
 .../apache/samza/task/TestTaskFactoryUtil.java  |   6 +-
 .../system/kafka/KafkaInputDescriptor.java      | 108 ----
 .../system/kafka/KafkaOutputDescriptor.java     |  39 --
 .../system/kafka/KafkaSystemDescriptor.java     | 245 --------
 .../kafka/descriptors/KafkaInputDescriptor.java | 108 ++++
 .../descriptors/KafkaOutputDescriptor.java      |  39 ++
 .../descriptors/KafkaSystemDescriptor.java      | 246 ++++++++
 .../system/kafka/TestKafkaInputDescriptor.java  |  66 --
 .../kafka/TestKafkaSystemAdminWithMock.java     |   1 -
 .../system/kafka/TestKafkaSystemDescriptor.java |  69 ---
 .../descriptors/TestKafkaInputDescriptor.java   |  64 ++
 .../descriptors/TestKafkaSystemDescriptor.java  |  70 +++
 .../kv/inmemory/InMemoryTableDescriptor.java    |  74 ---
 .../kv/inmemory/InMemoryTableProvider.java      |  70 ---
 .../inmemory/InMemoryTableProviderFactory.java  |  33 -
 .../descriptors/InMemoryTableDescriptor.java    |  74 +++
 .../descriptors/InMemoryTableProvider.java      |  71 +++
 .../InMemoryTableProviderFactory.java           |  33 +
 .../inmemory/TestInMemoryTableDescriptor.java   |  48 --
 .../kv/inmemory/TestInMemoryTableProvider.java  |  66 --
 .../TestInMemoryTableDescriptor.java            |  48 ++
 .../descriptors/TestInMemoryTableProvider.java  |  67 +++
 .../storage/kv/RocksDbTableDescriptor.java      | 339 -----------
 .../samza/storage/kv/RocksDbTableProvider.java  |  73 ---
 .../storage/kv/RocksDbTableProviderFactory.java |  31 -
 .../kv/descriptors/RocksDbTableDescriptor.java  | 339 +++++++++++
 .../kv/descriptors/RocksDbTableProvider.java    |  74 +++
 .../RocksDbTableProviderFactory.java            |  31 +
 .../storage/kv/TestRocksDbTableDescriptor.java  | 100 ---
 .../storage/kv/TestRocksDbTableProvider.java    |  67 ---
 .../descriptors/TestRocksDbTableDescriptor.java | 100 +++
 .../descriptors/TestRocksDbTableProvider.java   |  68 +++
 .../kv/BaseLocalStoreBackedTableDescriptor.java | 168 ------
 .../kv/BaseLocalStoreBackedTableProvider.java   | 147 -----
 .../BaseLocalStoreBackedTableDescriptor.java    | 168 ++++++
 .../BaseLocalStoreBackedTableProvider.java      | 149 +++++
 .../TestBaseLocalStoreBackedTableProvider.java  | 149 -----
 .../TestBaseLocalStoreBackedTableProvider.java  | 150 +++++
 .../sql/impl/ConfigBasedIOResolverFactory.java  |   4 +-
 .../samza/sql/interfaces/SqlIOConfig.java       |   2 +-
 .../samza/sql/runner/SamzaSqlApplication.java   |   2 +-
 .../samza/sql/translator/ModifyTranslator.java  |   8 +-
 .../samza/sql/translator/QueryTranslator.java   |  10 +-
 .../samza/sql/translator/ScanTranslator.java    |   6 +-
 .../samza/sql/translator/TranslatorContext.java |   4 +-
 .../sql/testutil/TestIOResolverFactory.java     |  12 +-
 .../sql/translator/TestFilterTranslator.java    |   2 +-
 .../sql/translator/TestJoinTranslator.java      |   6 +-
 .../sql/translator/TestProjectTranslator.java   |   2 +-
 .../sql/translator/TestQueryTranslator.java     |   2 +-
 .../sql/translator/TranslatorTestBase.java      |   5 +-
 .../example/AppWithGlobalConfigExample.java     |   8 +-
 .../apache/samza/example/BroadcastExample.java  |   8 +-
 .../samza/example/KeyValueStoreExample.java     |   8 +-
 .../org/apache/samza/example/MergeExample.java  |   8 +-
 .../samza/example/OrderShipmentJoinExample.java |   8 +-
 .../samza/example/PageViewCounterExample.java   |   8 +-
 .../samza/example/RepartitionExample.java       |   8 +-
 .../samza/example/TaskApplicationExample.java   |  12 +-
 .../org/apache/samza/example/WindowExample.java |   8 +-
 .../samza/test/framework/StreamAssert.java      |   6 +-
 .../apache/samza/test/framework/TestRunner.java |   6 +-
 .../system/InMemoryInputDescriptor.java         |  42 --
 .../system/InMemoryOutputDescriptor.java        |  46 --
 .../system/InMemorySystemDescriptor.java        | 109 ----
 .../descriptors/InMemoryInputDescriptor.java    |  42 ++
 .../descriptors/InMemoryOutputDescriptor.java   |  46 ++
 .../descriptors/InMemorySystemDescriptor.java   | 109 ++++
 .../TestStandaloneIntegrationApplication.java   |   8 +-
 .../EndOfStreamIntegrationTest.java             |   6 +-
 .../WatermarkIntegrationTest.java               |   6 +-
 .../AsyncStreamTaskIntegrationTest.java         |   6 +-
 .../test/framework/BroadcastAssertApp.java      |   6 +-
 .../test/framework/FaultInjectionTest.java      |   6 +-
 .../StreamApplicationIntegrationTest.java       |  19 +-
 .../framework/StreamTaskIntegrationTest.java    |  21 +-
 .../samza/test/framework/TestSchedulingApp.java |   6 +-
 .../test/operator/RepartitionJoinWindowApp.java |   6 +-
 .../test/operator/RepartitionWindowApp.java     |   8 +-
 .../samza/test/operator/SessionWindowApp.java   |   8 +-
 .../samza/test/operator/TumblingWindowApp.java  |   8 +-
 .../test/processor/TestStreamApplication.java   |   8 +-
 .../apache/samza/test/table/TestLocalTable.java |   8 +-
 .../table/TestLocalTableWithSideInputs.java     |  16 +-
 .../samza/test/table/TestRemoteTable.java       |  16 +-
 .../table/TestTableDescriptorsProvider.java     |  10 +-
 .../benchmark/SystemConsumerWithSamzaBench.java |   4 +-
 262 files changed, 8425 insertions(+), 8353 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/docs/startup/quick-start/versioned/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/quick-start/versioned/index.md b/docs/startup/quick-start/versioned/index.md
index 44b8376..a046ee7 100644
--- a/docs/startup/quick-start/versioned/index.md
+++ b/docs/startup/quick-start/versioned/index.md
@@ -54,7 +54,7 @@ Now let’s write some code! The first step is to create your own Samza applicat
 package samzaapp;
 
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 
 public class WordCount implements StreamApplication {
  @Override

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.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
deleted file mode 100644
index e806aad..0000000
--- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java
+++ /dev/null
@@ -1,94 +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.util.Map;
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.ApplicationContainerContextFactory;
-import org.apache.samza.context.ApplicationTaskContextFactory;
-import org.apache.samza.metrics.MetricsReporterFactory;
-import org.apache.samza.runtime.ProcessorLifecycleListenerFactory;
-
-
-/**
- * The interface class to describe the configuration, input and output streams, and processing logic in a {@link SamzaApplication}.
- * <p>
- * 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 <S> sub-class of user application descriptor.
- */
-@InterfaceStability.Evolving
-public interface ApplicationDescriptor<S extends ApplicationDescriptor> {
-
-  /**
-   * Get the {@link Config} of the application
-   * @return config of the application
-   */
-  Config getConfig();
-
-  /**
-   * Sets the {@link ApplicationContainerContextFactory} for this application. Each task will be given access to a
-   * different instance of the {@link org.apache.samza.context.ApplicationContainerContext} that this creates. The
-   * context can be accessed through the {@link org.apache.samza.context.Context}.
-   * <p>
-   * Setting this is optional.
-   *
-   * @param factory the {@link ApplicationContainerContextFactory} for this application
-   * @return type {@code S} of {@link ApplicationDescriptor} with {@code factory} set as its
-   * {@link ApplicationContainerContextFactory}
-   */
-  S withApplicationContainerContextFactory(ApplicationContainerContextFactory<?> factory);
-
-  /**
-   * Sets the {@link ApplicationTaskContextFactory} for this application. Each task will be given access to a different
-   * instance of the {@link org.apache.samza.context.ApplicationTaskContext} that this creates. The context can be
-   * accessed through the {@link org.apache.samza.context.Context}.
-   * <p>
-   * Setting this is optional.
-   *
-   * @param factory the {@link ApplicationTaskContextFactory} for this application
-   * @return type {@code S} of {@link ApplicationDescriptor} with {@code factory} set as its
-   * {@link ApplicationTaskContextFactory}
-   */
-  S withApplicationTaskContextFactory(ApplicationTaskContextFactory<?> factory);
-
-  /**
-   * Sets the {@link ProcessorLifecycleListenerFactory} for this application.
-   *
-   * <p>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 type {@code S} of {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory}
-   */
-  S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory);
-
-  /**
-   * Sets a set of customized {@link MetricsReporterFactory}s in the application
-   *
-   * @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<String, MetricsReporterFactory> reporterFactories);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java b/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java
index 7606be8..5423e2e 100644
--- a/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java
+++ b/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java
@@ -19,6 +19,7 @@
 package org.apache.samza.application;
 
 import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java
----------------------------------------------------------------------
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 a83cb37..fe77045 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,6 +19,7 @@
 package org.apache.samza.application;
 
 import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
 
 /**
  * Describes and initializes the transforms for processing message streams and generating results in high-level API. 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index dc24771..0000000
--- a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java
+++ /dev/null
@@ -1,107 +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 org.apache.samza.annotation.InterfaceStability;
-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.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 {@link SamzaApplication} in high-level API in Samza.
- */
-@InterfaceStability.Evolving
-public interface StreamApplicationDescriptor extends ApplicationDescriptor<StreamApplicationDescriptor> {
-
-  /**
-   * Sets the default SystemDescriptor to use for intermediate streams. This is equivalent to setting
-   * {@code job.default.system} and its properties in configuration.
-   * <p>
-   * If the default system descriptor is set, it must be set <b>before</b> creating any input/output/intermediate streams.
-   *
-   * @param defaultSystemDescriptor the default system descriptor to use
-   * @return type {@code S} of {@link ApplicationDescriptor} with {@code defaultSystemDescriptor} set as its default system
-   */
-  StreamApplicationDescriptor withDefaultSystem(SystemDescriptor<?> defaultSystemDescriptor);
-
-  /**
-   * Gets the input {@link MessageStream} corresponding to the {@code inputDescriptor}.
-   * <p>
-   * A {@code MessageStream<KV<K, V>}, obtained by calling this method with a descriptor with a {@code KVSerde<K, V>},
-   * can receive messages of type {@code KV<K, V>}. An input {@code MessageStream<M>}, obtained using a descriptor with
-   * any other {@code Serde<M>}, can receive messages of type M - the key in the incoming message is ignored.
-   * <p>
-   * A {@code KVSerde<NoOpSerde, NoOpSerde>} or {@code NoOpSerde} may be used for the descriptor if the
-   * {@code SystemConsumer} deserializes the incoming messages itself, and no further deserialization is required from
-   * the framework.
-   * <p>
-   * Multiple invocations of this method with the same {@code inputDescriptor} will throw an
-   * {@link IllegalStateException}.
-   *
-   * @param inputDescriptor the descriptor for the stream
-   * @param <M> the type of messages in the input {@link MessageStream}
-   * @return the input {@link MessageStream}
-   * @throws IllegalStateException when invoked multiple times with the same {@code inputDescriptor}
-   */
-  <M> MessageStream<M> getInputStream(InputDescriptor<M, ?> inputDescriptor);
-
-  /**
-   * Gets the {@link OutputStream} corresponding to the {@code outputDescriptor}.
-   * <p>
-   * An {@code OutputStream<KV<K, V>>}, obtained by calling this method with a descriptor with a {@code KVSerde<K, V>},
-   * can send messages of type {@code KV<K, V>}. An {@code OutputStream<M>}, obtained using a descriptor with any
-   * other {@code Serde<M>}, can send messages of type M without a key.
-   * <p>
-   * A {@code KVSerde<NoOpSerde, NoOpSerde>} or {@code NoOpSerde} may be used for the descriptor if the
-   * {@code SystemProducer} serializes the outgoing messages itself, and no prior serialization is required from
-   * the framework.
-   * <p>
-   * When sending messages to an {@code OutputStream<KV<K, V>>}, messages are partitioned using their serialized key.
-   * When sending messages to any other {@code OutputStream<M>}, messages are partitioned using a null partition key.
-   * <p>
-   * Multiple invocations of this method with the same {@code outputDescriptor} will throw an
-   * {@link IllegalStateException}.
-   *
-   * @param outputDescriptor the descriptor for the stream
-   * @param <M> the type of messages in the {@link OutputStream}
-   * @return the {@link OutputStream}
-   * @throws IllegalStateException when invoked multiple times with the same {@code outputDescriptor}
-   */
-  <M> OutputStream<M> getOutputStream(OutputDescriptor<M, ?> outputDescriptor);
-
-  /**
-   * Gets the {@link Table} corresponding to the {@link TableDescriptor}.
-   * <p>
-   * Multiple invocations of this method with the same {@link TableDescriptor} will throw an
-   * {@link IllegalStateException}.
-   *
-   * @param tableDescriptor the {@link TableDescriptor}
-   * @param <K> the type of the key
-   * @param <V> the type of the value
-   * @return the {@link Table} corresponding to the {@code tableDescriptor}
-   * @throws IllegalStateException when invoked multiple times with the same {@link TableDescriptor}
-   */
-  <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDescriptor);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/TaskApplication.java
----------------------------------------------------------------------
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 424634d..d84aa12 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
@@ -19,6 +19,7 @@
 package org.apache.samza.application;
 
 import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptor;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 0226bb5..0000000
--- a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java
+++ /dev/null
@@ -1,64 +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 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;
-
-
-/**
- *  The interface to describe a {@link SamzaApplication} that uses low-level API task for processing.
- */
-@InterfaceStability.Evolving
-public interface TaskApplicationDescriptor extends ApplicationDescriptor<TaskApplicationDescriptor> {
-
-  /**
-   * 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 {@link TaskFactory} including the low-level task processing logic. The only allowed task factory
-   *                classes are {@link org.apache.samza.task.StreamTaskFactory} and {@link org.apache.samza.task.AsyncStreamTaskFactory}.
-   */
-  void setTaskFactory(TaskFactory factory);
-
-  /**
-   * Adds the input stream to the application.
-   *
-   * @param isd the {@link InputDescriptor}
-   */
-  void addInputStream(InputDescriptor isd);
-
-  /**
-   * Adds the output stream to the application.
-   *
-   * @param osd the {@link OutputDescriptor} of the output stream
-   */
-  void addOutputStream(OutputDescriptor osd);
-
-  /**
-   * Adds the {@link TableDescriptor} used in the application
-   *
-   * @param table {@link TableDescriptor}
-   */
-  void addTable(TableDescriptor table);
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java
new file mode 100644
index 0000000..b1e78b0
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.application.descriptors;
+
+import java.util.Map;
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.context.ApplicationContainerContextFactory;
+import org.apache.samza.context.ApplicationTaskContextFactory;
+import org.apache.samza.metrics.MetricsReporterFactory;
+import org.apache.samza.runtime.ProcessorLifecycleListenerFactory;
+
+
+/**
+ * The interface class to describe the configuration, input and output streams, and processing logic in a
+ * {@link org.apache.samza.application.SamzaApplication}.
+ * <p>
+ * Sub-classes {@link StreamApplicationDescriptor} and {@link TaskApplicationDescriptor} are specific interfaces for
+ * applications written in high-level {@link org.apache.samza.application.StreamApplication} and low-level
+ * {@link org.apache.samza.application.TaskApplication} APIs, respectively.
+ *
+ * @param <S> sub-class of user application descriptor.
+ */
+@InterfaceStability.Evolving
+public interface ApplicationDescriptor<S extends ApplicationDescriptor> {
+
+  /**
+   * Get the {@link Config} of the application
+   * @return config of the application
+   */
+  Config getConfig();
+
+  /**
+   * Sets the {@link ApplicationContainerContextFactory} for this application. Each task will be given access to a
+   * different instance of the {@link org.apache.samza.context.ApplicationContainerContext} that this creates. The
+   * context can be accessed through the {@link org.apache.samza.context.Context}.
+   * <p>
+   * Setting this is optional.
+   *
+   * @param factory the {@link ApplicationContainerContextFactory} for this application
+   * @return type {@code S} of {@link ApplicationDescriptor} with {@code factory} set as its
+   * {@link ApplicationContainerContextFactory}
+   */
+  S withApplicationContainerContextFactory(ApplicationContainerContextFactory<?> factory);
+
+  /**
+   * Sets the {@link ApplicationTaskContextFactory} for this application. Each task will be given access to a different
+   * instance of the {@link org.apache.samza.context.ApplicationTaskContext} that this creates. The context can be
+   * accessed through the {@link org.apache.samza.context.Context}.
+   * <p>
+   * Setting this is optional.
+   *
+   * @param factory the {@link ApplicationTaskContextFactory} for this application
+   * @return type {@code S} of {@link ApplicationDescriptor} with {@code factory} set as its
+   * {@link ApplicationTaskContextFactory}
+   */
+  S withApplicationTaskContextFactory(ApplicationTaskContextFactory<?> factory);
+
+  /**
+   * Sets the {@link ProcessorLifecycleListenerFactory} for this application.
+   *
+   * <p>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 type {@code S} of {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory}
+   */
+  S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory);
+
+  /**
+   * Sets a set of customized {@link MetricsReporterFactory}s in the application
+   *
+   * @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<String, MetricsReporterFactory> reporterFactories);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java
new file mode 100644
index 0000000..383e9ce
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.table.Table;
+
+
+/**
+ * The interface class to describe a {@link org.apache.samza.application.SamzaApplication} in high-level API in Samza.
+ */
+@InterfaceStability.Evolving
+public interface StreamApplicationDescriptor extends ApplicationDescriptor<StreamApplicationDescriptor> {
+
+  /**
+   * Sets the default SystemDescriptor to use for intermediate streams. This is equivalent to setting
+   * {@code job.default.system} and its properties in configuration.
+   * <p>
+   * If the default system descriptor is set, it must be set <b>before</b> creating any input/output/intermediate streams.
+   *
+   * @param defaultSystemDescriptor the default system descriptor to use
+   * @return type {@code S} of {@link ApplicationDescriptor} with {@code defaultSystemDescriptor} set as its default system
+   */
+  StreamApplicationDescriptor withDefaultSystem(SystemDescriptor<?> defaultSystemDescriptor);
+
+  /**
+   * Gets the input {@link MessageStream} corresponding to the {@code inputDescriptor}.
+   * <p>
+   * A {@code MessageStream<KV<K, V>}, obtained by calling this method with a descriptor with a {@code KVSerde<K, V>},
+   * can receive messages of type {@code KV<K, V>}. An input {@code MessageStream<M>}, obtained using a descriptor with
+   * any other {@code Serde<M>}, can receive messages of type M - the key in the incoming message is ignored.
+   * <p>
+   * A {@code KVSerde<NoOpSerde, NoOpSerde>} or {@code NoOpSerde} may be used for the descriptor if the
+   * {@code SystemConsumer} deserializes the incoming messages itself, and no further deserialization is required from
+   * the framework.
+   * <p>
+   * Multiple invocations of this method with the same {@code inputDescriptor} will throw an
+   * {@link IllegalStateException}.
+   *
+   * @param inputDescriptor the descriptor for the stream
+   * @param <M> the type of messages in the input {@link MessageStream}
+   * @return the input {@link MessageStream}
+   * @throws IllegalStateException when invoked multiple times with the same {@code inputDescriptor}
+   */
+  <M> MessageStream<M> getInputStream(InputDescriptor<M, ?> inputDescriptor);
+
+  /**
+   * Gets the {@link OutputStream} corresponding to the {@code outputDescriptor}.
+   * <p>
+   * An {@code OutputStream<KV<K, V>>}, obtained by calling this method with a descriptor with a {@code KVSerde<K, V>},
+   * can send messages of type {@code KV<K, V>}. An {@code OutputStream<M>}, obtained using a descriptor with any
+   * other {@code Serde<M>}, can send messages of type M without a key.
+   * <p>
+   * A {@code KVSerde<NoOpSerde, NoOpSerde>} or {@code NoOpSerde} may be used for the descriptor if the
+   * {@code SystemProducer} serializes the outgoing messages itself, and no prior serialization is required from
+   * the framework.
+   * <p>
+   * When sending messages to an {@code OutputStream<KV<K, V>>}, messages are partitioned using their serialized key.
+   * When sending messages to any other {@code OutputStream<M>}, messages are partitioned using a null partition key.
+   * <p>
+   * Multiple invocations of this method with the same {@code outputDescriptor} will throw an
+   * {@link IllegalStateException}.
+   *
+   * @param outputDescriptor the descriptor for the stream
+   * @param <M> the type of messages in the {@link OutputStream}
+   * @return the {@link OutputStream}
+   * @throws IllegalStateException when invoked multiple times with the same {@code outputDescriptor}
+   */
+  <M> OutputStream<M> getOutputStream(OutputDescriptor<M, ?> outputDescriptor);
+
+  /**
+   * Gets the {@link Table} corresponding to the {@link TableDescriptor}.
+   * <p>
+   * Multiple invocations of this method with the same {@link TableDescriptor} will throw an
+   * {@link IllegalStateException}.
+   *
+   * @param tableDescriptor the {@link TableDescriptor}
+   * @param <K> the type of the key
+   * @param <V> the type of the value
+   * @return the {@link Table} corresponding to the {@code tableDescriptor}
+   * @throws IllegalStateException when invoked multiple times with the same {@link TableDescriptor}
+   */
+  <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDescriptor);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java
new file mode 100644
index 0000000..4730297
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.application.descriptors;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.task.TaskFactory;
+
+
+/**
+ *  The interface to describe a {@link org.apache.samza.application.SamzaApplication} that uses low-level API task
+ *  for processing.
+ */
+@InterfaceStability.Evolving
+public interface TaskApplicationDescriptor extends ApplicationDescriptor<TaskApplicationDescriptor> {
+
+  /**
+   * 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 {@link TaskFactory} including the low-level task processing logic. The only allowed task factory
+   *                classes are {@link org.apache.samza.task.StreamTaskFactory} and {@link org.apache.samza.task.AsyncStreamTaskFactory}.
+   */
+  void setTaskFactory(TaskFactory factory);
+
+  /**
+   * Adds the input stream to the application.
+   *
+   * @param isd the {@link InputDescriptor}
+   */
+  void addInputStream(InputDescriptor isd);
+
+  /**
+   * Adds the output stream to the application.
+   *
+   * @param osd the {@link OutputDescriptor} of the output stream
+   */
+  void addOutputStream(OutputDescriptor osd);
+
+  /**
+   * Adds the {@link TableDescriptor} used in the application
+   *
+   * @param table {@link TableDescriptor}
+   */
+  void addTable(TableDescriptor table);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java b/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java
index fbc2eef..074b0b4 100644
--- a/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java
+++ b/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java
@@ -29,7 +29,8 @@ import java.io.Serializable;
  * stage. At that stage, the framework-provided job-level and container-level contexts are available for creating the
  * {@link ApplicationContainerContext}.
  * <p>
- * This is {@link Serializable} because it is specified in {@link org.apache.samza.application.ApplicationDescriptor}.
+ * This is {@link Serializable} because it is specified in the
+ * {@link org.apache.samza.application.descriptors.ApplicationDescriptor}.
  * @param <T> concrete type of {@link ApplicationContainerContext} returned by this factory
  */
 public interface ApplicationContainerContextFactory<T extends ApplicationContainerContext> extends Serializable {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java b/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java
index af9ad68..619bbc7 100644
--- a/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java
+++ b/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java
@@ -29,7 +29,8 @@ import java.io.Serializable;
  * task. At that stage, the framework-provided job-level, container-level, and task-level contexts are available for
  * creating the {@link ApplicationTaskContext}. Also, the application-defined container-level context is available.
  * <p>
- * This is {@link Serializable} because it is specified in {@link org.apache.samza.application.ApplicationDescriptor}.
+ * This is {@link Serializable} because it is specified in the
+ * {@link org.apache.samza.application.descriptors.ApplicationDescriptor}.
  * @param <T> concrete type of {@link ApplicationTaskContext} returned by this factory
  */
 public interface ApplicationTaskContextFactory<T extends ApplicationTaskContext> extends Serializable {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
----------------------------------------------------------------------
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 97ac65d..f951a84 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
@@ -40,7 +40,7 @@ import org.apache.samza.table.Table;
  * A stream of messages that can be transformed into another {@link MessageStream}.
  * <p>
  * A {@link MessageStream} corresponding to an input stream can be obtained using
- * {@link org.apache.samza.application.StreamApplicationDescriptor#getInputStream}.
+ * {@link org.apache.samza.application.descriptors.StreamApplicationDescriptor#getInputStream}.
  *
  * @param <M> the type of messages in this stream
  */
@@ -214,7 +214,8 @@ public interface MessageStream<M> {
 
   /**
    * Re-partitions this {@link MessageStream} using keys from the {@code keyExtractor} by creating a new
-   * intermediate stream on the default system provided via {@link org.apache.samza.application.StreamApplicationDescriptor#withDefaultSystem}.
+   * intermediate stream on the default system provided via
+   * {@link org.apache.samza.application.descriptors.StreamApplicationDescriptor#withDefaultSystem}.
    * This intermediate stream is both an output and input to the job.
    * <p>
    * Uses the provided {@link KVSerde} for serialization of keys and values.

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
deleted file mode 100644
index dbcd65e..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
+++ /dev/null
@@ -1,64 +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;
-
-/**
- * User facing class to collect metadata that fully describes a
- * Samza table. This interface should be implemented by concrete table implementations.
- * <p>
- * Typical user code should look like the following, notice <code>withConfig()</code>
- * is defined in this class and the rest in subclasses.
- *
- * <pre>
- * {@code
- * TableDescriptor<Integer, String, ?> tableDesc = new RocksDbTableDescriptor("tbl",
- *         KVSerde.of(new IntegerSerde(), new StringSerde("UTF-8")))
- *     .withBlockSize(1024)
- *     .withConfig("some-key", "some-value");
- * }
- * </pre>
-
- * Once constructed, a table descriptor can be registered with the system. Internally,
- * the table descriptor is then converted to a {@link org.apache.samza.table.TableSpec},
- * which is used to track tables internally.
- *
- * @param <K> the type of the key in this table
- * @param <V> the type of the value in this table
- * @param <D> the type of the concrete table descriptor
- */
-@InterfaceStability.Unstable
-public interface TableDescriptor<K, V, D extends TableDescriptor<K, V, D>> {
-
-  /**
-   * Get the Id of the table
-   * @return Id of the table
-   */
-  String getTableId();
-
-  /**
-   * Add a configuration entry for the table
-   * @param key the key
-   * @param value the value
-   * @return this table descriptor instance
-   */
-  D withConfig(String key, String value);
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericInputDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericInputDescriptor.java
deleted file mode 100644
index 09dd381..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericInputDescriptor.java
+++ /dev/null
@@ -1,48 +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.descriptors;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for a generic input stream.
- * <p>
- * An instance of this descriptor may be obtained from an appropriately configured {@link GenericSystemDescriptor}.
- * <p>
- * If the system being used provides its own system and stream descriptor implementations, they should be used instead.
- * Otherwise, this {@link GenericInputDescriptor} may be used to provide Samza-specific properties of the input stream.
- * Additional system stream specific properties may be provided using {@link #withStreamConfigs}
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- */
-public final class GenericInputDescriptor<StreamMessageType>
-    extends InputDescriptor<StreamMessageType, GenericInputDescriptor<StreamMessageType>> {
-  GenericInputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde) {
-    super(streamId, serde, systemDescriptor, null);
-  }
-
-  @Override
-  public GenericInputDescriptor<StreamMessageType> withPhysicalName(String physicalName) {
-    return super.withPhysicalName(physicalName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericOutputDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericOutputDescriptor.java
deleted file mode 100644
index 155bd4e..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericOutputDescriptor.java
+++ /dev/null
@@ -1,48 +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.descriptors;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for a generic output stream.
- * <p>
- * An instance of this descriptor may be obtained from an appropriately configured {@link GenericSystemDescriptor}.
- * <p>
- * If the system being used provides its own system and stream descriptor implementations, they should be used instead.
- * Otherwise, this {@link GenericOutputDescriptor} may be used to provide Samza-specific properties of the output stream.
- * Additional system stream specific properties may be provided using {@link #withStreamConfigs}
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- */
-public final class GenericOutputDescriptor<StreamMessageType>
-    extends OutputDescriptor<StreamMessageType, GenericOutputDescriptor<StreamMessageType>> {
-  GenericOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde) {
-    super(streamId, serde, systemDescriptor);
-  }
-
-  @Override
-  public GenericOutputDescriptor<StreamMessageType> withPhysicalName(String physicalName) {
-    return super.withPhysicalName(physicalName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericSystemDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericSystemDescriptor.java
deleted file mode 100644
index 24f7932..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/GenericSystemDescriptor.java
+++ /dev/null
@@ -1,61 +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.descriptors;
-
-
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SimpleInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for a generic system.
- * <p>
- * If the system being used provides its own system and stream descriptor implementations, they should be used instead.
- * Otherwise, this {@link GenericSystemDescriptor} may be used to provide Samza-specific properties of the system.
- * Additional system specific properties may be provided using {@link #withSystemConfigs}
- * <p>
- * System properties configured using a descriptor override corresponding properties provided in configuration.
- */
-public final class GenericSystemDescriptor extends SystemDescriptor<GenericSystemDescriptor>
-    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
-
-  /**
-   * Constructs a {@link GenericSystemDescriptor} instance with no system level serde.
-   * Serdes must be provided explicitly at stream level when getting input or output descriptors.
-   *
-   * @param systemName name of this system
-   * @param factoryClassName name of the SystemFactory class for this system
-   */
-  public GenericSystemDescriptor(String systemName, String factoryClassName) {
-    super(systemName, factoryClassName, null, null);
-  }
-
-  @Override
-  public <StreamMessageType> GenericInputDescriptor<StreamMessageType> getInputDescriptor(
-      String streamId, Serde<StreamMessageType> serde) {
-    return new GenericInputDescriptor<>(streamId, this, serde);
-  }
-
-  @Override
-  public <StreamMessageType> GenericOutputDescriptor<StreamMessageType> getOutputDescriptor(
-      String streamId, Serde<StreamMessageType> serde) {
-    return new GenericOutputDescriptor<>(streamId, this, serde);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/InputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/InputDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/InputDescriptor.java
deleted file mode 100644
index 708dd2a..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/InputDescriptor.java
+++ /dev/null
@@ -1,183 +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.descriptors.base.stream;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.system.SystemStreamMetadata.OffsetType;
-
-/**
- * The base descriptor for an input stream. Allows setting properties that are common to all input streams.
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- * @param <SubClass> type of the concrete sub-class
- */
-public abstract class InputDescriptor<StreamMessageType, SubClass extends InputDescriptor<StreamMessageType, SubClass>>
-    extends StreamDescriptor<StreamMessageType, SubClass> {
-  private static final String RESET_OFFSET_CONFIG_KEY = "streams.%s.samza.reset.offset";
-  private static final String OFFSET_DEFAULT_CONFIG_KEY = "streams.%s.samza.offset.default";
-  private static final String PRIORITY_CONFIG_KEY = "streams.%s.samza.priority";
-  private static final String BOOTSTRAP_CONFIG_KEY = "streams.%s.samza.bootstrap";
-  private static final String BOUNDED_CONFIG_KEY = "streams.%s.samza.bounded";
-  private static final String DELETE_COMMITTED_MESSAGES_CONFIG_KEY = "streams.%s.samza.delete.committed.messages";
-
-  private final Optional<InputTransformer> transformerOptional;
-
-  private Optional<Boolean> resetOffsetOptional = Optional.empty();
-  private Optional<OffsetType> offsetDefaultOptional = Optional.empty();
-  private Optional<Integer> priorityOptional = Optional.empty();
-  private Optional<Boolean> isBootstrapOptional = Optional.empty();
-  private Optional<Boolean> isBoundedOptional = Optional.empty();
-  private Optional<Boolean> deleteCommittedMessagesOptional = Optional.empty();
-
-  /**
-   * Constructs an {@link InputDescriptor} instance.
-   *
-   * @param streamId id of the stream
-   * @param serde serde for messages in the stream
-   * @param systemDescriptor system descriptor this stream descriptor was obtained from
-   * @param transformer stream level input stream transform function if available, else null
-   */
-  public InputDescriptor(String streamId, Serde serde, SystemDescriptor systemDescriptor, InputTransformer transformer) {
-    super(streamId, serde, systemDescriptor);
-
-    // stream level transformer takes precedence over system level transformer
-    if (transformer != null) {
-      this.transformerOptional = Optional.of(transformer);
-    } else {
-      this.transformerOptional = systemDescriptor.getTransformer();
-    }
-  }
-
-  /**
-   * If set, when a Samza container starts up, it ignores any checkpointed offset for this particular
-   * input stream. Its behavior is thus determined by the {@link #withOffsetDefault} setting.
-   * Note that the reset takes effect every time a container is started, which may be every time you restart your job,
-   * or more frequently if a container fails and is restarted by the framework.
-   *
-   * @return this input descriptor
-   */
-  public SubClass shouldResetOffset() {
-    this.resetOffsetOptional = Optional.of(true);
-    return (SubClass) this;
-  }
-
-  /**
-   * If a container starts up without a checkpoint, this property determines where in the input stream we should start
-   * consuming. The value must be an OffsetType, one of the following:
-   * <ul>
-   *  <li>upcoming: Start processing messages that are published after the job starts.
-   *                Any messages published while the job was not running are not processed.
-   *  <li>oldest: Start processing at the oldest available message in the system,
-   *              and reprocess the entire available message history.
-   * </ul>
-   * This property is for an individual stream. To set it for all streams within a system, see
-   * {@link SystemDescriptor#withDefaultStreamOffsetDefault}. If both are defined, the stream-level definition
-   * takes precedence.
-   *
-   * @param offsetDefault offset type to start processing from
-   * @return this input descriptor
-   */
-  public SubClass withOffsetDefault(OffsetType offsetDefault) {
-    this.offsetDefaultOptional = Optional.ofNullable(offsetDefault);
-    return (SubClass) this;
-  }
-
-  /**
-   * If one or more streams have a priority set (any positive integer), they will be processed with higher priority
-   * than the other streams.
-   * <p>
-   * You can set several streams to the same priority, or define multiple priority levels by assigning a
-   * higher number to the higher-priority streams.
-   * <p>
-   * If a higher-priority stream has any messages available, they will always be processed first;
-   * messages from lower-priority streams are only processed when there are no new messages on higher-priority inputs.
-   *
-   * @param priority priority for this input stream
-   * @return this input descriptor
-   */
-  public SubClass withPriority(int priority) {
-    this.priorityOptional = Optional.of(priority);
-    return (SubClass) this;
-  }
-
-  /**
-   * If set, this stream will be processed as a bootstrap stream. This means that every time a Samza container
-   * starts up, this stream will be fully consumed before messages from any other stream are processed.
-   *
-   * @return this input descriptor
-   */
-  public SubClass shouldBootstrap() {
-    this.isBootstrapOptional = Optional.of(true);
-    return (SubClass) this;
-  }
-
-  /**
-   * If set, this stream will be considered a bounded stream. If all input streams in an application are
-   * bounded, the job is considered to be running in batch processing mode.
-   *
-   * @return this input descriptor
-   */
-  public SubClass isBounded() {
-    this.isBoundedOptional = Optional.of(true);
-    return (SubClass) this;
-  }
-
-  /**
-   * If set, and supported by the system implementation, messages older than the latest checkpointed offset
-   * for this stream may be deleted after the commit.
-   *
-   * @return this input descriptor
-   */
-  public SubClass shouldDeleteCommittedMessages() {
-    this.deleteCommittedMessagesOptional = Optional.of(true);
-    return (SubClass) this;
-  }
-
-  public Optional<InputTransformer> getTransformer() {
-    return this.transformerOptional;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    HashMap<String, String> configs = new HashMap<>(super.toConfig());
-    String streamId = getStreamId();
-    this.offsetDefaultOptional.ifPresent(od ->
-        configs.put(String.format(OFFSET_DEFAULT_CONFIG_KEY, streamId), od.name().toLowerCase()));
-    this.resetOffsetOptional.ifPresent(resetOffset ->
-        configs.put(String.format(RESET_OFFSET_CONFIG_KEY, streamId), Boolean.toString(resetOffset)));
-    this.priorityOptional.ifPresent(priority ->
-        configs.put(String.format(PRIORITY_CONFIG_KEY, streamId), Integer.toString(priority)));
-    this.isBootstrapOptional.ifPresent(bootstrap ->
-        configs.put(String.format(BOOTSTRAP_CONFIG_KEY, streamId), Boolean.toString(bootstrap)));
-    this.isBoundedOptional.ifPresent(bounded ->
-        configs.put(String.format(BOUNDED_CONFIG_KEY, streamId), Boolean.toString(bounded)));
-    this.deleteCommittedMessagesOptional.ifPresent(deleteCommittedMessages ->
-        configs.put(String.format(DELETE_COMMITTED_MESSAGES_CONFIG_KEY, streamId),
-            Boolean.toString(deleteCommittedMessages)));
-    return Collections.unmodifiableMap(configs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/OutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/OutputDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/OutputDescriptor.java
deleted file mode 100644
index 20bbc53..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/OutputDescriptor.java
+++ /dev/null
@@ -1,44 +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.descriptors.base.stream;
-
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * The base descriptor for an output stream. Allows setting properties that are common to all output streams.
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- * @param <SubClass> type of the concrete sub-class
- */
-public abstract class OutputDescriptor<StreamMessageType, SubClass extends OutputDescriptor<StreamMessageType, SubClass>>
-    extends StreamDescriptor<StreamMessageType, SubClass> {
-  /**
-   * Constructs an {@link OutputDescriptor} instance.
-   *
-   * @param streamId id of the stream
-   * @param serde serde for messages in the stream
-   * @param systemDescriptor system descriptor this stream descriptor was obtained from
-   */
-  public OutputDescriptor(String streamId, Serde serde, SystemDescriptor systemDescriptor) {
-    super(streamId, serde, systemDescriptor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/StreamDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/StreamDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/StreamDescriptor.java
deleted file mode 100644
index f7de728..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/stream/StreamDescriptor.java
+++ /dev/null
@@ -1,136 +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.descriptors.base.stream;
-
-import com.google.common.base.Preconditions;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.regex.Pattern;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * The base descriptor for an input or output stream. Allows setting properties that are common to all streams.
- * <p>
- * Stream properties configured using a descriptor override corresponding properties provided in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream.
- * @param <SubClass> type of the concrete sub-class
- */
-public abstract class StreamDescriptor<StreamMessageType, SubClass extends StreamDescriptor<StreamMessageType, SubClass>> {
-  private static final String SYSTEM_CONFIG_KEY = "streams.%s.samza.system";
-  private static final String PHYSICAL_NAME_CONFIG_KEY = "streams.%s.samza.physical.name";
-  private static final String STREAM_CONFIGS_CONFIG_KEY = "streams.%s.%s";
-  private static final Pattern STREAM_ID_PATTERN = Pattern.compile("[\\d\\w-_]+");
-
-  private final String streamId;
-  private final Serde serde;
-  private final SystemDescriptor systemDescriptor;
-
-  private final Map<String, String> streamConfigs = new HashMap<>();
-  private Optional<String> physicalNameOptional = Optional.empty();
-
-  /**
-   * Constructs a {@link StreamDescriptor} instance.
-   *
-   * @param streamId id of the stream
-   * @param serde serde for messages in the stream
-   * @param systemDescriptor system descriptor this stream descriptor was obtained from
-   */
-  StreamDescriptor(String streamId, Serde serde, SystemDescriptor systemDescriptor) {
-    Preconditions.checkArgument(systemDescriptor != null,
-        String.format("SystemDescriptor must not be null. streamId: %s", streamId));
-    String systemName = systemDescriptor.getSystemName();
-    Preconditions.checkState(isValidStreamId(streamId),
-        String.format("streamId must be non-empty and must not contain spaces or special characters. " +
-            "streamId: %s, systemName: %s", streamId, systemName));
-    Preconditions.checkArgument(serde != null,
-        String.format("Serde must not be null. streamId: %s systemName: %s", streamId, systemName));
-    this.streamId = streamId;
-    this.serde = serde;
-    this.systemDescriptor = systemDescriptor;
-  }
-
-  /**
-   * The physical name of the stream on the system on which this stream will be accessed.
-   * This is opposed to the {@code streamId} which is the logical name that Samza uses to identify the stream.
-   * <p>
-   * A physical name could be a Kafka topic name, an HDFS file URN, or any other system-specific identifier.
-   * <p>
-   * If not provided, the logical {@code streamId} is used as the physical name.
-   *
-   * @param physicalName physical name for this stream.
-   * @return this stream descriptor.
-   */
-  protected SubClass withPhysicalName(String physicalName) {
-    this.physicalNameOptional = Optional.ofNullable(physicalName);
-    return (SubClass) this;
-  }
-
-  /**
-   * Additional system-specific properties for this stream.
-   * <p>
-   * These properties are added under the {@code streams.stream-id.*} scope.
-   *
-   * @param streamConfigs system-specific properties for this stream
-   * @return this stream descriptor
-   */
-  public SubClass withStreamConfigs(Map<String, String> streamConfigs) {
-    this.streamConfigs.putAll(streamConfigs);
-    return (SubClass) this;
-  }
-
-  public String getStreamId() {
-    return this.streamId;
-  }
-
-  public String getSystemName() {
-    return this.systemDescriptor.getSystemName();
-  }
-
-  public Serde getSerde() {
-    return this.serde;
-  }
-
-  public SystemDescriptor getSystemDescriptor() {
-    return this.systemDescriptor;
-  }
-
-  public Optional<String> getPhysicalName() {
-    return physicalNameOptional;
-  }
-
-  private boolean isValidStreamId(String id) {
-    return StringUtils.isNotBlank(id) && STREAM_ID_PATTERN.matcher(id).matches();
-  }
-
-  public Map<String, String> toConfig() {
-    HashMap<String, String> configs = new HashMap<>();
-    configs.put(String.format(SYSTEM_CONFIG_KEY, streamId), getSystemName());
-    this.physicalNameOptional.ifPresent(physicalName ->
-        configs.put(String.format(PHYSICAL_NAME_CONFIG_KEY, streamId), physicalName));
-    this.streamConfigs.forEach((key, value) ->
-        configs.put(String.format(STREAM_CONFIGS_CONFIG_KEY, streamId, key), value));
-    return Collections.unmodifiableMap(configs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/ExpandingInputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/ExpandingInputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/ExpandingInputDescriptorProvider.java
deleted file mode 100644
index 05179dd..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/ExpandingInputDescriptorProvider.java
+++ /dev/null
@@ -1,44 +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.descriptors.base.system;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * Interface for advanced {@code SystemDescriptor}s that constrain the type of returned {@code InputDescriptor}s to
- * their own {@code StreamExpander} function result types.
- *
- * @param <StreamExpanderType> type of the system level {@code StreamExpander} results
- */
-public interface ExpandingInputDescriptorProvider<StreamExpanderType> {
-
-  /**
-   * Gets a {@link InputDescriptor} for an input stream on this system. The stream has the provided
-   * stream level serde, and the default system level {@code StreamExpander}
-   * <p>
-   * The type of messages in the stream is the type of messages returned by the default system level
-   * {@code StreamExpander}
-   *
-   * @param streamId id of the input stream
-   * @param serde stream level serde to be propagated to expanded input streams
-   * @return an {@link InputDescriptor} for the input stream
-   */
-  InputDescriptor<StreamExpanderType, ? extends InputDescriptor> getInputDescriptor(String streamId, Serde serde);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/OutputDescriptorProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/OutputDescriptorProvider.java b/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/OutputDescriptorProvider.java
deleted file mode 100644
index c2ceb53..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/descriptors/base/system/OutputDescriptorProvider.java
+++ /dev/null
@@ -1,48 +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.descriptors.base.system;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.serializers.Serde;
-
-
-/**
- * Interface for simple {@code SystemDescriptors} that return {@code OutputDescriptors} parameterized by the type of
- * the provided stream level serde.
- */
-public interface OutputDescriptorProvider {
-
-  /**
-   * Gets an {@link OutputDescriptor} representing an output stream on this system that uses the provided
-   * stream specific serde instead of the default system serde.
-   * <p>
-   * An {@code OutputStream<KV<K, V>>}, obtained using a descriptor with a {@code KVSerde<K, V>}, can send messages
-   * of type {@code KV<K, V>}. An {@code OutputStream<M>} with any other {@code Serde<M>} can send messages of
-   * type M without a key.
-   * <p>
-   * A {@code KVSerde<NoOpSerde, NoOpSerde>} or {@code NoOpSerde} may be used if the {@code SystemProducer}
-   * serializes the outgoing messages itself, and no prior serialization is required from the framework.
-   *
-   * @param streamId id of the output stream
-   * @param serde serde for this output stream that overrides the default system serde, if any.
-   * @param <StreamMessageType> type of messages in the output stream
-   * @return the {@link OutputDescriptor} for the output stream
-   */
-  <StreamMessageType> OutputDescriptor<StreamMessageType, ? extends OutputDescriptor> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde);
-}


[08/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java
----------------------------------------------------------------------
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 e9e2635..2b29a2b 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,6 +18,7 @@
  */
 package org.apache.samza.application;
 
+import org.apache.samza.application.descriptors.TaskApplicationDescriptor;
 import org.apache.samza.task.TaskFactoryUtil;
 
 /**

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 3bc3ed5..0000000
--- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java
+++ /dev/null
@@ -1,366 +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 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.List;
-import java.util.Map;
-import java.util.Optional;
-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.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.InputTransformer;
-import org.apache.samza.operators.functions.StreamExpander;
-import org.apache.samza.operators.spec.InputOperatorSpec;
-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;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.hybrid.BaseHybridTableDescriptor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class defines:
- * 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 StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl<StreamApplicationDescriptor>
-    implements StreamApplicationDescriptor {
-  private static final Logger LOGGER = LoggerFactory.getLogger(StreamApplicationDescriptorImpl.class);
-  private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_]+");
-
-  private final Map<String, InputDescriptor> inputDescriptors = new LinkedHashMap<>();
-  private final Map<String, OutputDescriptor> outputDescriptors = new LinkedHashMap<>();
-  private final Set<String> broadcastStreams = new HashSet<>();
-  private final Map<String, TableDescriptor> tableDescriptors = new LinkedHashMap<>();
-  private final Map<String, SystemDescriptor> systemDescriptors = new LinkedHashMap<>();
-  // We use a LHM for deterministic order in initializing and closing operators.
-  private final Map<String, InputOperatorSpec> inputOperators = new LinkedHashMap<>();
-  private final Map<String, OutputStreamImpl> outputStreams = new LinkedHashMap<>();
-  private final Map<String, TableImpl> tables = new LinkedHashMap<>();
-  private final Set<String> operatorIds = new HashSet<>();
-
-  private Optional<SystemDescriptor> defaultSystemDescriptorOptional = Optional.empty();
-
-  /**
-   * 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)}.
-   */
-  private int nextOpNum = 0;
-
-  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 <M> MessageStream<M> getInputStream(InputDescriptor<M, ?> inputDescriptor) {
-    SystemDescriptor systemDescriptor = inputDescriptor.getSystemDescriptor();
-    Optional<StreamExpander> expander = systemDescriptor.getExpander();
-    if (expander.isPresent()) {
-      return expander.get().apply(this, 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()));
-    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);
-
-    Serde serde = inputDescriptor.getSerde();
-    KV<Serde, Serde> kvSerdes = getOrCreateStreamSerdes(streamId, serde);
-    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),
-          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
-              + "stream level, so the same key and message Serde must be used for both.", streamId));
-    }
-
-    boolean isKeyed = serde instanceof KVSerde;
-    InputTransformer transformer = inputDescriptor.getTransformer().orElse(null);
-    InputOperatorSpec inputOperatorSpec =
-        OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(),
-            transformer, isKeyed, this.getNextOpId(OpCode.INPUT, null));
-    inputOperators.put(streamId, inputOperatorSpec);
-    return new MessageStreamImpl(this, inputOperators.get(streamId));
-  }
-
-  @Override
-  public <M> OutputStream<M> getOutputStream(OutputDescriptor<M, ?> outputDescriptor) {
-    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);
-
-    Serde serde = outputDescriptor.getSerde();
-    KV<Serde, Serde> kvSerdes = getOrCreateStreamSerdes(streamId, serde);
-    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),
-          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
-              + "stream level, so the same key and message Serde must be used for both.", streamId));
-    }
-
-    boolean isKeyed = serde instanceof KVSerde;
-    outputStreams.put(streamId, new OutputStreamImpl<>(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed));
-    return outputStreams.get(streamId);
-  }
-
-  @Override
-  public <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDescriptor) {
-
-    if (tableDescriptor instanceof BaseHybridTableDescriptor) {
-      List<? extends TableDescriptor<K, V, ?>> tableDescs = ((BaseHybridTableDescriptor) tableDescriptor).getTableDescriptors();
-      tableDescs.forEach(td -> getTable(td));
-    }
-
-    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()));
-    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);
-
-    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.getId(), new TableImpl(tableSpec));
-    getOrCreateTableSerdes(tableSpec.getId(), baseTableDescriptor.getSerde());
-    return tables.get(tableSpec.getId());
-  }
-
-  /**
-   * Get all the {@link InputDescriptor}s to this application
-   *
-   * @return an immutable map of streamId to {@link InputDescriptor}
-   */
-  @Override
-  public Map<String, InputDescriptor> 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<String, OutputDescriptor> getOutputDescriptors() {
-    return Collections.unmodifiableMap(outputDescriptors);
-  }
-
-  /**
-   * Get all the broadcast streamIds from this application
-   *
-   * @return an immutable set of streamIds
-   */
-  @Override
-  public Set<String> 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<TableDescriptor> 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<SystemDescriptor> 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()));
-  }
-
-  @Override
-  public Set<String> getInputStreamIds() {
-    return Collections.unmodifiableSet(new HashSet<>(inputOperators.keySet()));
-  }
-
-  @Override
-  public Set<String> getOutputStreamIds() {
-    return Collections.unmodifiableSet(new HashSet<>(outputStreams.keySet()));
-  }
-
-  /**
-   * Get the default {@link SystemDescriptor} in this application
-   *
-   * @return the default {@link SystemDescriptor}
-   */
-  @Override
-  public Optional<SystemDescriptor> getDefaultSystemDescriptor() {
-    return defaultSystemDescriptorOptional;
-  }
-
-  public OperatorSpecGraph getOperatorSpecGraph() {
-    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 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) {
-    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",
-        config.get(JobConfig.JOB_NAME()),
-        config.get(JobConfig.JOB_ID(), "1"),
-        opCode.name().toLowerCase(),
-        StringUtils.isNotBlank(userDefinedId) ? userDefinedId.trim() : String.valueOf(nextOpNum));
-    if (!operatorIds.add(nextOpId)) {
-      throw new SamzaException(
-          String.format("Found duplicate operator ID %s in the graph. Operator IDs must be unique.", nextOpId));
-    }
-    nextOpNum++;
-    return nextOpId;
-  }
-
-  /**
-   * 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
-   * @return the unique ID for the next operator in the graph
-   */
-  public String getNextOpId(OpCode opCode) {
-    return getNextOpId(opCode, null);
-  }
-
-  public Map<String, InputOperatorSpec> getInputOperators() {
-    return Collections.unmodifiableMap(inputOperators);
-  }
-
-  public Map<String, OutputStreamImpl> getOutputStreams() {
-    return Collections.unmodifiableMap(outputStreams);
-  }
-
-  public Map<String, TableImpl> 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.
-   *
-   * @param streamId the id of the stream to be created.
-   * @param serde the {@link Serde} to use for the message in the intermediate stream. If null, the default serde
-   *              is used.
-   * @param isBroadcast whether the stream is a broadcast stream.
-   * @param <M> the type of messages in the intermediate {@link MessageStream}
-   * @return  the intermediate {@link MessageStreamImpl}
-   */
-  @VisibleForTesting
-  public <M> IntermediateMessageStreamImpl<M> getIntermediateStream(String streamId, Serde<M> serde, boolean isBroadcast) {
-    Preconditions.checkNotNull(serde, "serde must not be null for intermediate stream: " + streamId);
-    Preconditions.checkState(!inputOperators.containsKey(streamId) && !outputStreams.containsKey(streamId),
-        "getIntermediateStream must not be called multiple times with the same streamId: " + streamId);
-
-    if (isBroadcast) {
-      broadcastStreams.add(streamId);
-    }
-
-    boolean isKeyed = serde instanceof KVSerde;
-    KV<Serde, Serde> kvSerdes = getOrCreateStreamSerdes(streamId, serde);
-
-    InputTransformer transformer = (InputTransformer) getDefaultSystemDescriptor()
-        .flatMap(SystemDescriptor::getTransformer).orElse(null);
-
-    InputOperatorSpec inputOperatorSpec =
-        OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(),
-            transformer, 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));
-  }
-
-  // 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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index b4fde1e..0000000
--- a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java
+++ /dev/null
@@ -1,143 +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 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.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.task.TaskFactory;
-
-
-/**
- * This class implements interface {@link TaskApplicationDescriptor}.
- * <p>
- * 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 TaskApplicationDescriptorImpl extends ApplicationDescriptorImpl<TaskApplicationDescriptor>
-    implements TaskApplicationDescriptor {
-
-  private final Map<String, InputDescriptor> inputDescriptors = new LinkedHashMap<>();
-  private final Map<String, OutputDescriptor> outputDescriptors = new LinkedHashMap<>();
-  private final Set<String> broadcastStreams = new HashSet<>();
-  private final Map<String, TableDescriptor> tableDescriptors = new LinkedHashMap<>();
-  private final Map<String, SystemDescriptor> systemDescriptors = new LinkedHashMap<>();
-
-  private TaskFactory taskFactory = null;
-
-  public TaskApplicationDescriptorImpl(TaskApplication userApp, Config config) {
-    super(userApp, config);
-    userApp.describe(this);
-  }
-
-  @Override
-  public void setTaskFactory(TaskFactory factory) {
-    this.taskFactory = factory;
-  }
-
-  @Override
-  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());
-  }
-
-  @Override
-  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());
-  }
-
-  @Override
-  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);
-  }
-
-  @Override
-  public Map<String, InputDescriptor> getInputDescriptors() {
-    return Collections.unmodifiableMap(inputDescriptors);
-  }
-
-  @Override
-  public Map<String, OutputDescriptor> getOutputDescriptors() {
-    return Collections.unmodifiableMap(outputDescriptors);
-  }
-
-  @Override
-  public Set<String> getBroadcastStreams() {
-    return Collections.unmodifiableSet(broadcastStreams);
-  }
-
-  @Override
-  public Set<TableDescriptor> getTableDescriptors() {
-    return Collections.unmodifiableSet(new HashSet<>(tableDescriptors.values()));
-  }
-
-  @Override
-  public Set<SystemDescriptor> 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()));
-  }
-
-  @Override
-  public Set<String> getInputStreamIds() {
-    return Collections.unmodifiableSet(new HashSet<>(inputDescriptors.keySet()));
-  }
-
-  @Override
-  public Set<String> getOutputStreamIds() {
-    return Collections.unmodifiableSet(new HashSet<>(outputDescriptors.keySet()));
-  }
-
-  /**
-   * Get the user-defined {@link TaskFactory}
-   * @return the {@link TaskFactory} object
-   */
-  public TaskFactory getTaskFactory() {
-    return taskFactory;
-  }
-
-  // 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);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorImpl.java
new file mode 100644
index 0000000..f3c34a9
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorImpl.java
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.samza.application.SamzaApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.context.ApplicationContainerContext;
+import org.apache.samza.context.ApplicationContainerContextFactory;
+import org.apache.samza.context.ApplicationTaskContext;
+import org.apache.samza.context.ApplicationTaskContextFactory;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.metrics.MetricsReporterFactory;
+import org.apache.samza.operators.KV;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is the base class that implements interface {@link ApplicationDescriptor}.
+ * <p>
+ * This base class contains the common objects that are used by both high-level and low-level API applications, such as
+ * {@link Config}, {@link ApplicationContainerContextFactory}, {@link ApplicationTaskContextFactory}, and
+ * {@link ProcessorLifecycleListenerFactory}.
+ *
+ * @param <S> the type of {@link ApplicationDescriptor} interface this implements. It has to be either
+ *            {@link StreamApplicationDescriptor} or {@link TaskApplicationDescriptor}
+ */
+public abstract class ApplicationDescriptorImpl<S extends ApplicationDescriptor>
+    implements ApplicationDescriptor<S> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ApplicationDescriptorImpl.class);
+
+  private final Class<? extends SamzaApplication> appClass;
+  private final Map<String, MetricsReporterFactory> reporterFactories = new LinkedHashMap<>();
+  // serdes used by input/output/intermediate streams, keyed by streamId
+  private final Map<String, KV<Serde, Serde>> streamSerdes = new HashMap<>();
+  // serdes used by tables, keyed by tableId
+  private final Map<String, KV<Serde, Serde>> tableSerdes = new HashMap<>();
+  final Config config;
+
+  private Optional<ApplicationContainerContextFactory<?>> applicationContainerContextFactoryOptional = Optional.empty();
+  private Optional<ApplicationTaskContextFactory<?>> applicationTaskContextFactoryOptional = Optional.empty();
+
+  // Default to no-op  ProcessorLifecycleListenerFactory
+  ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { };
+
+  ApplicationDescriptorImpl(SamzaApplication app, Config config) {
+    this.config = config;
+    this.appClass = app.getClass();
+  }
+
+  @Override
+  public Config getConfig() {
+    return config;
+  }
+
+  @Override
+  public S withApplicationContainerContextFactory(ApplicationContainerContextFactory<?> factory) {
+    this.applicationContainerContextFactoryOptional = Optional.of(factory);
+    return (S) this;
+  }
+
+  @Override
+  public S withApplicationTaskContextFactory(ApplicationTaskContextFactory<?> factory) {
+    this.applicationTaskContextFactoryOptional = Optional.of(factory);
+    return (S) this;
+  }
+
+  @Override
+  public S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory) {
+    this.listenerFactory = listenerFactory;
+    return (S) this;
+  }
+
+  @Override
+  public S withMetricsReporterFactories(Map<String, MetricsReporterFactory> reporterFactories) {
+    this.reporterFactories.clear();
+    this.reporterFactories.putAll(reporterFactories);
+    return (S) this;
+  }
+
+  /**
+   * Get the application class
+   *
+   * @return an implementation of {@link SamzaApplication}
+   */
+  public Class<? extends SamzaApplication> getAppClass() {
+    return appClass;
+  }
+
+  /**
+   * Get the {@link ApplicationContainerContextFactory} specified by the application.
+   *
+   * @return {@link ApplicationContainerContextFactory} if application specified it; empty otherwise
+   */
+  public Optional<ApplicationContainerContextFactory<ApplicationContainerContext>> getApplicationContainerContextFactory() {
+    @SuppressWarnings("unchecked") // ok because all context types are at least ApplicationContainerContext
+    Optional<ApplicationContainerContextFactory<ApplicationContainerContext>> factoryOptional =
+        (Optional) this.applicationContainerContextFactoryOptional;
+    return factoryOptional;
+  }
+
+  /**
+   * Get the {@link ApplicationTaskContextFactory} specified by the application.
+   *
+   * @return {@link ApplicationTaskContextFactory} if application specified it; empty otherwise
+   */
+  public Optional<ApplicationTaskContextFactory<ApplicationTaskContext>> getApplicationTaskContextFactory() {
+    @SuppressWarnings("unchecked") // ok because all context types are at least ApplicationTaskContext
+    Optional<ApplicationTaskContextFactory<ApplicationTaskContext>> factoryOptional =
+        (Optional) this.applicationTaskContextFactoryOptional;
+    return factoryOptional;
+  }
+
+  /**
+   * Get the {@link ProcessorLifecycleListenerFactory} associated with this application
+   *
+   * @return the {@link ProcessorLifecycleListenerFactory} in this application
+   */
+  public ProcessorLifecycleListenerFactory getProcessorLifecycleListenerFactory() {
+    return listenerFactory;
+  }
+
+  /**
+   * Get the {@link MetricsReporterFactory}s used in the application
+   *
+   * @return the map of {@link MetricsReporterFactory}s
+   */
+  public Map<String, MetricsReporterFactory> getMetricsReporterFactories() {
+    return Collections.unmodifiableMap(reporterFactories);
+  }
+
+  /**
+   * Get the default {@link SystemDescriptor} in this application
+   *
+   * @return the default {@link SystemDescriptor}
+   */
+  public Optional<SystemDescriptor> getDefaultSystemDescriptor() {
+    // default is not set
+    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<Serde, Serde> 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<Serde, Serde> 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 org.apache.samza.application.TaskApplication}
+   */
+  public Map<String, InputOperatorSpec> getInputOperators() {
+    return Collections.EMPTY_MAP;
+  }
+
+  /**
+   * Get all the {@link InputDescriptor}s to this application
+   *
+   * @return an immutable map of streamId to {@link InputDescriptor}
+   */
+  public abstract Map<String, InputDescriptor> getInputDescriptors();
+
+  /**
+   * Get all the {@link OutputDescriptor}s from this application
+   *
+   * @return an immutable map of streamId to {@link OutputDescriptor}
+   */
+  public abstract Map<String, OutputDescriptor> getOutputDescriptors();
+
+  /**
+   * Get all the broadcast streamIds from this application
+   *
+   * @return an immutable set of streamIds
+   */
+  public abstract Set<String> getBroadcastStreams();
+
+  /**
+   * Get all the {@link TableDescriptor}s in this application
+   *
+   * @return an immutable set of {@link TableDescriptor}s
+   */
+  public abstract Set<TableDescriptor> getTableDescriptors();
+
+  /**
+   * Get all the unique {@link SystemDescriptor}s in this application
+   *
+   * @return an immutable set of {@link SystemDescriptor}s
+   */
+  public abstract Set<SystemDescriptor> getSystemDescriptors();
+
+  /**
+   * Get all the unique input streamIds in this application
+   *
+   * @return an immutable set of input streamIds
+   */
+  public abstract Set<String> getInputStreamIds();
+
+  /**
+   * Get all the unique output streamIds in this application
+   *
+   * @return an immutable set of output streamIds
+   */
+  public abstract Set<String> getOutputStreamIds();
+
+  KV<Serde, Serde> getOrCreateStreamSerdes(String streamId, Serde serde) {
+    Serde keySerde, valueSerde;
+
+    KV<Serde, Serde> currentSerdePair = streamSerdes.get(streamId);
+
+    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);
+  }
+
+  KV<Serde, Serde> 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<Serde, Serde> 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

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorUtil.java b/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorUtil.java
new file mode 100644
index 0000000..e3c2d5c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptorUtil.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.descriptors;
+
+import org.apache.samza.application.SamzaApplication;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.application.TaskApplication;
+import org.apache.samza.config.Config;
+
+
+/**
+ * Util class to help creating {@link ApplicationDescriptorImpl} instance from {@link SamzaApplication} and {@link Config}
+ */
+public class ApplicationDescriptorUtil {
+
+  private ApplicationDescriptorUtil() {
+
+  }
+
+  /**
+   * Create a new instance of {@link ApplicationDescriptorImpl} based on {@link SamzaApplication} and {@link 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<? extends ApplicationDescriptor> getAppDescriptor(SamzaApplication app, Config config) {
+    if (app instanceof StreamApplication) {
+      return new StreamApplicationDescriptorImpl((StreamApplication) app, 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.", app.getClass().getName()));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptorImpl.java
new file mode 100644
index 0000000..ec36cf3
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptorImpl.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+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.List;
+import java.util.Map;
+import java.util.Optional;
+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.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.table.descriptors.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.table.descriptors.TableDescriptor;
+import org.apache.samza.operators.TableImpl;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.system.descriptors.StreamExpander;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+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;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.descriptors.BaseHybridTableDescriptor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class defines:
+ * 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 StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl<StreamApplicationDescriptor>
+    implements StreamApplicationDescriptor {
+  private static final Logger LOGGER = LoggerFactory.getLogger(StreamApplicationDescriptorImpl.class);
+  private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_]+");
+
+  private final Map<String, InputDescriptor> inputDescriptors = new LinkedHashMap<>();
+  private final Map<String, OutputDescriptor> outputDescriptors = new LinkedHashMap<>();
+  private final Set<String> broadcastStreams = new HashSet<>();
+  private final Map<String, TableDescriptor> tableDescriptors = new LinkedHashMap<>();
+  private final Map<String, SystemDescriptor> systemDescriptors = new LinkedHashMap<>();
+  // We use a LHM for deterministic order in initializing and closing operators.
+  private final Map<String, InputOperatorSpec> inputOperators = new LinkedHashMap<>();
+  private final Map<String, OutputStreamImpl> outputStreams = new LinkedHashMap<>();
+  private final Map<String, TableImpl> tables = new LinkedHashMap<>();
+  private final Set<String> operatorIds = new HashSet<>();
+
+  private Optional<SystemDescriptor> defaultSystemDescriptorOptional = Optional.empty();
+
+  /**
+   * 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)}.
+   */
+  private int nextOpNum = 0;
+
+  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 <M> MessageStream<M> getInputStream(InputDescriptor<M, ?> inputDescriptor) {
+    SystemDescriptor systemDescriptor = inputDescriptor.getSystemDescriptor();
+    Optional<StreamExpander> expander = systemDescriptor.getExpander();
+    if (expander.isPresent()) {
+      return expander.get().apply(this, 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()));
+    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);
+
+    Serde serde = inputDescriptor.getSerde();
+    KV<Serde, Serde> kvSerdes = getOrCreateStreamSerdes(streamId, serde);
+    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),
+          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
+              + "stream level, so the same key and message Serde must be used for both.", streamId));
+    }
+
+    boolean isKeyed = serde instanceof KVSerde;
+    InputTransformer transformer = inputDescriptor.getTransformer().orElse(null);
+    InputOperatorSpec inputOperatorSpec =
+        OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(),
+            transformer, isKeyed, this.getNextOpId(OpCode.INPUT, null));
+    inputOperators.put(streamId, inputOperatorSpec);
+    return new MessageStreamImpl(this, inputOperators.get(streamId));
+  }
+
+  @Override
+  public <M> OutputStream<M> getOutputStream(OutputDescriptor<M, ?> outputDescriptor) {
+    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);
+
+    Serde serde = outputDescriptor.getSerde();
+    KV<Serde, Serde> kvSerdes = getOrCreateStreamSerdes(streamId, serde);
+    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),
+          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
+              + "stream level, so the same key and message Serde must be used for both.", streamId));
+    }
+
+    boolean isKeyed = serde instanceof KVSerde;
+    outputStreams.put(streamId, new OutputStreamImpl<>(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed));
+    return outputStreams.get(streamId);
+  }
+
+  @Override
+  public <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDescriptor) {
+
+    if (tableDescriptor instanceof BaseHybridTableDescriptor) {
+      List<? extends TableDescriptor<K, V, ?>> tableDescs = ((BaseHybridTableDescriptor) tableDescriptor).getTableDescriptors();
+      tableDescs.forEach(td -> getTable(td));
+    }
+
+    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()));
+    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);
+
+    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.getId(), new TableImpl(tableSpec));
+    getOrCreateTableSerdes(tableSpec.getId(), baseTableDescriptor.getSerde());
+    return tables.get(tableSpec.getId());
+  }
+
+  /**
+   * Get all the {@link InputDescriptor}s to this application
+   *
+   * @return an immutable map of streamId to {@link InputDescriptor}
+   */
+  @Override
+  public Map<String, InputDescriptor> 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<String, OutputDescriptor> getOutputDescriptors() {
+    return Collections.unmodifiableMap(outputDescriptors);
+  }
+
+  /**
+   * Get all the broadcast streamIds from this application
+   *
+   * @return an immutable set of streamIds
+   */
+  @Override
+  public Set<String> 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<TableDescriptor> 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<SystemDescriptor> 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()));
+  }
+
+  @Override
+  public Set<String> getInputStreamIds() {
+    return Collections.unmodifiableSet(new HashSet<>(inputOperators.keySet()));
+  }
+
+  @Override
+  public Set<String> getOutputStreamIds() {
+    return Collections.unmodifiableSet(new HashSet<>(outputStreams.keySet()));
+  }
+
+  /**
+   * Get the default {@link SystemDescriptor} in this application
+   *
+   * @return the default {@link SystemDescriptor}
+   */
+  @Override
+  public Optional<SystemDescriptor> getDefaultSystemDescriptor() {
+    return defaultSystemDescriptorOptional;
+  }
+
+  public OperatorSpecGraph getOperatorSpecGraph() {
+    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 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) {
+    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",
+        config.get(JobConfig.JOB_NAME()),
+        config.get(JobConfig.JOB_ID(), "1"),
+        opCode.name().toLowerCase(),
+        StringUtils.isNotBlank(userDefinedId) ? userDefinedId.trim() : String.valueOf(nextOpNum));
+    if (!operatorIds.add(nextOpId)) {
+      throw new SamzaException(
+          String.format("Found duplicate operator ID %s in the graph. Operator IDs must be unique.", nextOpId));
+    }
+    nextOpNum++;
+    return nextOpId;
+  }
+
+  /**
+   * 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
+   * @return the unique ID for the next operator in the graph
+   */
+  public String getNextOpId(OpCode opCode) {
+    return getNextOpId(opCode, null);
+  }
+
+  public Map<String, InputOperatorSpec> getInputOperators() {
+    return Collections.unmodifiableMap(inputOperators);
+  }
+
+  public Map<String, OutputStreamImpl> getOutputStreams() {
+    return Collections.unmodifiableMap(outputStreams);
+  }
+
+  public Map<String, TableImpl> 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.
+   *
+   * @param streamId the id of the stream to be created.
+   * @param serde the {@link Serde} to use for the message in the intermediate stream. If null, the default serde
+   *              is used.
+   * @param isBroadcast whether the stream is a broadcast stream.
+   * @param <M> the type of messages in the intermediate {@link MessageStream}
+   * @return  the intermediate {@link MessageStreamImpl}
+   */
+  @VisibleForTesting
+  public <M> IntermediateMessageStreamImpl<M> getIntermediateStream(String streamId, Serde<M> serde, boolean isBroadcast) {
+    Preconditions.checkNotNull(serde, "serde must not be null for intermediate stream: " + streamId);
+    Preconditions.checkState(!inputOperators.containsKey(streamId) && !outputStreams.containsKey(streamId),
+        "getIntermediateStream must not be called multiple times with the same streamId: " + streamId);
+
+    if (isBroadcast) {
+      broadcastStreams.add(streamId);
+    }
+
+    boolean isKeyed = serde instanceof KVSerde;
+    KV<Serde, Serde> kvSerdes = getOrCreateStreamSerdes(streamId, serde);
+
+    InputTransformer transformer = (InputTransformer) getDefaultSystemDescriptor()
+        .flatMap(SystemDescriptor::getTransformer).orElse(null);
+
+    InputOperatorSpec inputOperatorSpec =
+        OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(),
+            transformer, 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));
+  }
+
+  // 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptorImpl.java
new file mode 100644
index 0000000..cb924ab
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptorImpl.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.application.descriptors;
+
+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.application.TaskApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.task.TaskFactory;
+
+
+/**
+ * This class implements interface {@link TaskApplicationDescriptor}.
+ * <p>
+ * 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 TaskApplicationDescriptorImpl extends ApplicationDescriptorImpl<TaskApplicationDescriptor>
+    implements TaskApplicationDescriptor {
+
+  private final Map<String, InputDescriptor> inputDescriptors = new LinkedHashMap<>();
+  private final Map<String, OutputDescriptor> outputDescriptors = new LinkedHashMap<>();
+  private final Set<String> broadcastStreams = new HashSet<>();
+  private final Map<String, TableDescriptor> tableDescriptors = new LinkedHashMap<>();
+  private final Map<String, SystemDescriptor> systemDescriptors = new LinkedHashMap<>();
+
+  private TaskFactory taskFactory = null;
+
+  public TaskApplicationDescriptorImpl(TaskApplication userApp, Config config) {
+    super(userApp, config);
+    userApp.describe(this);
+  }
+
+  @Override
+  public void setTaskFactory(TaskFactory factory) {
+    this.taskFactory = factory;
+  }
+
+  @Override
+  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());
+  }
+
+  @Override
+  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());
+  }
+
+  @Override
+  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);
+  }
+
+  @Override
+  public Map<String, InputDescriptor> getInputDescriptors() {
+    return Collections.unmodifiableMap(inputDescriptors);
+  }
+
+  @Override
+  public Map<String, OutputDescriptor> getOutputDescriptors() {
+    return Collections.unmodifiableMap(outputDescriptors);
+  }
+
+  @Override
+  public Set<String> getBroadcastStreams() {
+    return Collections.unmodifiableSet(broadcastStreams);
+  }
+
+  @Override
+  public Set<TableDescriptor> getTableDescriptors() {
+    return Collections.unmodifiableSet(new HashSet<>(tableDescriptors.values()));
+  }
+
+  @Override
+  public Set<SystemDescriptor> 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()));
+  }
+
+  @Override
+  public Set<String> getInputStreamIds() {
+    return Collections.unmodifiableSet(new HashSet<>(inputDescriptors.keySet()));
+  }
+
+  @Override
+  public Set<String> getOutputStreamIds() {
+    return Collections.unmodifiableSet(new HashSet<>(outputDescriptors.keySet()));
+  }
+
+  /**
+   * Get the user-defined {@link TaskFactory}
+   * @return the {@link TaskFactory} object
+   */
+  public TaskFactory getTaskFactory() {
+    return taskFactory;
+  }
+
+  // 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);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java b/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java
index ed013c4..06f5606 100644
--- a/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java
@@ -59,9 +59,9 @@ public class JavaTableConfig extends MapConfig {
   }
 
   /**
-   * Get the {@link org.apache.samza.table.TableProviderFactory} class for a table
+   * Get the {@link org.apache.samza.table.descriptors.TableProviderFactory} class for a table
    * @param tableId Id of the table
-   * @return the {@link org.apache.samza.table.TableProviderFactory} class name
+   * @return the {@link org.apache.samza.table.descriptors.TableProviderFactory} class name
    */
   public String getTableProviderFactory(String tableId) {
     return get(String.format(TABLE_PROVIDER_FACTORY, tableId), null);

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.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 b80f7df..10a4215 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
@@ -35,15 +35,15 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.commons.collections4.ListUtils;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.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;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.StreamConfig;
-import org.apache.samza.operators.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
----------------------------------------------------------------------
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 f43b24e..6d9faf3 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
@@ -30,8 +30,8 @@ import java.util.Map;
 import java.util.Queue;
 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.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java
----------------------------------------------------------------------
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 18705e4..4a2a235 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
@@ -200,8 +200,8 @@ import org.codehaus.jackson.map.ObjectMapper;
   }
 
   /**
-   * Create JSON POJO for a {@link JobNode}, including the {@link org.apache.samza.application.ApplicationDescriptorImpl}
-   * for this job
+   * Create JSON POJO for a {@link JobNode}, including the
+   * {@link org.apache.samza.application.descriptors.ApplicationDescriptorImpl} for this job
    *
    * @param jobNode job node in the {@link JobGraph}
    * @return {@link org.apache.samza.execution.JobGraphJsonGenerator.JobNodeJson}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
----------------------------------------------------------------------
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 af556f5..82b4178 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
@@ -26,8 +26,8 @@ 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.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.application.LegacyTaskApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java
----------------------------------------------------------------------
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 83f3f61..dc0fc59 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
@@ -26,8 +26,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.application.LegacyTaskApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java
----------------------------------------------------------------------
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 86aca0f..6ca5f3d 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,8 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java
----------------------------------------------------------------------
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 54f86d5..13b29df 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,8 @@ package org.apache.samza.execution;
 import java.util.List;
 import java.util.UUID;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index dd47af2..0000000
--- a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java
+++ /dev/null
@@ -1,110 +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 java.util.HashMap;
-import java.util.Map;
-
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.table.TableSpec;
-
-
-/**
- * Base class for all table descriptor implementations.
- *
- * @param <K> the type of the key in this table
- * @param <V> the type of the value in this table
- * @param <D> the type of the concrete table descriptor
- */
-abstract public class BaseTableDescriptor<K, V, D extends BaseTableDescriptor<K, V, D>>
-    implements TableDescriptor<K, V, D> {
-
-  protected final String tableId;
-
-  protected KVSerde<K, V> serde = KVSerde.of(new NoOpSerde(), new NoOpSerde());
-
-  protected final Map<String, String> config = new HashMap<>();
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   */
-  protected BaseTableDescriptor(String tableId) {
-    this.tableId = tableId;
-  }
-
-  /**
-   * Constructs a table descriptor instance
-   * @param tableId Id of the table, it must conform to pattern {@literal [\\d\\w-_]+}
-   * @param serde the serde for key and value
-   */
-  protected BaseTableDescriptor(String tableId, KVSerde<K, V> serde) {
-    this.tableId = tableId;
-    this.serde = serde;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public D withConfig(String key, String value) {
-    config.put(key, value);
-    return (D) this;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public String getTableId() {
-    return tableId;
-  }
-
-  /**
-   * Get the serde assigned to this {@link TableDescriptor}
-   *
-   * @return {@link KVSerde} used by this table
-   */
-  public KVSerde<K, V> getSerde() {
-    return serde;
-  }
-
-  /**
-   * Generate config for {@link TableSpec}; this method is used internally.
-   * @param tableSpecConfig configuration for the {@link TableSpec}
-   */
-  protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
-    tableSpecConfig.putAll(config);
-  }
-
-  /**
-   * Validate that this table descriptor is constructed properly; this method is used internally.
-   */
-  protected void validate() {
-  }
-
-  /**
-   * Create a {@link TableSpec} from this table descriptor; this method is used internally.
-   *
-   * @return the {@link TableSpec}
-   */
-  abstract public TableSpec getTableSpec();
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
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 09e4868..0f43c5e 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.time.Duration;
 import java.util.Collection;
 
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.functions.JoinFunction;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java
----------------------------------------------------------------------
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 5329fd7..a83739d 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.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/descriptors/DelegatingSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/descriptors/DelegatingSystemDescriptor.java b/samza-core/src/main/java/org/apache/samza/operators/descriptors/DelegatingSystemDescriptor.java
deleted file mode 100644
index 6c4ae49..0000000
--- a/samza-core/src/main/java/org/apache/samza/operators/descriptors/DelegatingSystemDescriptor.java
+++ /dev/null
@@ -1,64 +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.descriptors;
-
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SimpleInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-/**
- * A descriptor for samza framework internal usage.
- * <p>
- * Allows creating a {@link SystemDescriptor} without setting the factory class name, and delegating
- * rest of the system customization to configurations.
- * <p>
- * Useful for code-generation and testing use cases where the factory name is not known in advance.
- */
-@SuppressWarnings("unchecked")
-public final class DelegatingSystemDescriptor extends SystemDescriptor<DelegatingSystemDescriptor>
-    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
-
-  /**
-   * Constructs an {@link DelegatingSystemDescriptor} instance with no system level serde.
-   * Serdes must be provided explicitly at stream level when getting input or output descriptors.
-   * SystemFactory class name must be provided in configuration.
-   *
-   * @param systemName name of this system
-   */
-  @VisibleForTesting
-  public DelegatingSystemDescriptor(String systemName) {
-    super(systemName, null, null, null);
-  }
-
-  @Override
-  public <StreamMessageType> GenericInputDescriptor<StreamMessageType> getInputDescriptor(
-      String streamId, Serde<StreamMessageType> serde) {
-    return new GenericInputDescriptor<>(streamId, this, serde);
-  }
-
-  @Override
-  public <StreamMessageType> GenericOutputDescriptor<StreamMessageType> getOutputDescriptor(
-      String streamId, Serde<StreamMessageType> serde) {
-    return new GenericOutputDescriptor<>(streamId, this, serde);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java
index 2a73064..fbeda3e 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java
@@ -20,7 +20,7 @@ package org.apache.samza.operators.impl;
 
 import org.apache.samza.context.Context;
 import org.apache.samza.operators.KV;
-import org.apache.samza.operators.functions.InputTransformer;
+import org.apache.samza.system.descriptors.InputTransformer;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.system.IncomingMessageEnvelope;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
----------------------------------------------------------------------
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 1af4806..b467d60 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
@@ -18,7 +18,7 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.functions.InputTransformer;
+import org.apache.samza.system.descriptors.InputTransformer;
 import org.apache.samza.operators.functions.ScheduledFunction;
 import org.apache.samza.operators.functions.WatermarkFunction;
 import org.apache.samza.serializers.Serde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
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 0442f7c..1886d1b 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,8 @@ public abstract class OperatorSpec<M, OM> implements Serializable {
   }
 
   /**
-   * Get the unique ID of this operator in the {@link org.apache.samza.application.StreamApplicationDescriptorImpl}.
+   * Get the unique ID of this operator in the
+   * {@link org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl}.
    * @return  the unique operator ID
    */
   public final String getOpId() {

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
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 6ebbdae..8d3ff60 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
@@ -22,7 +22,7 @@ package org.apache.samza.operators.spec;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.operators.functions.InputTransformer;
+import org.apache.samza.system.descriptors.InputTransformer;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.functions.SinkFunction;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java
----------------------------------------------------------------------
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 4db8e60..44f62d9 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.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.OutputStream;
 import org.apache.samza.operators.spec.InputOperatorSpec;


[05/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/application/descriptors/TestStreamApplicationDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/application/descriptors/TestStreamApplicationDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/descriptors/TestStreamApplicationDescriptorImpl.java
new file mode 100644
index 0000000..d889486
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/application/descriptors/TestStreamApplicationDescriptorImpl.java
@@ -0,0 +1,602 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.samza.SamzaException;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.context.ApplicationContainerContextFactory;
+import org.apache.samza.context.ApplicationTaskContextFactory;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.operators.data.TestMessageEnvelope;
+import org.apache.samza.system.descriptors.ExpandingInputDescriptorProvider;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.TransformingInputDescriptorProvider;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.system.descriptors.StreamExpander;
+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;
+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.table.TableSpec;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+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.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Unit test for {@link StreamApplicationDescriptorImpl}
+ */
+public class TestStreamApplicationDescriptorImpl {
+
+  @Test
+  public void testConstructor() {
+    StreamApplication mockApp = mock(StreamApplication.class);
+    Config mockConfig = mock(Config.class);
+    StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(mockApp, mockConfig);
+    verify(mockApp).describe(appDesc);
+    assertEquals(mockConfig, appDesc.getConfig());
+  }
+
+  @Test
+  public void testGetInputStreamWithValueSerde() {
+
+    String streamId = "test-stream-1";
+    Serde mockValueSerde = mock(Serde.class);
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mockValueSerde);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+      }, mock(Config.class));
+
+    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(streamId, inputOpSpec.getStreamId());
+    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
+    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
+  }
+
+  @Test
+  public void testGetInputStreamWithKeyValueSerde() {
+
+    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();
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mockKVSerde);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+      }, mock(Config.class));
+
+    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(streamId, inputOpSpec.getStreamId());
+    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
+    assertEquals(mockKeySerde, inputOpSpec.getKeySerde());
+    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetInputStreamWithNullSerde() {
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd = sd.getInputDescriptor("mockStreamId", null);
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+      }, mock(Config.class));
+  }
+
+  @Test
+  public void testGetInputStreamWithTransformFunction() {
+    String streamId = "test-stream-1";
+    Serde mockValueSerde = mock(Serde.class);
+    InputTransformer transformer = ime -> ime;
+    MockTransformingSystemDescriptor sd = new MockTransformingSystemDescriptor("mockSystem", transformer);
+    MockInputDescriptor isd = sd.getInputDescriptor(streamId, mockValueSerde);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+      }, mock(Config.class));
+
+    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(streamId, inputOpSpec.getStreamId());
+    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
+    assertEquals(transformer, inputOpSpec.getTransformer());
+  }
+
+  @Test
+  public void testGetInputStreamWithExpandingSystem() {
+    String streamId = "test-stream-1";
+    String expandedStreamId = "expanded-stream";
+    AtomicInteger expandCallCount = new AtomicInteger();
+    StreamExpander expander = (sg, isd) -> {
+      expandCallCount.incrementAndGet();
+      InputDescriptor expandedISD =
+          new GenericSystemDescriptor("expanded-system", "mockFactoryClass")
+              .getInputDescriptor(expandedStreamId, new IntegerSerde());
+
+      return sg.getInputStream(expandedISD);
+    };
+    MockExpandingSystemDescriptor sd = new MockExpandingSystemDescriptor("mock-system", expander);
+    MockInputDescriptor isd = sd.getInputDescriptor(streamId, new IntegerSerde());
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+      }, mock(Config.class));
+
+    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(expandedStreamId);
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(1, expandCallCount.get());
+    assertFalse(streamAppDesc.getInputOperators().containsKey(streamId));
+    assertFalse(streamAppDesc.getInputDescriptors().containsKey(streamId));
+    assertTrue(streamAppDesc.getInputDescriptors().containsKey(expandedStreamId));
+    assertEquals(expandedStreamId, inputOpSpec.getStreamId());
+  }
+
+  @Test
+  public void testGetInputStreamWithRelaxedTypes() {
+    String streamId = "test-stream-1";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mock(Serde.class));
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+      }, mock(Config.class));
+
+    InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId);
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(streamId, inputOpSpec.getStreamId());
+    assertEquals(isd, streamAppDesc.getInputDescriptors().get(streamId));
+  }
+
+  @Test
+  public void testMultipleGetInputStreams() {
+    String streamId1 = "test-stream-1";
+    String streamId2 = "test-stream-2";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd1 = sd.getInputDescriptor(streamId1, mock(Serde.class));
+    GenericInputDescriptor isd2 = sd.getInputDescriptor(streamId2, mock(Serde.class));
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd1);
+        appDesc.getInputStream(isd2);
+      }, mock(Config.class));
+
+    InputOperatorSpec inputOpSpec1 = streamAppDesc.getInputOperators().get(streamId1);
+    InputOperatorSpec inputOpSpec2 = streamAppDesc.getInputOperators().get(streamId2);
+
+    assertEquals(2, streamAppDesc.getInputOperators().size());
+    assertEquals(streamId1, inputOpSpec1.getStreamId());
+    assertEquals(streamId2, inputOpSpec2.getStreamId());
+    assertEquals(2, streamAppDesc.getInputDescriptors().size());
+    assertEquals(isd1, streamAppDesc.getInputDescriptors().get(streamId1));
+    assertEquals(isd2, streamAppDesc.getInputDescriptors().get(streamId2));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetSameInputStreamTwice() {
+    String streamId = "test-stream-1";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd1 = sd.getInputDescriptor(streamId, mock(Serde.class));
+    GenericInputDescriptor isd2 = sd.getInputDescriptor(streamId, mock(Serde.class));
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd1);
+        // should throw exception
+        appDesc.getInputStream(isd2);
+      }, mock(Config.class));
+  }
+
+  @Test
+  public void testMultipleSystemDescriptorForSameSystemName() {
+    GenericSystemDescriptor sd1 = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericSystemDescriptor sd2 = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd1 = sd1.getInputDescriptor("test-stream-1", mock(Serde.class));
+    GenericInputDescriptor isd2 = sd2.getInputDescriptor("test-stream-2", mock(Serde.class));
+    GenericOutputDescriptor osd1 = sd2.getOutputDescriptor("test-stream-3", mock(Serde.class));
+
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd1);
+        try {
+          appDesc.getInputStream(isd2);
+          fail("Adding input stream with the same system name but different SystemDescriptor should have failed");
+        } catch (IllegalStateException e) { }
+
+        try {
+          appDesc.getOutputStream(osd1);
+          fail("adding output stream with the same system name but different SystemDescriptor should have failed");
+        } catch (IllegalStateException e) { }
+      }, mock(Config.class));
+
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        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");
+        } catch (IllegalStateException e) { }
+      }, mock(Config.class));
+  }
+
+  @Test
+  public void testGetOutputStreamWithKeyValueSerde() {
+    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();
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mockKVSerde);
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getOutputStream(osd);
+      }, mock(Config.class));
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId);
+    assertEquals(streamId, outputStreamImpl.getStreamId());
+    assertEquals(osd, streamAppDesc.getOutputDescriptors().get(streamId));
+    assertEquals(mockKeySerde, outputStreamImpl.getKeySerde());
+    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetOutputStreamWithNullSerde() {
+    String streamId = "test-stream-1";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, null);
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getOutputStream(osd);
+      }, mock(Config.class));
+  }
+
+  @Test
+  public void testGetOutputStreamWithValueSerde() {
+    String streamId = "test-stream-1";
+    Serde mockValueSerde = mock(Serde.class);
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mockValueSerde);
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getOutputStream(osd);
+      }, mock(Config.class));
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId);
+    assertEquals(streamId, outputStreamImpl.getStreamId());
+    assertEquals(osd, streamAppDesc.getOutputDescriptors().get(streamId));
+    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSetDefaultSystemDescriptorAfterGettingInputStream() {
+    String streamId = "test-stream-1";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mock(Serde.class));
+
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getInputStream(isd);
+        appDesc.withDefaultSystem(sd); // should throw exception
+      }, mock(Config.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSetDefaultSystemDescriptorAfterGettingOutputStream() {
+    String streamId = "test-stream-1";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mock(Serde.class));
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getOutputStream(osd);
+        appDesc.withDefaultSystem(sd); // should throw exception
+      }, mock(Config.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSetDefaultSystemDescriptorAfterGettingIntermediateStream() {
+    String streamId = "test-stream-1";
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class));
+    streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false);
+    streamAppDesc.withDefaultSystem(mock(SystemDescriptor.class)); // should throw exception
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetSameOutputStreamTwice() {
+    String streamId = "test-stream-1";
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    GenericOutputDescriptor osd1 = sd.getOutputDescriptor(streamId, mock(Serde.class));
+    GenericOutputDescriptor osd2 = sd.getOutputDescriptor(streamId, mock(Serde.class));
+    new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getOutputStream(osd1);
+        appDesc.getOutputStream(osd2); // should throw exception
+      }, mock(Config.class));
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithValueSerde() {
+    String streamId = "stream-1";
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class));
+
+    Serde mockValueSerde = mock(Serde.class);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        streamAppDesc.getIntermediateStream(streamId, mockValueSerde, false);
+
+    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());
+    assertTrue(((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, ((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithKeyValueSerde() {
+    String streamId = "streamId";
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, 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();
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        streamAppDesc.getIntermediateStream(streamId, mockKVSerde, false);
+
+    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());
+    assertEquals(mockKeySerde, ((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde());
+    assertEquals(mockValueSerde, ((InputOperatorSpec) (OperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithDefaultSystemDescriptor() {
+    Config mockConfig = mock(Config.class);
+    String streamId = "streamId";
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mock-system", "mock-system-factory");
+    streamAppDesc.withDefaultSystem(sd);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false);
+
+    assertEquals(streamAppDesc.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec());
+    assertEquals(streamAppDesc.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream());
+    assertEquals(streamId, intermediateStreamImpl.getStreamId());
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testGetIntermediateStreamWithNoSerde() {
+    Config mockConfig = mock(Config.class);
+    String streamId = "streamId";
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        streamAppDesc.getIntermediateStream(streamId, null, false); // should throw
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetSameIntermediateStreamTwice() {
+    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);
+  }
+
+  @Test
+  public void testGetNextOpIdIncrementsId() {
+    Config mockConfig = mock(Config.class);
+    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
+    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
+
+    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));
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testGetNextOpIdRejectsDuplicates() {
+    Config mockConfig = mock(Config.class);
+    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
+    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
+    assertEquals("jobName-1234-join-customName", streamAppDesc.getNextOpId(OpCode.JOIN, "customName"));
+    streamAppDesc.getNextOpId(OpCode.JOIN, "customName"); // should throw
+  }
+
+  @Test
+  public void testOpIdValidation() {
+    Config mockConfig = mock(Config.class);
+    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
+    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
+
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig);
+
+    // null and empty userDefinedIDs should fall back to autogenerated IDs.
+    try {
+      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.");
+    }
+
+    List<String> validOpIds = ImmutableList.of("op_id", "op-id", "1000", "op_1", "OP_ID");
+    for (String validOpId: validOpIds) {
+      try {
+        streamAppDesc.getNextOpId(OpCode.FILTER, validOpId);
+      } catch (Exception e) {
+        fail("Received an exception with a valid operator ID: " + validOpId);
+      }
+    }
+
+    List<String> invalidOpIds = ImmutableList.of("op id", "op#id");
+    for (String invalidOpId: invalidOpIds) {
+      try {
+        streamAppDesc.getNextOpId(OpCode.FILTER, invalidOpId);
+        fail("Did not receive an exception with an invalid operator ID: " + invalidOpId);
+      } catch (SamzaException e) { }
+    }
+  }
+
+  @Test
+  public void testGetInputStreamPreservesInsertionOrder() {
+    Config mockConfig = mock(Config.class);
+
+    String testStreamId1 = "test-stream-1";
+    String testStreamId2 = "test-stream-2";
+    String testStreamId3 = "test-stream-3";
+
+    GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass");
+    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)));
+      }, mockConfig);
+
+    List<InputOperatorSpec> inputSpecs = new ArrayList<>(streamAppDesc.getInputOperators().values());
+    assertEquals(inputSpecs.size(), 3);
+    assertEquals(inputSpecs.get(0).getStreamId(), testStreamId1);
+    assertEquals(inputSpecs.get(1).getStreamId(), testStreamId2);
+    assertEquals(inputSpecs.get(2).getStreamId(), testStreamId3);
+  }
+
+  @Test
+  public void testGetTable() throws Exception {
+    Config mockConfig = mock(Config.class);
+
+    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());
+    when(mockTableDescriptor.getSerde()).thenReturn(testTableSpec.getSerde());
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> {
+        appDesc.getTable(mockTableDescriptor);
+      }, mockConfig);
+    assertNotNull(streamAppDesc.getTables().get(testTableSpec.getId()));
+  }
+
+  @Test
+  public void testApplicationContainerContextFactory() {
+    ApplicationContainerContextFactory factory = mock(ApplicationContainerContextFactory.class);
+    StreamApplication testApp = appDesc -> appDesc.withApplicationContainerContextFactory(factory);
+    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.of(factory));
+  }
+
+  @Test
+  public void testNoApplicationContainerContextFactory() {
+    StreamApplication testApp = appDesc -> {
+    };
+    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.empty());
+  }
+
+  @Test
+  public void testApplicationTaskContextFactory() {
+    ApplicationTaskContextFactory factory = mock(ApplicationTaskContextFactory.class);
+    StreamApplication testApp = appDesc -> appDesc.withApplicationTaskContextFactory(factory);
+    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.of(factory));
+  }
+
+  @Test
+  public void testNoApplicationTaskContextFactory() {
+    StreamApplication testApp = appDesc -> {
+    };
+    StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.empty());
+  }
+
+  @Test
+  public void testProcessorLifecycleListenerFactory() {
+    ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class);
+    StreamApplication testApp = appSpec -> appSpec.withProcessorLifecycleListenerFactory(mockFactory);
+    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 StreamApplicationDescriptorImpl(appDesc -> {
+        BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class);
+        when(mockTableDescriptor.getTableId()).thenReturn("my.table");
+        appDesc.getTable(mockTableDescriptor);
+      }, mockConfig);
+  }
+
+  class MockExpandingSystemDescriptor extends SystemDescriptor<MockExpandingSystemDescriptor> implements ExpandingInputDescriptorProvider<Integer> {
+    public MockExpandingSystemDescriptor(String systemName, StreamExpander expander) {
+      super(systemName, "factory.class", null, expander);
+    }
+
+    @Override
+    public MockInputDescriptor<Integer> getInputDescriptor(String streamId, Serde serde) {
+      return new MockInputDescriptor<>(streamId, this, serde);
+    }
+  }
+
+  class MockTransformingSystemDescriptor extends SystemDescriptor<MockTransformingSystemDescriptor> implements TransformingInputDescriptorProvider<Integer> {
+    public MockTransformingSystemDescriptor(String systemName, InputTransformer transformer) {
+      super(systemName, "factory.class", transformer, null);
+    }
+
+    @Override
+    public MockInputDescriptor<Integer> getInputDescriptor(String streamId, Serde serde) {
+      return new MockInputDescriptor<>(streamId, this, serde);
+    }
+  }
+
+  public class MockInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, MockInputDescriptor<StreamMessageType>> {
+    MockInputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde serde) {
+      super(streamId, serde, systemDescriptor, null);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/application/descriptors/TestTaskApplicationDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/application/descriptors/TestTaskApplicationDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/descriptors/TestTaskApplicationDescriptorImpl.java
new file mode 100644
index 0000000..60c1c38
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/application/descriptors/TestTaskApplicationDescriptorImpl.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.descriptors;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.samza.application.TaskApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.context.ApplicationContainerContextFactory;
+import org.apache.samza.context.ApplicationTaskContextFactory;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.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;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+/**
+ * Unit test for {@link TaskApplicationDescriptorImpl}
+ */
+public class TestTaskApplicationDescriptorImpl {
+
+  private Config config = mock(Config.class);
+  private String defaultSystemName = "test-system";
+  private SystemDescriptor defaultSystemDescriptor = mock(SystemDescriptor.class);
+  private List<InputDescriptor> mockInputs = new ArrayList<InputDescriptor>() { {
+      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<OutputDescriptor> mockOutputs = new ArrayList<OutputDescriptor>() { {
+      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<TableDescriptor> mockTables = new HashSet<TableDescriptor>() { {
+      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);
+    } };
+
+  @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);
+    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(mockApp, config);
+    verify(mockApp).describe(appDesc);
+    assertEquals(config, appDesc.getConfig());
+  }
+
+  @Test
+  public void testAddInputStreams() {
+    TaskApplication testApp = appDesc -> {
+      mockInputs.forEach(appDesc::addInputStream);
+    };
+    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
+    assertEquals(mockInputs.toArray(), appDesc.getInputDescriptors().values().toArray());
+  }
+
+  @Test
+  public void testAddOutputStreams() {
+    TaskApplication testApp = appDesc -> {
+      mockOutputs.forEach(appDesc::addOutputStream);
+    };
+    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
+    assertEquals(mockOutputs.toArray(), appDesc.getOutputDescriptors().values().toArray());
+  }
+
+  @Test
+  public void testAddTables() {
+    TaskApplication testApp = appDesc -> {
+      mockTables.forEach(appDesc::addTable);
+    };
+    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
+    assertEquals(mockTables, appDesc.getTableDescriptors());
+  }
+
+  @Test
+  public void testSetTaskFactory() {
+    TaskFactory mockTf = mock(TaskFactory.class);
+    TaskApplication testApp = appDesc -> appDesc.setTaskFactory(mockTf);
+    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
+    assertEquals(appDesc.getTaskFactory(), mockTf);
+  }
+
+  @Test
+  public void testApplicationContainerContextFactory() {
+    ApplicationContainerContextFactory factory = mock(ApplicationContainerContextFactory.class);
+    TaskApplication testApp = appDesc -> appDesc.withApplicationContainerContextFactory(factory);
+    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.of(factory));
+  }
+
+  @Test
+  public void testNoApplicationContainerContextFactory() {
+    TaskApplication testApp = appDesc -> {
+    };
+    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationContainerContextFactory(), Optional.empty());
+  }
+
+  @Test
+  public void testApplicationTaskContextFactory() {
+    ApplicationTaskContextFactory factory = mock(ApplicationTaskContextFactory.class);
+    TaskApplication testApp = appDesc -> appDesc.withApplicationTaskContextFactory(factory);
+    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.of(factory));
+  }
+
+  @Test
+  public void testNoApplicationTaskContextFactory() {
+    TaskApplication testApp = appDesc -> {
+    };
+    TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl(testApp, mock(Config.class));
+    assertEquals(appSpec.getApplicationTaskContextFactory(), Optional.empty());
+  }
+
+  @Test
+  public void testProcessorLifecycleListener() {
+    ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class);
+    TaskApplication testApp = appDesc -> {
+      appDesc.withProcessorLifecycleListenerFactory(mockFactory);
+    };
+    TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config);
+    assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/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
index 6308589..a5fde45 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
@@ -21,20 +21,20 @@ 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.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.application.LegacyTaskApplication;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.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.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 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;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
----------------------------------------------------------------------
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 4cfcfd2..6208206 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
@@ -32,26 +32,26 @@ 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.descriptors.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.application.descriptors.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptorImpl;
 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.operators.BaseTableDescriptor;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
+import org.apache.samza.table.descriptors.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.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.descriptors.TableDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.StreamTableJoinFunction;
 import org.apache.samza.operators.windows.Windows;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java
----------------------------------------------------------------------
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 4de0485..979be94 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
@@ -23,7 +23,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.system.StreamSpec;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java
----------------------------------------------------------------------
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 34c16b3..b47014d 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,17 +26,17 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 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.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 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.windows.Windows;
 import org.apache.samza.serializers.JsonSerdeV2;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java
index 4618e52..dda0ee1 100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java
@@ -19,8 +19,8 @@
 package org.apache.samza.execution;
 
 import com.google.common.base.Joiner;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
-import org.apache.samza.application.TaskApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.ConfigRewriter;
 import org.apache.samza.config.JobConfig;
@@ -29,10 +29,10 @@ import org.apache.samza.config.SerializerConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.config.TaskConfigJava;
 import org.apache.samza.context.Context;
-import org.apache.samza.operators.BaseTableDescriptor;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
 import org.apache.samza.operators.KV;
-import org.apache.samza.operators.TableDescriptor;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.operators.impl.store.TimestampedValueSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
@@ -41,8 +41,8 @@ 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.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
 import org.apache.samza.table.TableSpec;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java
----------------------------------------------------------------------
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 9ed57fa..5a9b634 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,9 +22,9 @@ import com.google.common.collect.ImmutableList;
 import java.util.Collections;
 import java.util.List;
 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.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java
----------------------------------------------------------------------
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 85921f4..56be9a0 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,9 +20,9 @@ package org.apache.samza.execution;
 
 import java.util.Collections;
 import java.util.List;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java
----------------------------------------------------------------------
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 1315912..b3b0a5d 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,15 +21,15 @@ package org.apache.samza.operators;
 import com.google.common.collect.ImmutableSet;
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.context.Context;
 import org.apache.samza.context.MockContext;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 import org.apache.samza.job.model.TaskModel;
 import org.apache.samza.metrics.MetricsRegistryMap;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
-import org.apache.samza.operators.descriptors.GenericSystemDescriptor;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.impl.store.TestInMemoryStore;
 import org.apache.samza.operators.impl.store.TimestampedValueSerde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
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 0f4784d..ce9435f 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.time.Duration;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.operators.data.TestMessageEnvelope;
 import org.apache.samza.operators.data.TestOutputMessageEnvelope;
 import org.apache.samza.operators.functions.FilterFunction;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java
----------------------------------------------------------------------
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 abbbd3b..bbb70d2 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.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.operators.functions.ScheduledFunction;
 import org.apache.samza.operators.functions.WatermarkFunction;
 import org.apache.samza.operators.spec.InputOperatorSpec;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
----------------------------------------------------------------------
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 d760805..6bbd674 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
@@ -22,7 +22,7 @@ package org.apache.samza.operators.impl;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import org.apache.samza.Partition;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.MapConfig;
@@ -31,6 +31,9 @@ import org.apache.samza.container.TaskName;
 import org.apache.samza.context.Context;
 import org.apache.samza.context.MockContext;
 import org.apache.samza.context.TaskContextImpl;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericOutputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 import org.apache.samza.job.model.ContainerModel;
 import org.apache.samza.job.model.JobModel;
 import org.apache.samza.job.model.TaskModel;
@@ -38,9 +41,6 @@ import org.apache.samza.metrics.MetricsRegistryMap;
 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.ClosableFunction;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.InitableFunction;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java
----------------------------------------------------------------------
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 20d5e25..eae1ef4 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
@@ -31,20 +31,20 @@ import java.util.List;
 import java.util.Map;
 import org.apache.samza.Partition;
 import org.apache.samza.application.StreamApplication;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.container.TaskName;
 import org.apache.samza.context.Context;
 import org.apache.samza.context.MockContext;
 import org.apache.samza.context.TaskContextImpl;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 import org.apache.samza.job.model.TaskModel;
 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.descriptors.GenericInputDescriptor;
-import org.apache.samza.operators.descriptors.GenericSystemDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.impl.store.TestInMemoryStore;
 import org.apache.samza.operators.impl.store.TimeSeriesKeySerde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java
----------------------------------------------------------------------
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 fd4a7fb..64609b0 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,14 +20,14 @@ package org.apache.samza.operators.spec;
 
 import java.util.Collection;
 import java.util.Map;
-import org.apache.samza.application.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
+import org.apache.samza.system.descriptors.GenericInputDescriptor;
+import org.apache.samza.system.descriptors.GenericSystemDescriptor;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OperatorSpecGraph;
 import org.apache.samza.operators.Scheduler;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
-import org.apache.samza.operators.descriptors.GenericSystemDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.functions.ScheduledFunction;
 import org.apache.samza.operators.functions.WatermarkFunction;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
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 fd0ddf8..1b93072 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,11 +23,11 @@ import java.time.Duration;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.samza.application.ApplicationDescriptor;
-import org.apache.samza.application.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.ApplicationDescriptor;
+import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
 import org.apache.samza.application.LegacyTaskApplication;
 import org.apache.samza.application.SamzaApplication;
-import org.apache.samza.application.ApplicationDescriptorUtil;
+import org.apache.samza.application.descriptors.ApplicationDescriptorUtil;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java
----------------------------------------------------------------------
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 0952a87..cf23766 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
@@ -24,6 +24,8 @@ import org.apache.samza.config.JavaTableConfig;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.SerializerConfig;
 import org.apache.samza.context.MockContext;
+import org.apache.samza.table.descriptors.TableProvider;
+import org.apache.samza.table.descriptors.TableProviderFactory;
 import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.Serde;
 import org.apache.samza.serializers.SerializableSerde;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java b/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
index 128b938..865c611 100644
--- a/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
+++ b/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
@@ -28,15 +28,17 @@ import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.Gauge;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.Timer;
-import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.descriptors.BaseTableDescriptor;
+import org.apache.samza.table.descriptors.TableDescriptor;
 import org.apache.samza.storage.kv.Entry;
 import org.apache.samza.table.ReadWriteTable;
 import org.apache.samza.table.ReadableTable;
 import org.apache.samza.table.TableSpec;
 import org.apache.samza.table.caching.guava.GuavaCacheTable;
-import org.apache.samza.table.caching.guava.GuavaCacheTableDescriptor;
-import org.apache.samza.table.caching.guava.GuavaCacheTableProvider;
+import org.apache.samza.table.caching.descriptors.CachingTableDescriptor;
+import org.apache.samza.table.caching.descriptors.CachingTableProvider;
+import org.apache.samza.table.caching.guava.descriptors.GuavaCacheTableDescriptor;
+import org.apache.samza.table.caching.guava.descriptors.GuavaCacheTableProvider;
 import org.apache.samza.table.remote.RemoteReadWriteTable;
 import org.apache.samza.table.remote.TableRateLimiter;
 import org.apache.samza.table.remote.TableReadFunction;

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java b/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java
deleted file mode 100644
index f587885..0000000
--- a/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java
+++ /dev/null
@@ -1,236 +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.table.remote;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.samza.container.TaskName;
-import org.apache.samza.context.Context;
-import org.apache.samza.context.MockContext;
-import org.apache.samza.job.model.ContainerModel;
-import org.apache.samza.job.model.TaskModel;
-import org.apache.samza.metrics.Counter;
-import org.apache.samza.metrics.MetricsRegistry;
-import org.apache.samza.metrics.Timer;
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.apache.samza.table.retry.RetriableReadFunction;
-import org.apache.samza.table.retry.RetriableWriteFunction;
-import org.apache.samza.table.retry.TableRetryPolicy;
-import org.apache.samza.util.EmbeddedTaggedRateLimiter;
-import org.apache.samza.util.RateLimiter;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ThreadPoolExecutor;
-
-import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_READ_TAG;
-import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_WRITE_TAG;
-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.when;
-
-
-public class TestRemoteTableDescriptor {
-  private void doTestSerialize(RateLimiter rateLimiter,
-      TableRateLimiter.CreditFunction readCredFn,
-      TableRateLimiter.CreditFunction writeCredFn) {
-    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
-    desc.withReadFunction(mock(TableReadFunction.class));
-    desc.withWriteFunction(mock(TableWriteFunction.class));
-    if (rateLimiter != null) {
-      desc.withRateLimiter(rateLimiter, readCredFn, writeCredFn);
-    } else {
-      desc.withReadRateLimit(100);
-      desc.withWriteRateLimit(200);
-    }
-    TableSpec spec = desc.getTableSpec();
-    Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.RATE_LIMITER));
-    Assert.assertEquals(readCredFn != null, spec.getConfig().containsKey(RemoteTableProvider.READ_CREDIT_FN));
-    Assert.assertEquals(writeCredFn != null, spec.getConfig().containsKey(RemoteTableProvider.WRITE_CREDIT_FN));
-  }
-
-  @Test
-  public void testSerializeSimple() {
-    doTestSerialize(null, null, null);
-  }
-
-  @Test
-  public void testSerializeWithLimiter() {
-    doTestSerialize(mock(RateLimiter.class), null, null);
-  }
-
-  @Test
-  public void testSerializeWithLimiterAndReadCredFn() {
-    doTestSerialize(mock(RateLimiter.class), (k, v) -> 1, null);
-  }
-
-  @Test
-  public void testSerializeWithLimiterAndWriteCredFn() {
-    doTestSerialize(mock(RateLimiter.class), null, (k, v) -> 1);
-  }
-
-  @Test
-  public void testSerializeWithLimiterAndReadWriteCredFns() {
-    doTestSerialize(mock(RateLimiter.class), (key, value) -> 1, (key, value) -> 1);
-  }
-
-  @Test
-  public void testSerializeNullWriteFunction() {
-    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
-    desc.withReadFunction(mock(TableReadFunction.class));
-    TableSpec spec = desc.getTableSpec();
-    Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.READ_FN));
-    Assert.assertFalse(spec.getConfig().containsKey(RemoteTableProvider.WRITE_FN));
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testSerializeNullReadFunction() {
-    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
-    TableSpec spec = desc.getTableSpec();
-    Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.READ_FN));
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testSpecifyBothRateAndRateLimiter() {
-    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
-    desc.withReadFunction(mock(TableReadFunction.class));
-    desc.withReadRateLimit(100);
-    desc.withRateLimiter(mock(RateLimiter.class), null, null);
-    desc.getTableSpec();
-  }
-
-  private Context createMockContext() {
-    Context context = new MockContext();
-
-    MetricsRegistry metricsRegistry = mock(MetricsRegistry.class);
-    doReturn(mock(Timer.class)).when(metricsRegistry).newTimer(anyString(), anyString());
-    doReturn(mock(Counter.class)).when(metricsRegistry).newCounter(anyString(), anyString());
-    doReturn(metricsRegistry).when(context.getTaskContext()).getTaskMetricsRegistry();
-
-    TaskName taskName = new TaskName("MyTask");
-    TaskModel taskModel = mock(TaskModel.class);
-    when(taskModel.getTaskName()).thenReturn(taskName);
-    when(context.getTaskContext().getTaskModel()).thenReturn(taskModel);
-
-    ContainerModel containerModel = mock(ContainerModel.class);
-    when(containerModel.getTasks()).thenReturn(ImmutableMap.of(taskName, taskModel));
-    when(context.getContainerContext().getContainerModel()).thenReturn(containerModel);
-
-    return context;
-  }
-
-  static class CountingCreditFunction<K, V> implements TableRateLimiter.CreditFunction<K, V> {
-    int numCalls = 0;
-    @Override
-    public int getCredits(K key, V value) {
-      numCalls++;
-      return 1;
-    }
-  }
-
-  private void doTestDeserializeReadFunctionAndLimiter(boolean rateOnly, boolean rlGets, boolean rlPuts) {
-    int numRateLimitOps = (rlGets ? 1 : 0) + (rlPuts ? 1 : 0);
-    RemoteTableDescriptor<String, String> desc = new RemoteTableDescriptor("1");
-    TableRetryPolicy retryPolicy = new TableRetryPolicy();
-    retryPolicy.withRetryPredicate((ex) -> false);
-    desc.withReadFunction(mock(TableReadFunction.class), retryPolicy);
-    desc.withWriteFunction(mock(TableWriteFunction.class));
-    desc.withAsyncCallbackExecutorPoolSize(10);
-
-    if (rateOnly) {
-      if (rlGets) {
-        desc.withReadRateLimit(1000);
-      }
-      if (rlPuts) {
-        desc.withWriteRateLimit(2000);
-      }
-    } else {
-      if (numRateLimitOps > 0) {
-        Map<String, Integer> tagCredits = new HashMap<>();
-        if (rlGets) {
-          tagCredits.put(RL_READ_TAG, 1000);
-        }
-        if (rlPuts) {
-          tagCredits.put(RL_WRITE_TAG, 2000);
-        }
-
-        // Spy the rate limiter to verify call count
-        RateLimiter rateLimiter = spy(new EmbeddedTaggedRateLimiter(tagCredits));
-        desc.withRateLimiter(rateLimiter, new CountingCreditFunction(), new CountingCreditFunction());
-      }
-    }
-
-    TableSpec spec = desc.getTableSpec();
-    RemoteTableProvider provider = new RemoteTableProvider(spec);
-    provider.init(createMockContext());
-    Table table = provider.getTable();
-    Assert.assertTrue(table instanceof RemoteReadWriteTable);
-    RemoteReadWriteTable rwTable = (RemoteReadWriteTable) table;
-    if (numRateLimitOps > 0) {
-      Assert.assertTrue(!rlGets || rwTable.readRateLimiter != null);
-      Assert.assertTrue(!rlPuts || rwTable.writeRateLimiter != null);
-    }
-
-    ThreadPoolExecutor callbackExecutor = (ThreadPoolExecutor) rwTable.callbackExecutor;
-    Assert.assertEquals(10, callbackExecutor.getCorePoolSize());
-
-    Assert.assertNotNull(rwTable.readFn instanceof RetriableReadFunction);
-    Assert.assertNotNull(!(rwTable.writeFn instanceof RetriableWriteFunction));
-  }
-
-  @Test
-  public void testDeserializeReadFunctionNoRateLimit() {
-    doTestDeserializeReadFunctionAndLimiter(false, false, false);
-  }
-
-  @Test
-  public void testDeserializeReadFunctionAndLimiterWrite() {
-    doTestDeserializeReadFunctionAndLimiter(false, false, true);
-  }
-
-  @Test
-  public void testDeserializeReadFunctionAndLimiterRead() {
-    doTestDeserializeReadFunctionAndLimiter(false, true, false);
-  }
-
-  @Test
-  public void testDeserializeReadFunctionAndLimiterReadWrite() {
-    doTestDeserializeReadFunctionAndLimiter(false, true, true);
-  }
-
-  @Test
-  public void testDeserializeReadFunctionAndLimiterRateOnlyWrite() {
-    doTestDeserializeReadFunctionAndLimiter(true, false, true);
-  }
-
-  @Test
-  public void testDeserializeReadFunctionAndLimiterRateOnlyRead() {
-    doTestDeserializeReadFunctionAndLimiter(true, true, false);
-  }
-
-  @Test
-  public void testDeserializeReadFunctionAndLimiterRateOnlyReadWrite() {
-    doTestDeserializeReadFunctionAndLimiter(true, true, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/table/remote/descriptors/TestRemoteTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/table/remote/descriptors/TestRemoteTableDescriptor.java b/samza-core/src/test/java/org/apache/samza/table/remote/descriptors/TestRemoteTableDescriptor.java
new file mode 100644
index 0000000..e2841b6
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/table/remote/descriptors/TestRemoteTableDescriptor.java
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.remote.descriptors;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.context.Context;
+import org.apache.samza.context.MockContext;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.remote.RemoteReadWriteTable;
+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.retry.RetriableReadFunction;
+import org.apache.samza.table.retry.RetriableWriteFunction;
+import org.apache.samza.table.retry.TableRetryPolicy;
+import org.apache.samza.util.EmbeddedTaggedRateLimiter;
+import org.apache.samza.util.RateLimiter;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+
+public class TestRemoteTableDescriptor {
+  private void doTestSerialize(RateLimiter rateLimiter,
+      TableRateLimiter.CreditFunction readCredFn,
+      TableRateLimiter.CreditFunction writeCredFn) {
+    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
+    desc.withReadFunction(mock(TableReadFunction.class));
+    desc.withWriteFunction(mock(TableWriteFunction.class));
+    if (rateLimiter != null) {
+      desc.withRateLimiter(rateLimiter, readCredFn, writeCredFn);
+    } else {
+      desc.withReadRateLimit(100);
+      desc.withWriteRateLimit(200);
+    }
+    TableSpec spec = desc.getTableSpec();
+    Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.RATE_LIMITER));
+    Assert.assertEquals(readCredFn != null, spec.getConfig().containsKey(RemoteTableProvider.READ_CREDIT_FN));
+    Assert.assertEquals(writeCredFn != null, spec.getConfig().containsKey(RemoteTableProvider.WRITE_CREDIT_FN));
+  }
+
+  @Test
+  public void testSerializeSimple() {
+    doTestSerialize(null, null, null);
+  }
+
+  @Test
+  public void testSerializeWithLimiter() {
+    doTestSerialize(mock(RateLimiter.class), null, null);
+  }
+
+  @Test
+  public void testSerializeWithLimiterAndReadCredFn() {
+    doTestSerialize(mock(RateLimiter.class), (k, v) -> 1, null);
+  }
+
+  @Test
+  public void testSerializeWithLimiterAndWriteCredFn() {
+    doTestSerialize(mock(RateLimiter.class), null, (k, v) -> 1);
+  }
+
+  @Test
+  public void testSerializeWithLimiterAndReadWriteCredFns() {
+    doTestSerialize(mock(RateLimiter.class), (key, value) -> 1, (key, value) -> 1);
+  }
+
+  @Test
+  public void testSerializeNullWriteFunction() {
+    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
+    desc.withReadFunction(mock(TableReadFunction.class));
+    TableSpec spec = desc.getTableSpec();
+    Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.READ_FN));
+    Assert.assertFalse(spec.getConfig().containsKey(RemoteTableProvider.WRITE_FN));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testSerializeNullReadFunction() {
+    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
+    TableSpec spec = desc.getTableSpec();
+    Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.READ_FN));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testSpecifyBothRateAndRateLimiter() {
+    RemoteTableDescriptor desc = new RemoteTableDescriptor("1");
+    desc.withReadFunction(mock(TableReadFunction.class));
+    desc.withReadRateLimit(100);
+    desc.withRateLimiter(mock(RateLimiter.class), null, null);
+    desc.getTableSpec();
+  }
+
+  private Context createMockContext() {
+    Context context = new MockContext();
+
+    MetricsRegistry metricsRegistry = mock(MetricsRegistry.class);
+    doReturn(mock(Timer.class)).when(metricsRegistry).newTimer(Matchers.anyString(), Matchers.anyString());
+    doReturn(mock(Counter.class)).when(metricsRegistry).newCounter(Matchers.anyString(), Matchers.anyString());
+    doReturn(metricsRegistry).when(context.getTaskContext()).getTaskMetricsRegistry();
+
+    TaskName taskName = new TaskName("MyTask");
+    TaskModel taskModel = mock(TaskModel.class);
+    when(taskModel.getTaskName()).thenReturn(taskName);
+    when(context.getTaskContext().getTaskModel()).thenReturn(taskModel);
+
+    ContainerModel containerModel = mock(ContainerModel.class);
+    when(containerModel.getTasks()).thenReturn(ImmutableMap.of(taskName, taskModel));
+    when(context.getContainerContext().getContainerModel()).thenReturn(containerModel);
+
+    return context;
+  }
+
+  static class CountingCreditFunction<K, V> implements TableRateLimiter.CreditFunction<K, V> {
+    int numCalls = 0;
+    @Override
+    public int getCredits(K key, V value) {
+      numCalls++;
+      return 1;
+    }
+  }
+
+  private void doTestDeserializeReadFunctionAndLimiter(boolean rateOnly, boolean rlGets, boolean rlPuts) {
+    int numRateLimitOps = (rlGets ? 1 : 0) + (rlPuts ? 1 : 0);
+    RemoteTableDescriptor<String, String> desc = new RemoteTableDescriptor("1");
+    TableRetryPolicy retryPolicy = new TableRetryPolicy();
+    retryPolicy.withRetryPredicate((ex) -> false);
+    desc.withReadFunction(mock(TableReadFunction.class), retryPolicy);
+    desc.withWriteFunction(mock(TableWriteFunction.class));
+    desc.withAsyncCallbackExecutorPoolSize(10);
+
+    if (rateOnly) {
+      if (rlGets) {
+        desc.withReadRateLimit(1000);
+      }
+      if (rlPuts) {
+        desc.withWriteRateLimit(2000);
+      }
+    } else {
+      if (numRateLimitOps > 0) {
+        Map<String, Integer> tagCredits = new HashMap<>();
+        if (rlGets) {
+          tagCredits.put(RemoteTableDescriptor.RL_READ_TAG, 1000);
+        }
+        if (rlPuts) {
+          tagCredits.put(RemoteTableDescriptor.RL_WRITE_TAG, 2000);
+        }
+
+        // Spy the rate limiter to verify call count
+        RateLimiter rateLimiter = spy(new EmbeddedTaggedRateLimiter(tagCredits));
+        desc.withRateLimiter(rateLimiter, new CountingCreditFunction(), new CountingCreditFunction());
+      }
+    }
+
+    TableSpec spec = desc.getTableSpec();
+    RemoteTableProvider provider = new RemoteTableProvider(spec);
+    provider.init(createMockContext());
+    Table table = provider.getTable();
+    Assert.assertTrue(table instanceof RemoteReadWriteTable);
+    RemoteReadWriteTable rwTable = (RemoteReadWriteTable) table;
+    if (numRateLimitOps > 0) {
+      Assert.assertTrue(!rlGets || rwTable.getReadRateLimiter() != null);
+      Assert.assertTrue(!rlPuts || rwTable.getWriteRateLimiter() != null);
+    }
+
+    ThreadPoolExecutor callbackExecutor = (ThreadPoolExecutor) rwTable.getCallbackExecutor();
+    Assert.assertEquals(10, callbackExecutor.getCorePoolSize());
+
+    Assert.assertNotNull(rwTable.getReadFn() instanceof RetriableReadFunction);
+    Assert.assertNotNull(!(rwTable.getWriteFn() instanceof RetriableWriteFunction));
+  }
+
+  @Test
+  public void testDeserializeReadFunctionNoRateLimit() {
+    doTestDeserializeReadFunctionAndLimiter(false, false, false);
+  }
+
+  @Test
+  public void testDeserializeReadFunctionAndLimiterWrite() {
+    doTestDeserializeReadFunctionAndLimiter(false, false, true);
+  }
+
+  @Test
+  public void testDeserializeReadFunctionAndLimiterRead() {
+    doTestDeserializeReadFunctionAndLimiter(false, true, false);
+  }
+
+  @Test
+  public void testDeserializeReadFunctionAndLimiterReadWrite() {
+    doTestDeserializeReadFunctionAndLimiter(false, true, true);
+  }
+
+  @Test
+  public void testDeserializeReadFunctionAndLimiterRateOnlyWrite() {
+    doTestDeserializeReadFunctionAndLimiter(true, false, true);
+  }
+
+  @Test
+  public void testDeserializeReadFunctionAndLimiterRateOnlyRead() {
+    doTestDeserializeReadFunctionAndLimiter(true, true, false);
+  }
+
+  @Test
+  public void testDeserializeReadFunctionAndLimiterRateOnlyReadWrite() {
+    doTestDeserializeReadFunctionAndLimiter(true, true, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java
----------------------------------------------------------------------
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 f96ab19..82fb41c 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,9 +21,9 @@ package org.apache.samza.task;
 import java.lang.reflect.Field;
 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.application.descriptors.ApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.StreamApplicationDescriptorImpl;
+import org.apache.samza.application.descriptors.TaskApplicationDescriptorImpl;
 import org.apache.samza.config.ConfigException;
 import org.apache.samza.operators.OperatorSpecGraph;
 import org.junit.Test;


[10/12] samza git commit: Consolidating package names for System, Stream, Application and Table descriptors.

Posted by pm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestExpandingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestExpandingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestExpandingInputDescriptor.java
deleted file mode 100644
index 1ec81ce..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestExpandingInputDescriptor.java
+++ /dev/null
@@ -1,59 +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.descriptors;
-
-import java.util.Collections;
-import org.apache.samza.operators.descriptors.expanding.ExampleExpandingInputDescriptor;
-import org.apache.samza.operators.descriptors.expanding.ExampleExpandingOutputDescriptor;
-import org.apache.samza.operators.descriptors.expanding.ExampleExpandingSystemDescriptor;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestExpandingInputDescriptor {
-  public void testAPIUsage() {
-    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
-    // and validates that the method calls can be chained.
-    ExampleExpandingSystemDescriptor expandingSystem = new ExampleExpandingSystemDescriptor("expandingSystem");
-    ExampleExpandingInputDescriptor<Long> input1 = expandingSystem.getInputDescriptor("input1", new IntegerSerde());
-    ExampleExpandingOutputDescriptor<Integer> output1 = expandingSystem.getOutputDescriptor("output1", new IntegerSerde());
-
-    input1
-        .shouldBootstrap()
-        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
-        .withPriority(1)
-        .shouldResetOffset()
-        .withStreamConfigs(Collections.emptyMap());
-
-    output1
-        .withStreamConfigs(Collections.emptyMap());
-  }
-
-  @Test
-  public void testISDObjectsWithOverrides() {
-    ExampleExpandingSystemDescriptor expandingSystem = new ExampleExpandingSystemDescriptor("expandingSystem");
-    IntegerSerde streamSerde = new IntegerSerde();
-    ExampleExpandingInputDescriptor<Long> expandingISD = expandingSystem.getInputDescriptor("input-stream", streamSerde);
-
-    assertEquals(streamSerde, expandingISD.getSerde());
-    assertEquals(expandingSystem.getTransformer().get(), expandingISD.getTransformer().get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericInputDescriptor.java
deleted file mode 100644
index 012da1b..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericInputDescriptor.java
+++ /dev/null
@@ -1,123 +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.descriptors;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.samza.serializers.DoubleSerde;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-public class TestGenericInputDescriptor {
-  @Test
-  public void testAPIUsage() {
-    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
-    // and validates that the method calls can be chained.
-    GenericSystemDescriptor mySystem =
-        new GenericSystemDescriptor("input-system", "factory.class.name")
-            .withSystemConfigs(Collections.emptyMap())
-            .withDefaultStreamConfigs(Collections.emptyMap());
-    GenericInputDescriptor<Integer> input1 = mySystem.getInputDescriptor("input1", new IntegerSerde());
-    GenericOutputDescriptor<Integer> output1 = mySystem.getOutputDescriptor("output1", new IntegerSerde());
-
-    input1
-        .withPhysicalName("input-1")
-        .shouldBootstrap()
-        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
-        .withPriority(1)
-        .shouldResetOffset()
-        .isBounded()
-        .shouldDeleteCommittedMessages()
-        .withStreamConfigs(Collections.emptyMap());
-
-    output1
-        .withPhysicalName("output-1")
-        .withStreamConfigs(Collections.emptyMap());
-  }
-
-
-  @Test
-  public void testISDConfigsWithOverrides() {
-    GenericSystemDescriptor mySystem =
-        new GenericSystemDescriptor("input-system", "factory.class.name")
-            .withSystemConfigs(Collections.emptyMap())
-            .withDefaultStreamConfigs(Collections.emptyMap());
-
-    GenericInputDescriptor<Double> isd = mySystem.getInputDescriptor("input-stream", new DoubleSerde())
-            .withPhysicalName("physical-name")
-            .shouldBootstrap()
-            .isBounded()
-            .shouldDeleteCommittedMessages()
-            .withOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST)
-            .withPriority(12)
-            .shouldResetOffset()
-            .withStreamConfigs(ImmutableMap.of("custom-config-key", "custom-config-value"));
-
-    Map<String, String> generatedConfigs = isd.toConfig();
-    Map<String, String> expectedConfigs = new HashMap<>();
-    expectedConfigs.put("streams.input-stream.samza.system", "input-system");
-    expectedConfigs.put("streams.input-stream.samza.physical.name", "physical-name");
-    expectedConfigs.put("streams.input-stream.samza.bootstrap", "true");
-    expectedConfigs.put("streams.input-stream.samza.bounded", "true");
-    expectedConfigs.put("streams.input-stream.samza.delete.committed.messages", "true");
-    expectedConfigs.put("streams.input-stream.samza.reset.offset", "true");
-    expectedConfigs.put("streams.input-stream.samza.offset.default", "oldest");
-    expectedConfigs.put("streams.input-stream.samza.priority", "12");
-    expectedConfigs.put("streams.input-stream.custom-config-key", "custom-config-value");
-
-    assertEquals(expectedConfigs, generatedConfigs);
-  }
-
-  @Test
-  public void testISDConfigsWithDefaults() {
-    GenericSystemDescriptor mySystem =
-        new GenericSystemDescriptor("input-system", "factory.class.name")
-            .withSystemConfigs(Collections.emptyMap())
-            .withDefaultStreamConfigs(Collections.emptyMap());
-
-    DoubleSerde streamSerde = new DoubleSerde();
-    GenericInputDescriptor<Double> isd = mySystem.getInputDescriptor("input-stream", streamSerde);
-
-    Map<String, String> generatedConfigs = isd.toConfig();
-    Map<String, String> expectedConfigs = ImmutableMap.of("streams.input-stream.samza.system", "input-system");
-    assertEquals(expectedConfigs, generatedConfigs);
-    assertEquals(streamSerde, isd.getSerde());
-    assertFalse(isd.getTransformer().isPresent());
-  }
-
-  @Test
-  public void testISDObjectsWithOverrides() {
-    GenericSystemDescriptor mySystem =
-        new GenericSystemDescriptor("input-system", "factory.class.name")
-            .withSystemConfigs(Collections.emptyMap())
-            .withDefaultStreamConfigs(Collections.emptyMap());
-    IntegerSerde streamSerde = new IntegerSerde();
-    GenericInputDescriptor<Integer> isd = mySystem.getInputDescriptor("input-stream", streamSerde);
-
-    assertEquals(streamSerde, isd.getSerde());
-    assertFalse(isd.getTransformer().isPresent());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericSystemDescriptor.java
deleted file mode 100644
index 937b56a..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestGenericSystemDescriptor.java
+++ /dev/null
@@ -1,63 +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.descriptors;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Map;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestGenericSystemDescriptor {
-  @Test
-  public void testSDConfigs() {
-    GenericSystemDescriptor mySystem =
-        new GenericSystemDescriptor("input-system", "factory.class.name")
-            .withSystemConfigs(ImmutableMap.of("custom-config-key", "custom-config-value"))
-            .withDefaultStreamConfigs(ImmutableMap.of("custom-stream-config-key", "custom-stream-config-value"))
-            .withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.UPCOMING);
-
-    Map<String, String> generatedConfigs = mySystem.toConfig();
-    Map<String, String> expectedConfigs = ImmutableMap.of(
-        "systems.input-system.samza.factory", "factory.class.name",
-        "systems.input-system.custom-config-key", "custom-config-value",
-        "systems.input-system.default.stream.custom-stream-config-key", "custom-stream-config-value",
-        "systems.input-system.default.stream.samza.offset.default", "upcoming"
-    );
-    assertEquals(expectedConfigs, generatedConfigs);
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testGetInputDescriptorWithNullSerde() {
-    GenericSystemDescriptor mySystem = new GenericSystemDescriptor("input-system", "factory.class.name");
-    mySystem.getInputDescriptor("streamId", null); // should throw an exception
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testGetSystemDescriptorWithNullSystemName() {
-    new GenericSystemDescriptor(null, "factory.class.name");
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testGetSystemDescriptorWithEmptySystemName() {
-    new GenericSystemDescriptor(" ", "factory.class.name");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestSimpleInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestSimpleInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestSimpleInputDescriptor.java
deleted file mode 100644
index b013db4..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestSimpleInputDescriptor.java
+++ /dev/null
@@ -1,63 +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.descriptors;
-
-import java.util.Collections;
-import org.apache.samza.operators.descriptors.serde.ExampleSimpleInputDescriptor;
-import org.apache.samza.operators.descriptors.serde.ExampleSimpleOutputDescriptor;
-import org.apache.samza.operators.descriptors.serde.ExampleSimpleSystemDescriptor;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-public class TestSimpleInputDescriptor {
-  @Test
-  public void testAPIUsage() {
-    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
-    // and validates that the method calls can be chained.
-    ExampleSimpleSystemDescriptor kafkaSystem =
-        new ExampleSimpleSystemDescriptor("kafka-system")
-            .withSystemConfigs(Collections.emptyMap());
-    ExampleSimpleInputDescriptor<Integer> input1 = kafkaSystem.getInputDescriptor("input1", new IntegerSerde());
-    ExampleSimpleOutputDescriptor<Integer> output1 = kafkaSystem.getOutputDescriptor("output1", new IntegerSerde());
-
-    input1
-        .shouldBootstrap()
-        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
-        .withPriority(1)
-        .shouldResetOffset()
-        .withStreamConfigs(Collections.emptyMap());
-
-    output1
-        .withStreamConfigs(Collections.emptyMap());
-  }
-
-  @Test
-  public void testISDObjectsWithOverrides() {
-    ExampleSimpleSystemDescriptor ssd = new ExampleSimpleSystemDescriptor("kafka-system");
-    IntegerSerde streamSerde = new IntegerSerde();
-    ExampleSimpleInputDescriptor<Integer> isd = ssd.getInputDescriptor("input-stream", streamSerde);
-
-    assertEquals(streamSerde, isd.getSerde());
-    assertFalse(isd.getTransformer().isPresent());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestTransformingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestTransformingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestTransformingInputDescriptor.java
deleted file mode 100644
index f53f66d..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/TestTransformingInputDescriptor.java
+++ /dev/null
@@ -1,64 +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.descriptors;
-
-import java.util.Collections;
-import org.apache.samza.operators.descriptors.transforming.ExampleTransformingInputDescriptor;
-import org.apache.samza.operators.descriptors.transforming.ExampleTransformingOutputDescriptor;
-import org.apache.samza.operators.descriptors.transforming.ExampleTransformingSystemDescriptor;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestTransformingInputDescriptor {
-  @Test
-  public void testAPIUsage() {
-    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
-    // and validates that the method calls can be chained.
-    ExampleTransformingSystemDescriptor imeTransformingSystem =
-        new ExampleTransformingSystemDescriptor("imeTransformingSystem")
-            .withSystemConfigs(Collections.emptyMap());
-    ExampleTransformingInputDescriptor<Long> input1 = imeTransformingSystem.getInputDescriptor("input1", new IntegerSerde());
-    ExampleTransformingOutputDescriptor<Integer> output1 = imeTransformingSystem.getOutputDescriptor("output1", new IntegerSerde());
-
-    input1
-        .shouldBootstrap()
-        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
-        .withPriority(1)
-        .shouldResetOffset()
-        .withStreamConfigs(Collections.emptyMap());
-
-    output1
-        .withStreamConfigs(Collections.emptyMap());
-  }
-
-  @Test
-  public void testISDObjectsWithOverrides() {
-    ExampleTransformingSystemDescriptor imeTransformingSystem =
-        new ExampleTransformingSystemDescriptor("imeTransformingSystem");
-    IntegerSerde streamSerde = new IntegerSerde();
-    ExampleTransformingInputDescriptor<Long> overridingISD =
-        imeTransformingSystem.getInputDescriptor("input-stream", streamSerde);
-
-    assertEquals(streamSerde, overridingISD.getSerde());
-    assertEquals(imeTransformingSystem.getTransformer().get(), overridingISD.getTransformer().get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingInputDescriptor.java
deleted file mode 100644
index 1f382a3..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingInputDescriptor.java
+++ /dev/null
@@ -1,30 +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.descriptors.expanding;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleExpandingInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, ExampleExpandingInputDescriptor<StreamMessageType>> {
-  ExampleExpandingInputDescriptor(String streamId, SystemDescriptor systemDescriptor, InputTransformer<StreamMessageType> transformer, Serde serde) {
-    super(streamId, serde, systemDescriptor, transformer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingOutputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingOutputDescriptor.java
deleted file mode 100644
index 705b866..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingOutputDescriptor.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.operators.descriptors.expanding;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleExpandingOutputDescriptor<StreamMessageType> extends OutputDescriptor<StreamMessageType, ExampleExpandingOutputDescriptor<StreamMessageType>> {
-  ExampleExpandingOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
-    super(streamId, serde, systemDescriptor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingSystemDescriptor.java
deleted file mode 100644
index c81c8aa..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/expanding/ExampleExpandingSystemDescriptor.java
+++ /dev/null
@@ -1,49 +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.descriptors.expanding;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.descriptors.base.system.ExpandingInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.system.IncomingMessageEnvelope;
-
-public class ExampleExpandingSystemDescriptor extends SystemDescriptor<ExampleExpandingSystemDescriptor>
-    implements ExpandingInputDescriptorProvider<Long>, OutputDescriptorProvider {
-  private static final String FACTORY_CLASS_NAME = "org.apache.samza.GraphExpandingSystemFactory";
-
-  public ExampleExpandingSystemDescriptor(String systemName) {
-    super(systemName, FACTORY_CLASS_NAME,
-        (InputTransformer<String>) IncomingMessageEnvelope::toString,
-        (streamGraph, inputDescriptor) -> (MessageStream<Long>) streamGraph.getInputStream(inputDescriptor)
-    );
-  }
-
-  @Override
-  public ExampleExpandingInputDescriptor<Long> getInputDescriptor(String streamId, Serde serde) {
-    return new ExampleExpandingInputDescriptor<>(streamId, this, null, serde);
-  }
-
-  @Override
-  public <StreamMessageType> ExampleExpandingOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
-    return new ExampleExpandingOutputDescriptor<>(streamId, this, serde);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleInputDescriptor.java
deleted file mode 100644
index c5df448..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleInputDescriptor.java
+++ /dev/null
@@ -1,30 +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.descriptors.serde;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleSimpleInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, ExampleSimpleInputDescriptor<StreamMessageType>> {
-  ExampleSimpleInputDescriptor(String streamId, SystemDescriptor systemDescriptor, InputTransformer<StreamMessageType> transformer, Serde serde) {
-    super(streamId, serde, systemDescriptor, transformer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleOutputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleOutputDescriptor.java
deleted file mode 100644
index aeda2e0..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleOutputDescriptor.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.operators.descriptors.serde;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleSimpleOutputDescriptor<StreamMessageType> extends OutputDescriptor<StreamMessageType, ExampleSimpleOutputDescriptor<StreamMessageType>> {
-  ExampleSimpleOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
-    super(streamId, serde, systemDescriptor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleSystemDescriptor.java
deleted file mode 100644
index b0af4d8..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/serde/ExampleSimpleSystemDescriptor.java
+++ /dev/null
@@ -1,43 +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.descriptors.serde;
-
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SimpleInputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleSimpleSystemDescriptor extends SystemDescriptor<ExampleSimpleSystemDescriptor>
-    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
-  private static final String FACTORY_CLASS_NAME = "org.apache.kafka.KafkaSystemFactory";
-
-  public ExampleSimpleSystemDescriptor(String systemName) {
-    super(systemName, FACTORY_CLASS_NAME, null, null);
-  }
-
-  @Override
-  public <StreamMessageType> ExampleSimpleInputDescriptor<StreamMessageType> getInputDescriptor(String streamId, Serde<StreamMessageType> serde) {
-    return new ExampleSimpleInputDescriptor<>(streamId, this, null, serde);
-  }
-
-  @Override
-  public <StreamMessageType> ExampleSimpleOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
-    return new ExampleSimpleOutputDescriptor<>(streamId, this, serde);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingInputDescriptor.java
deleted file mode 100644
index 78b6f33..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingInputDescriptor.java
+++ /dev/null
@@ -1,30 +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.descriptors.transforming;
-
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.functions.InputTransformer;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleTransformingInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, ExampleTransformingInputDescriptor<StreamMessageType>> {
-  ExampleTransformingInputDescriptor(String streamId, SystemDescriptor systemDescriptor, InputTransformer<StreamMessageType> transformer, Serde serde) {
-    super(streamId, serde, systemDescriptor, transformer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingOutputDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingOutputDescriptor.java
deleted file mode 100644
index c37906b..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingOutputDescriptor.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.operators.descriptors.transforming;
-
-import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleTransformingOutputDescriptor<StreamMessageType> extends OutputDescriptor<StreamMessageType, ExampleTransformingOutputDescriptor<StreamMessageType>> {
-  ExampleTransformingOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
-    super(streamId, serde, systemDescriptor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingSystemDescriptor.java
deleted file mode 100644
index f8aff61..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/descriptors/transforming/ExampleTransformingSystemDescriptor.java
+++ /dev/null
@@ -1,43 +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.descriptors.transforming;
-
-import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.operators.descriptors.base.system.TransformingInputDescriptorProvider;
-import org.apache.samza.serializers.Serde;
-
-public class ExampleTransformingSystemDescriptor extends SystemDescriptor<ExampleTransformingSystemDescriptor>
-    implements TransformingInputDescriptorProvider<Long>, OutputDescriptorProvider {
-  private static final String FACTORY_CLASS_NAME = "org.apache.samza.IMETransformingSystemFactory";
-
-  public ExampleTransformingSystemDescriptor(String systemName) {
-    super(systemName, FACTORY_CLASS_NAME, ime -> 1L, null);
-  }
-
-  @Override
-  public ExampleTransformingInputDescriptor<Long> getInputDescriptor(String streamId, Serde serde) {
-    return new ExampleTransformingInputDescriptor<>(streamId, this, null, serde);
-  }
-
-  @Override
-  public <StreamMessageType> ExampleTransformingOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
-    return new ExampleTransformingOutputDescriptor<>(streamId, this, serde);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/TestExpandingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/TestExpandingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestExpandingInputDescriptor.java
new file mode 100644
index 0000000..a34fac9
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestExpandingInputDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import java.util.Collections;
+import org.apache.samza.system.descriptors.examples.expanding.ExampleExpandingInputDescriptor;
+import org.apache.samza.system.descriptors.examples.expanding.ExampleExpandingOutputDescriptor;
+import org.apache.samza.system.descriptors.examples.expanding.ExampleExpandingSystemDescriptor;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestExpandingInputDescriptor {
+  public void testAPIUsage() {
+    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
+    // and validates that the method calls can be chained.
+    ExampleExpandingSystemDescriptor expandingSystem = new ExampleExpandingSystemDescriptor("expandingSystem");
+    ExampleExpandingInputDescriptor<Long> input1 = expandingSystem.getInputDescriptor("input1", new IntegerSerde());
+    ExampleExpandingOutputDescriptor<Integer> output1 = expandingSystem.getOutputDescriptor("output1", new IntegerSerde());
+
+    input1
+        .shouldBootstrap()
+        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
+        .withPriority(1)
+        .shouldResetOffset()
+        .withStreamConfigs(Collections.emptyMap());
+
+    output1
+        .withStreamConfigs(Collections.emptyMap());
+  }
+
+  @Test
+  public void testISDObjectsWithOverrides() {
+    ExampleExpandingSystemDescriptor expandingSystem = new ExampleExpandingSystemDescriptor("expandingSystem");
+    IntegerSerde streamSerde = new IntegerSerde();
+    ExampleExpandingInputDescriptor<Long> expandingISD = expandingSystem.getInputDescriptor("input-stream", streamSerde);
+
+    assertEquals(streamSerde, expandingISD.getSerde());
+    assertEquals(expandingSystem.getTransformer().get(), expandingISD.getTransformer().get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericInputDescriptor.java
new file mode 100644
index 0000000..91b3768
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericInputDescriptor.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.serializers.DoubleSerde;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestGenericInputDescriptor {
+  @Test
+  public void testAPIUsage() {
+    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
+    // and validates that the method calls can be chained.
+    GenericSystemDescriptor mySystem =
+        new GenericSystemDescriptor("input-system", "factory.class.name")
+            .withSystemConfigs(Collections.emptyMap())
+            .withDefaultStreamConfigs(Collections.emptyMap());
+    GenericInputDescriptor<Integer> input1 = mySystem.getInputDescriptor("input1", new IntegerSerde());
+    GenericOutputDescriptor<Integer> output1 = mySystem.getOutputDescriptor("output1", new IntegerSerde());
+
+    input1
+        .withPhysicalName("input-1")
+        .shouldBootstrap()
+        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
+        .withPriority(1)
+        .shouldResetOffset()
+        .isBounded()
+        .shouldDeleteCommittedMessages()
+        .withStreamConfigs(Collections.emptyMap());
+
+    output1
+        .withPhysicalName("output-1")
+        .withStreamConfigs(Collections.emptyMap());
+  }
+
+
+  @Test
+  public void testISDConfigsWithOverrides() {
+    GenericSystemDescriptor mySystem =
+        new GenericSystemDescriptor("input-system", "factory.class.name")
+            .withSystemConfigs(Collections.emptyMap())
+            .withDefaultStreamConfigs(Collections.emptyMap());
+
+    GenericInputDescriptor<Double> isd = mySystem.getInputDescriptor("input-stream", new DoubleSerde())
+            .withPhysicalName("physical-name")
+            .shouldBootstrap()
+            .isBounded()
+            .shouldDeleteCommittedMessages()
+            .withOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST)
+            .withPriority(12)
+            .shouldResetOffset()
+            .withStreamConfigs(ImmutableMap.of("custom-config-key", "custom-config-value"));
+
+    Map<String, String> generatedConfigs = isd.toConfig();
+    Map<String, String> expectedConfigs = new HashMap<>();
+    expectedConfigs.put("streams.input-stream.samza.system", "input-system");
+    expectedConfigs.put("streams.input-stream.samza.physical.name", "physical-name");
+    expectedConfigs.put("streams.input-stream.samza.bootstrap", "true");
+    expectedConfigs.put("streams.input-stream.samza.bounded", "true");
+    expectedConfigs.put("streams.input-stream.samza.delete.committed.messages", "true");
+    expectedConfigs.put("streams.input-stream.samza.reset.offset", "true");
+    expectedConfigs.put("streams.input-stream.samza.offset.default", "oldest");
+    expectedConfigs.put("streams.input-stream.samza.priority", "12");
+    expectedConfigs.put("streams.input-stream.custom-config-key", "custom-config-value");
+
+    assertEquals(expectedConfigs, generatedConfigs);
+  }
+
+  @Test
+  public void testISDConfigsWithDefaults() {
+    GenericSystemDescriptor mySystem =
+        new GenericSystemDescriptor("input-system", "factory.class.name")
+            .withSystemConfigs(Collections.emptyMap())
+            .withDefaultStreamConfigs(Collections.emptyMap());
+
+    DoubleSerde streamSerde = new DoubleSerde();
+    GenericInputDescriptor<Double> isd = mySystem.getInputDescriptor("input-stream", streamSerde);
+
+    Map<String, String> generatedConfigs = isd.toConfig();
+    Map<String, String> expectedConfigs = ImmutableMap.of("streams.input-stream.samza.system", "input-system");
+    assertEquals(expectedConfigs, generatedConfigs);
+    assertEquals(streamSerde, isd.getSerde());
+    assertFalse(isd.getTransformer().isPresent());
+  }
+
+  @Test
+  public void testISDObjectsWithOverrides() {
+    GenericSystemDescriptor mySystem =
+        new GenericSystemDescriptor("input-system", "factory.class.name")
+            .withSystemConfigs(Collections.emptyMap())
+            .withDefaultStreamConfigs(Collections.emptyMap());
+    IntegerSerde streamSerde = new IntegerSerde();
+    GenericInputDescriptor<Integer> isd = mySystem.getInputDescriptor("input-stream", streamSerde);
+
+    assertEquals(streamSerde, isd.getSerde());
+    assertFalse(isd.getTransformer().isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericSystemDescriptor.java
new file mode 100644
index 0000000..b92a904
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestGenericSystemDescriptor.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestGenericSystemDescriptor {
+  @Test
+  public void testSDConfigs() {
+    GenericSystemDescriptor mySystem =
+        new GenericSystemDescriptor("input-system", "factory.class.name")
+            .withSystemConfigs(ImmutableMap.of("custom-config-key", "custom-config-value"))
+            .withDefaultStreamConfigs(ImmutableMap.of("custom-stream-config-key", "custom-stream-config-value"))
+            .withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.UPCOMING);
+
+    Map<String, String> generatedConfigs = mySystem.toConfig();
+    Map<String, String> expectedConfigs = ImmutableMap.of(
+        "systems.input-system.samza.factory", "factory.class.name",
+        "systems.input-system.custom-config-key", "custom-config-value",
+        "systems.input-system.default.stream.custom-stream-config-key", "custom-stream-config-value",
+        "systems.input-system.default.stream.samza.offset.default", "upcoming"
+    );
+    assertEquals(expectedConfigs, generatedConfigs);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetInputDescriptorWithNullSerde() {
+    GenericSystemDescriptor mySystem = new GenericSystemDescriptor("input-system", "factory.class.name");
+    mySystem.getInputDescriptor("streamId", null); // should throw an exception
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetSystemDescriptorWithNullSystemName() {
+    new GenericSystemDescriptor(null, "factory.class.name");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetSystemDescriptorWithEmptySystemName() {
+    new GenericSystemDescriptor(" ", "factory.class.name");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/TestSimpleInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/TestSimpleInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestSimpleInputDescriptor.java
new file mode 100644
index 0000000..8b18fc5
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestSimpleInputDescriptor.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import java.util.Collections;
+import org.apache.samza.system.descriptors.examples.serde.ExampleSimpleInputDescriptor;
+import org.apache.samza.system.descriptors.examples.serde.ExampleSimpleOutputDescriptor;
+import org.apache.samza.system.descriptors.examples.serde.ExampleSimpleSystemDescriptor;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestSimpleInputDescriptor {
+  @Test
+  public void testAPIUsage() {
+    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
+    // and validates that the method calls can be chained.
+    ExampleSimpleSystemDescriptor kafkaSystem =
+        new ExampleSimpleSystemDescriptor("kafka-system")
+            .withSystemConfigs(Collections.emptyMap());
+    ExampleSimpleInputDescriptor<Integer> input1 = kafkaSystem.getInputDescriptor("input1", new IntegerSerde());
+    ExampleSimpleOutputDescriptor<Integer> output1 = kafkaSystem.getOutputDescriptor("output1", new IntegerSerde());
+
+    input1
+        .shouldBootstrap()
+        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
+        .withPriority(1)
+        .shouldResetOffset()
+        .withStreamConfigs(Collections.emptyMap());
+
+    output1
+        .withStreamConfigs(Collections.emptyMap());
+  }
+
+  @Test
+  public void testISDObjectsWithOverrides() {
+    ExampleSimpleSystemDescriptor ssd = new ExampleSimpleSystemDescriptor("kafka-system");
+    IntegerSerde streamSerde = new IntegerSerde();
+    ExampleSimpleInputDescriptor<Integer> isd = ssd.getInputDescriptor("input-stream", streamSerde);
+
+    assertEquals(streamSerde, isd.getSerde());
+    assertFalse(isd.getTransformer().isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/TestTransformingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/TestTransformingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestTransformingInputDescriptor.java
new file mode 100644
index 0000000..6d92d6c
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/TestTransformingInputDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors;
+
+import java.util.Collections;
+import org.apache.samza.system.descriptors.examples.transforming.ExampleTransformingInputDescriptor;
+import org.apache.samza.system.descriptors.examples.transforming.ExampleTransformingOutputDescriptor;
+import org.apache.samza.system.descriptors.examples.transforming.ExampleTransformingSystemDescriptor;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTransformingInputDescriptor {
+  @Test
+  public void testAPIUsage() {
+    // does not assert anything, but acts as a compile-time check on expected descriptor type parameters
+    // and validates that the method calls can be chained.
+    ExampleTransformingSystemDescriptor imeTransformingSystem =
+        new ExampleTransformingSystemDescriptor("imeTransformingSystem")
+            .withSystemConfigs(Collections.emptyMap());
+    ExampleTransformingInputDescriptor<Long> input1 = imeTransformingSystem.getInputDescriptor("input1", new IntegerSerde());
+    ExampleTransformingOutputDescriptor<Integer> output1 = imeTransformingSystem.getOutputDescriptor("output1", new IntegerSerde());
+
+    input1
+        .shouldBootstrap()
+        .withOffsetDefault(SystemStreamMetadata.OffsetType.NEWEST)
+        .withPriority(1)
+        .shouldResetOffset()
+        .withStreamConfigs(Collections.emptyMap());
+
+    output1
+        .withStreamConfigs(Collections.emptyMap());
+  }
+
+  @Test
+  public void testISDObjectsWithOverrides() {
+    ExampleTransformingSystemDescriptor imeTransformingSystem =
+        new ExampleTransformingSystemDescriptor("imeTransformingSystem");
+    IntegerSerde streamSerde = new IntegerSerde();
+    ExampleTransformingInputDescriptor<Long> overridingISD =
+        imeTransformingSystem.getInputDescriptor("input-stream", streamSerde);
+
+    assertEquals(streamSerde, overridingISD.getSerde());
+    assertEquals(imeTransformingSystem.getTransformer().get(), overridingISD.getTransformer().get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingInputDescriptor.java
new file mode 100644
index 0000000..fdc87f8
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingInputDescriptor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors.examples.expanding;
+
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleExpandingInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, ExampleExpandingInputDescriptor<StreamMessageType>> {
+  ExampleExpandingInputDescriptor(String streamId, SystemDescriptor systemDescriptor, InputTransformer<StreamMessageType> transformer, Serde serde) {
+    super(streamId, serde, systemDescriptor, transformer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingOutputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingOutputDescriptor.java
new file mode 100644
index 0000000..3f493b3
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingOutputDescriptor.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.system.descriptors.examples.expanding;
+
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleExpandingOutputDescriptor<StreamMessageType> extends OutputDescriptor<StreamMessageType, ExampleExpandingOutputDescriptor<StreamMessageType>> {
+  ExampleExpandingOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
+    super(streamId, serde, systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingSystemDescriptor.java
new file mode 100644
index 0000000..6ab1c8b
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/expanding/ExampleExpandingSystemDescriptor.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors.examples.expanding;
+
+import org.apache.samza.system.descriptors.ExpandingInputDescriptorProvider;
+import org.apache.samza.system.descriptors.OutputDescriptorProvider;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.IncomingMessageEnvelope;
+
+public class ExampleExpandingSystemDescriptor extends SystemDescriptor<ExampleExpandingSystemDescriptor>
+    implements ExpandingInputDescriptorProvider<Long>, OutputDescriptorProvider {
+  private static final String FACTORY_CLASS_NAME = "org.apache.samza.GraphExpandingSystemFactory";
+
+  public ExampleExpandingSystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME,
+        (InputTransformer<String>) IncomingMessageEnvelope::toString,
+        (streamGraph, inputDescriptor) -> (MessageStream<Long>) streamGraph.getInputStream(inputDescriptor)
+    );
+  }
+
+  @Override
+  public ExampleExpandingInputDescriptor<Long> getInputDescriptor(String streamId, Serde serde) {
+    return new ExampleExpandingInputDescriptor<>(streamId, this, null, serde);
+  }
+
+  @Override
+  public <StreamMessageType> ExampleExpandingOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
+    return new ExampleExpandingOutputDescriptor<>(streamId, this, serde);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleInputDescriptor.java
new file mode 100644
index 0000000..203982a
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleInputDescriptor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors.examples.serde;
+
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleSimpleInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, ExampleSimpleInputDescriptor<StreamMessageType>> {
+  ExampleSimpleInputDescriptor(String streamId, SystemDescriptor systemDescriptor, InputTransformer<StreamMessageType> transformer, Serde serde) {
+    super(streamId, serde, systemDescriptor, transformer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleOutputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleOutputDescriptor.java
new file mode 100644
index 0000000..4e18981
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleOutputDescriptor.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.system.descriptors.examples.serde;
+
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleSimpleOutputDescriptor<StreamMessageType> extends OutputDescriptor<StreamMessageType, ExampleSimpleOutputDescriptor<StreamMessageType>> {
+  ExampleSimpleOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
+    super(streamId, serde, systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleSystemDescriptor.java
new file mode 100644
index 0000000..65dd042
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/serde/ExampleSimpleSystemDescriptor.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors.examples.serde;
+
+import org.apache.samza.system.descriptors.OutputDescriptorProvider;
+import org.apache.samza.system.descriptors.SimpleInputDescriptorProvider;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleSimpleSystemDescriptor extends SystemDescriptor<ExampleSimpleSystemDescriptor>
+    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
+  private static final String FACTORY_CLASS_NAME = "org.apache.kafka.KafkaSystemFactory";
+
+  public ExampleSimpleSystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME, null, null);
+  }
+
+  @Override
+  public <StreamMessageType> ExampleSimpleInputDescriptor<StreamMessageType> getInputDescriptor(String streamId, Serde<StreamMessageType> serde) {
+    return new ExampleSimpleInputDescriptor<>(streamId, this, null, serde);
+  }
+
+  @Override
+  public <StreamMessageType> ExampleSimpleOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
+    return new ExampleSimpleOutputDescriptor<>(streamId, this, serde);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingInputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingInputDescriptor.java
new file mode 100644
index 0000000..edc285d
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingInputDescriptor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors.examples.transforming;
+
+import org.apache.samza.system.descriptors.InputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.InputTransformer;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleTransformingInputDescriptor<StreamMessageType> extends InputDescriptor<StreamMessageType, ExampleTransformingInputDescriptor<StreamMessageType>> {
+  ExampleTransformingInputDescriptor(String streamId, SystemDescriptor systemDescriptor, InputTransformer<StreamMessageType> transformer, Serde serde) {
+    super(streamId, serde, systemDescriptor, transformer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingOutputDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingOutputDescriptor.java
new file mode 100644
index 0000000..31761ab
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingOutputDescriptor.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.system.descriptors.examples.transforming;
+
+import org.apache.samza.system.descriptors.OutputDescriptor;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleTransformingOutputDescriptor<StreamMessageType> extends OutputDescriptor<StreamMessageType, ExampleTransformingOutputDescriptor<StreamMessageType>> {
+  ExampleTransformingOutputDescriptor(String streamId, SystemDescriptor systemDescriptor, Serde<StreamMessageType> serde) {
+    super(streamId, serde, systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingSystemDescriptor.java b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingSystemDescriptor.java
new file mode 100644
index 0000000..8f7cfac
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/descriptors/examples/transforming/ExampleTransformingSystemDescriptor.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system.descriptors.examples.transforming;
+
+import org.apache.samza.system.descriptors.OutputDescriptorProvider;
+import org.apache.samza.system.descriptors.SystemDescriptor;
+import org.apache.samza.system.descriptors.TransformingInputDescriptorProvider;
+import org.apache.samza.serializers.Serde;
+
+public class ExampleTransformingSystemDescriptor extends SystemDescriptor<ExampleTransformingSystemDescriptor>
+    implements TransformingInputDescriptorProvider<Long>, OutputDescriptorProvider {
+  private static final String FACTORY_CLASS_NAME = "org.apache.samza.IMETransformingSystemFactory";
+
+  public ExampleTransformingSystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME, ime -> 1L, null);
+  }
+
+  @Override
+  public ExampleTransformingInputDescriptor<Long> getInputDescriptor(String streamId, Serde serde) {
+    return new ExampleTransformingInputDescriptor<>(streamId, this, null, serde);
+  }
+
+  @Override
+  public <StreamMessageType> ExampleTransformingOutputDescriptor<StreamMessageType> getOutputDescriptor(String streamId, Serde<StreamMessageType> serde) {
+    return new ExampleTransformingOutputDescriptor<>(streamId, this, serde);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/74675cea/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsInputDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsInputDescriptor.java
deleted file mode 100644
index d151d16..0000000
--- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubsInputDescriptor.java
+++ /dev/null
@@ -1,121 +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.system.eventhub;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.config.ConfigException;
-import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
-import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
-import org.apache.samza.serializers.Serde;
-
-
-/**
- * A descriptor for the Event Hubs output stream
- *<p>
- *   An instance of this descriptor may be obtained from an {@link EventHubsSystemDescriptor}
- *</p>
- * Stream properties configured using a descriptor overrides corresponding properties and property defaults provided
- * in configuration.
- *
- * @param <StreamMessageType> type of messages in this stream
- */
-public class EventHubsInputDescriptor<StreamMessageType>
-    extends InputDescriptor<StreamMessageType, EventHubsInputDescriptor<StreamMessageType>> {
-  private String namespace;
-  private String entityPath;
-  private Optional<String> sasKeyName = Optional.empty();
-  private Optional<String> sasToken = Optional.empty();
-  private Optional<String> consumerGroup = Optional.empty();
-
-  /**
-   * Constructs an {@link InputDescriptor} instance.
-   *
-   * @param streamId id of the stream
-   * @param namespace namespace for the Event Hubs entity to consume from, not null
-   * @param entityPath entity path for the Event Hubs entity to consume from, not null
-   * @param serde serde for messages in the stream
-   * @param systemDescriptor system descriptor this stream descriptor was obtained from
-   */
-  EventHubsInputDescriptor(String streamId, String namespace, String entityPath, Serde serde,
-      SystemDescriptor systemDescriptor) {
-    super(streamId, serde, systemDescriptor, null);
-    this.namespace = StringUtils.stripToNull(namespace);
-    this.entityPath = StringUtils.stripToNull(entityPath);
-    if (this.namespace == null || this.entityPath == null) {
-      throw new ConfigException(String.format("Missing namespace and entity path Event Hubs input descriptor in " //
-          + "system: {%s}, stream: {%s}", getSystemName(), streamId));
-    }
-  }
-
-  /**
-   * SAS Key name of the associated input stream. Required to access the input Event Hubs entity per stream.
-   *
-   * @param sasKeyName the name of the SAS key required to access the Event Hubs entity
-   * @return this input descriptor
-   */
-  public EventHubsInputDescriptor<StreamMessageType> withSasKeyName(String sasKeyName) {
-    this.sasKeyName = Optional.of(StringUtils.stripToNull(sasKeyName));
-    return this;
-  }
-
-  /**
-   * SAS Token of the associated input stream. Required to access the input Event Hubs per stream.
-   *
-   * @param sasToken the SAS token required to access the Event Hubs entity
-   * @return this input descriptor
-   */
-  public EventHubsInputDescriptor<StreamMessageType> withSasKey(String sasToken) {
-    this.sasToken = Optional.of(StringUtils.stripToNull(sasToken));
-    return this;
-  }
-
-  /**
-   * Set the consumer group from the upstream Event Hubs entity that the consumer is part of. Defaults to the
-   * <code>$Default</code> group that is initially present in all Event Hubs entities (unless removed)
-   *
-   * @param consumerGroup the name of the consumer group upstream
-   * @return this input descriptor
-   */
-  public EventHubsInputDescriptor<StreamMessageType> withConsumerGroup(String consumerGroup) {
-    this.consumerGroup = Optional.of(StringUtils.stripToNull(consumerGroup));
-    return this;
-  }
-
-  @Override
-  public Map<String, String> toConfig() {
-    HashMap<String, String> ehConfigs = new HashMap<>(super.toConfig());
-
-    String streamId = getStreamId();
-
-    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, streamId), namespace);
-    ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, streamId), entityPath);
-
-    sasKeyName.ifPresent(keyName ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, streamId), keyName));
-    sasToken.ifPresent(key ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, streamId), key));
-    this.consumerGroup.ifPresent(consumerGroupName ->
-        ehConfigs.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_GROUP, streamId), consumerGroupName));
-    return ehConfigs;
-  }
-
-}