http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/api/programming-model.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/api/programming-model.md 
b/docs/learn/documentation/versioned/api/programming-model.md
index 1c9bd1c..efdcfa3 100644
--- a/docs/learn/documentation/versioned/api/programming-model.md
+++ b/docs/learn/documentation/versioned/api/programming-model.md
@@ -18,74 +18,81 @@ title: Programming Model
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-# Introduction
-Samza provides different sets of programming APIs to meet requirements from 
different sets of users. The APIs are listed below:
+### Introduction
+Samza provides multiple programming APIs to fit your use case:
 
-1. Java programming APIs: Samza provides Java programming APIs for users who 
are familiar with imperative programming languages. The overall programming 
model to create a Samza application in Java will be described here. Samza also 
provides two sets of APIs to describe user processing logic:
-    1. [High-level API](high-level-api.md): this API allows users to describe 
the end-to-end stream processing pipeline in a connected DAG (Directional 
Acyclic Graph). It also provides a rich set of build-in operators to help users 
implementing common transformation logic, such as filter, map, join, and window.
-    2. [Task API](low-level-api.md): this is low-level Java API which provides 
“bare-metal” programming interfaces to the users. Task API allows users to 
explicitly access physical implementation details in the system, such as 
accessing the physical system stream partition of an incoming message and 
explicitly controlling the thread pool to execute asynchronous processing 
method.
-2. [Samza SQL](samza-sql.md): Samza provides SQL for users who are familiar 
with declarative query languages, which allows the users to focus on data 
manipulation via SQL predicates and UDFs, not the physical implementation 
details.
-3. Beam API: Samza also provides a [Beam 
runner](https://beam.apache.org/documentation/runners/capability-matrix/) to 
run applications written in Beam API. This is considered as an extension to 
existing operators supported by the high-level API in Samza.
+1. Java APIs: Samza's provides two Java programming APIs that are ideal for 
building advanced Stream Processing applications. 
+    1. [High Level Streams API](high-level-api.md): Samza's flexible High 
Level Streams API lets you describe your complex stream processing pipeline in 
the form of a Directional Acyclic Graph (DAG) of operations on message streams. 
It provides a rich set of built-in operators that simplify common stream 
processing operations such as filtering, projection, repartitioning, joins, and 
windows.
+    2. [Low Level Task API](low-level-api.md): Samza's powerful Low Level Task 
API lets you write your application in terms of processing logic for each 
incoming message. 
+2. [Samza SQL](samza-sql.md): Samza SQL provides a declarative query language 
for describing your stream processing logic. It lets you manipulate streams 
using SQL predicates and UDFs instead of working with the physical 
implementation details.
+3. Apache Beam API: Samza also provides a [Apache Beam 
runner](https://beam.apache.org/documentation/runners/capability-matrix/) to 
run applications written using the Apache Beam API. This is considered as an 
extension to the operators supported by the High Level Streams API in Samza.
 
-The following sections will be focused on Java programming APIs.
-
-# Key Concepts for a Samza Java Application
-To write a Samza Java application, you will typically follow the steps below:
-1. Define your input and output streams and tables
-2. Define your main processing logic
 
+### Key Concepts
 The following sections will talk about key concepts in writing your Samza 
applications in Java.
 
-## Samza Applications
-When writing your stream processing application using Java API in Samza, you 
implement either a 
[StreamApplication](javadocs/org/apache/samza/application/StreamApplication.html)
 or 
[TaskApplication](javadocs/org/apache/samza/application/TaskApplication.html) 
and define your processing logic in the describe method.
-- For StreamApplication:
+#### Samza Applications
+A 
[SamzaApplication](javadocs/org/apache/samza/application/SamzaApplication.html) 
describes the inputs, outputs, state, configuration and the logic for 
processing data from one or more streaming sources. 
+
+You can implement a 
+[StreamApplication](javadocs/org/apache/samza/application/StreamApplication.html)
 and use the provided 
[StreamApplicationDescriptor](javadocs/org/apache/samza/application/descriptors/StreamApplicationDescriptor)
 to describe the processing logic using Samza's High Level Streams API in terms 
of [MessageStream](javadocs/org/apache/samza/operators/MessageStream.html) 
operators. 
 
 {% highlight java %}
-    
-    public void describe(StreamApplicationDescriptor appDesc) { … }
+
+    public class MyStreamApplication implements StreamApplication {
+        @Override
+        public void describe(StreamApplicationDescriptor appDesc) {
+            // Describe your application here 
+        }
+    }
 
 {% endhighlight %}
+
+Alternatively, you can implement a 
[TaskApplication](javadocs/org/apache/samza/application/TaskApplication.html) 
and use the provided 
[TaskApplicationDescriptor](javadocs/org/apache/samza/application/descriptors/TaskApplicationDescriptor)
 to describe it using Samza's Low Level API in terms of per-message processing 
logic.
+
+
 - For TaskApplication:
 
 {% highlight java %}
     
-    public void describe(TaskApplicationDescriptor appDesc) { … }
+    public class MyTaskApplication implements TaskApplication {
+        @Override
+        public void describe(TaskApplicationDescriptor appDesc) {
+            // Describe your application here
+        }
+    }
 
 {% endhighlight %}
 
-## Descriptors for Data Streams and Tables
-There are three different types of descriptors in Samza: 
[InputDescriptor](javadocs/org/apache/samza/system/descriptors/InputDescriptor.html),
 
[OutputDescriptor](javadocs/org/apache/samza/system/descriptors/OutputDescriptor.html),
 and 
[TableDescriptor](javadocs/org/apache/samza/table/descriptors/TableDescriptor.html).
 The InputDescriptor and OutputDescriptor are used to describe the physical 
sources and destinations of a stream, while a TableDescriptor is used to 
describe the physical dataset and IO functions for a table.
-Usually, you will obtain InputDescriptor and OutputDescriptor from a 
[SystemDescriptor](javadocs/org/apache/samza/system/descriptors/SystemDescriptor.html),
 which include all information about producer and consumers to a physical 
system. The following code snippet illustrate how you will obtain 
InputDescriptor and OutputDescriptor from a SystemDescriptor.
 
-{% highlight java %}
-    
-    public class BadPageViewFilter implements StreamApplication {
-      @Override
-      public void describe(StreamApplicationDescriptor appDesc) {
-        KafkaSystemDescriptor kafka = new KafkaSystemDescriptor();
-        InputDescriptor<PageView> pageViewInput = 
kafka.getInputDescriptor(“page-views”, new JsonSerdeV2<>(PageView.class));
-        OutputDescriptor<DecoratedPageView> pageViewOutput = 
kafka.getOutputDescriptor(“decorated-page-views”, new 
JsonSerdeV2<>(DecoratedPageView.class));
+#### Streams and Table Descriptors
+Descriptors let you specify the properties of various aspects of your 
application from within it. 
 
-        // Now, implement your main processing logic
-      }
-    }
-    
-{% endhighlight %}
+[InputDescriptor](javadocs/org/apache/samza/system/descriptors/InputDescriptor.html)s
 and 
[OutputDescriptor](javadocs/org/apache/samza/system/descriptors/OutputDescriptor.html)s
 can be used for specifying Samza and implementation-specific properties of the 
streaming inputs and outputs for your application. You can obtain 
InputDescriptors and OutputDescriptors using a 
[SystemDescriptor](javadocs/org/apache/samza/system/descriptors/SystemDescriptor.html)
 for your system. This SystemDescriptor can be used for specify Samza and 
implementation-specific properties of the producer and consumers for your I/O 
system. Most Samza system implementations come with their own 
SystemDescriptors, but if one isn't available, you 
+can use the 
[GenericSystemDescriptor](javadocs/org/apache/samza/system/descriptors/GenericSystemDescriptor.html).
 
-You can also add a TableDescriptor to your application.
+A 
[TableDescriptor](javadocs/org/apache/samza/table/descriptors/TableDescriptor.html)
 can be used for specifying Samza and implementation-specific properties of a 
[Table](javadocs/org/apache/samza/table/Table.html). You can use a Local 
TableDescriptor (e.g. 
[RocksDbTableDescriptor](javadocs/org/apache/samza/storage/kv/descriptors/RocksDbTableDescriptor.html)
 or a 
[RemoteTableDescriptor](javadocs/org/apache/samza/table/descriptors/RemoteTableDescriptor).
+
+
+The following example illustrates how you can use input and output descriptors 
for a Kafka system, and a table descriptor for a local RocksDB table within 
your application:
 
 {% highlight java %}
-     
-    public class BadPageViewFilter implements StreamApplication {
+    
+    public class MyStreamApplication implements StreamApplication {
       @Override
-      public void describe(StreamApplicationDescriptor appDesc) {
-        KafkaSystemDescriptor kafka = new KafkaSystemDescriptor();
-        InputDescriptor<PageView> pageViewInput = 
kafka.getInputDescriptor(“page-views”, new JsonSerdeV2<>(PageView.class));
-        OutputDescriptor<DecoratedPageView> pageViewOutput = 
kafka.getOutputDescriptor(“decorated-page-views”, new 
JsonSerdeV2<>(DecoratedPageView.class));
-        TableDescriptor<String, Integer> viewCountTable = new 
RocksDBTableDescriptor(
-            “pageViewCountTable”, KVSerde.of(new StringSerde(), new 
IntegerSerde()));
-
-        // Now, implement your main processing logic
+      public void describe(StreamApplicationDescriptor appDescriptor) {
+        KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("kafka")
+            .withConsumerZkConnect(ImmutableList.of("..."))
+            .withProducerBootstrapServers(ImmutableList.of("...", "..."));
+        KafkaInputDescriptor<PageView> kid = 
+            ksd.getInputDescriptor(“page-views”, new 
JsonSerdeV2<>(PageView.class));
+        KafkaOutputDescriptor<DecoratedPageView> kod = 
+            ksd.getOutputDescriptor(“decorated-page-views”, new 
JsonSerdeV2<>(DecoratedPageView.class));
+
+        RocksDbTableDescriptor<String, Integer> td = 
+            new RocksDbTableDescriptor(“viewCounts”, KVSerde.of(new 
StringSerde(), new IntegerSerde()));
+            
+        // Implement your processing logic here
       }
     }
     
@@ -93,21 +100,21 @@ You can also add a TableDescriptor to your application.
 
 The same code in the above describe method applies to TaskApplication as well.
 
-## Stream Processing Logic
+#### Stream Processing Logic
 
-Samza provides two sets of APIs to define the main stream processing logic, 
high-level API and Task API, via StreamApplication and TaskApplication, 
respectively. 
+Samza provides two sets of APIs to define the main stream processing logic, 
High Level Streams API and Low Level Task API, via StreamApplication and 
TaskApplication, respectively. 
 
-High-level API allows you to describe the processing logic in a connected DAG 
of transformation operators, like the example below:
+High Level Streams API allows you to describe the processing logic in a 
connected DAG of transformation operators, like the example below:
 
 {% highlight java %}
 
     public class BadPageViewFilter implements StreamApplication {
       @Override
       public void describe(StreamApplicationDescriptor appDesc) {
-        KafkaSystemDescriptor kafka = new KafkaSystemDescriptor();
+        KafkaSystemDescriptor ksd = new KafkaSystemDescriptor();
         InputDescriptor<PageView> pageViewInput = 
kafka.getInputDescriptor(“page-views”, new JsonSerdeV2<>(PageView.class));
         OutputDescriptor<DecoratedPageView> pageViewOutput = 
kafka.getOutputDescriptor(“decorated-page-views”, new 
JsonSerdeV2<>(DecoratedPageView.class));
-        TableDescriptor<String, Integer> viewCountTable = new 
RocksDBTableDescriptor(
+        RocksDbTableDescriptor<String, Integer> viewCountTable = new 
RocksDbTableDescriptor(
             “pageViewCountTable”, KVSerde.of(new StringSerde(), new 
IntegerSerde()));
 
         // Now, implement your main processing logic
@@ -120,7 +127,7 @@ High-level API allows you to describe the processing logic 
in a connected DAG of
     
 {% endhighlight %}
 
-Task API allows you to describe the processing logic in a customized 
StreamTaskFactory or AsyncStreamTaskFactory, like the example below:
+Low Level Task API allows you to describe the processing logic in a customized 
StreamTaskFactory or AsyncStreamTaskFactory, like the example below:
 
 {% highlight java %}
 
@@ -130,7 +137,7 @@ Task API allows you to describe the processing logic in a 
customized StreamTaskF
         KafkaSystemDescriptor kafka = new KafkaSystemDescriptor();
         InputDescriptor<PageView> pageViewInput = 
kafka.getInputDescriptor(“page-views”, new JsonSerdeV2<>(PageView.class));
         OutputDescriptor<DecoratedPageView> pageViewOutput = 
kafka.getOutputDescriptor(“decorated-page-views”, new 
JsonSerdeV2<>(DecoratedPageView.class));
-        TableDescriptor<String, Integer> viewCountTable = new 
RocksDBTableDescriptor(
+        RocksDbTableDescriptor<String, Integer> viewCountTable = new 
RocksDbTableDescriptor(
             “pageViewCountTable”, KVSerde.of(new StringSerde(), new 
IntegerSerde()));
 
         // Now, implement your main processing logic
@@ -142,11 +149,10 @@ Task API allows you to describe the processing logic in a 
customized StreamTaskF
     
 {% endhighlight %}
 
-Details for [high-level API](high-level-api.md) and [Task 
API](low-level-api.md) are explained later.
+#### Configuration for a Samza Application
 
-## Configuration for a Samza Application
+To deploy a Samza application, you need to specify the implementation class 
for your application and the ApplicationRunner to launch your application. The 
following is an incomplete example of minimum required configuration to set up 
the Samza application and the runner. For additional configuration, see the 
Configuration Reference.
 
-To deploy a Samza application, you will need to specify the implementation 
class for your application and the ApplicationRunner to launch your 
application. The following is an incomplete example of minimum required 
configuration to set up the Samza application and the runner:
 {% highlight jproperties %}
     
     # This is the class implementing StreamApplication

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/api/samza-sql.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/api/samza-sql.md 
b/docs/learn/documentation/versioned/api/samza-sql.md
index 13b059f..7412f6c 100644
--- a/docs/learn/documentation/versioned/api/samza-sql.md
+++ b/docs/learn/documentation/versioned/api/samza-sql.md
@@ -87,7 +87,7 @@ Note: Samza sql console right now doesn’t support queries 
that need state, for
 
 
 # Running Samza SQL on YARN
-The [hello-samza](https://github.com/apache/samza-hello-samza) project is an 
example project designed to help you run your first Samza application. It has 
examples of applications using the low level task API, high level API as well 
as Samza SQL.
+The [hello-samza](https://github.com/apache/samza-hello-samza) project is an 
example project designed to help you run your first Samza application. It has 
examples of applications using the Low Level Task API, High Level Streams API 
as well as Samza SQL.
 
 This tutorial demonstrates a simple Samza application that uses SQL to perform 
stream processing.
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/api/table-api.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/api/table-api.md 
b/docs/learn/documentation/versioned/api/table-api.md
index f5efa88..0a9c33c 100644
--- a/docs/learn/documentation/versioned/api/table-api.md
+++ b/docs/learn/documentation/versioned/api/table-api.md
@@ -181,7 +181,7 @@ join with a table and finally write the output to another 
table.
 
 # Using Table with Samza Low Level API
 
-The code snippet below illustrates the usage of table in Samza low level API.
+The code snippet below illustrates the usage of table in Samza Low Level Task 
API.
 
 {% highlight java %}
  1  class SamzaTaskApplication implements TaskApplication {
@@ -273,8 +273,7 @@ The table below summarizes table metrics:
 
 
[`RemoteTable`](https://github.com/apache/samza/blob/master/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java)
 
 provides a unified abstraction for Samza applications to access any remote 
data 
-store through stream-table join in high-level API or direct access in 
low-level 
-API. Remote Table is a store-agnostic abstraction that can be customized to 
+store through stream-table join in High Level Streams API or direct access in 
Low Level Task API. Remote Table is a store-agnostic abstraction that can be 
customized to 
 access new types of stores by writing pluggable I/O "Read/Write" functions, 
 implementations of 
 
[`TableReadFunction`](https://github.com/apache/samza/blob/master/samza-core/src/main/java/org/apache/samza/table/remote/TableReadFunction.java)
 and 
@@ -283,7 +282,7 @@ interfaces. Remote Table also provides common 
functionality, eg. rate limiting
 (built-in) and caching (hybrid).
 
 The async APIs in Remote Table are recommended over the sync versions for 
higher 
-throughput. They can be used with Samza with low-level API to achieve the 
maximum 
+throughput. They can be used with Samza with Low Level Task API to achieve the 
maximum 
 throughput. 
 
 Remote Tables are represented by class 
@@ -420,7 +419,7 @@ created during instantiation of Samza container.
 The life of a table goes through a few phases
 
 1. **Declaration** - at first one declares the table by creating a 
`TableDescriptor`. In both 
-   Samza high level and low level API, the `TableDescriptor` is registered 
with stream 
+   Samza High Level Streams API and Low Level Task API, the `TableDescriptor` 
is registered with stream 
    graph, internally converted to `TableSpec` and in return a reference to a 
`Table` 
    object is obtained that can participate in the building of the DAG.
 2. **Instantiation** - during planning stage, configuration is 
@@ -436,7 +435,7 @@ The life of a table goes through a few phases
    * In Samza high level API, all table instances can be retrieved from 
`TaskContext` using 
      table-id during initialization of a 
      [`InitableFunction`] 
(https://github.com/apache/samza/blob/master/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java).
-   * In Samza low level API, all table instances can be retrieved from 
`TaskContext` using 
+   * In Samza Low Level Task API, all table instances can be retrieved from 
`TaskContext` using 
      table-id during initialization of a 
    [`InitableTask`] 
(https://github.com/apache/samza/blob/master/samza-api/src/main/java/org/apache/samza/task/InitableTask.java).
 4. **Cleanup** - 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/connectors/eventhubs.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/connectors/eventhubs.md 
b/docs/learn/documentation/versioned/connectors/eventhubs.md
index 0be288b..9fdc861 100644
--- a/docs/learn/documentation/versioned/connectors/eventhubs.md
+++ b/docs/learn/documentation/versioned/connectors/eventhubs.md
@@ -126,7 +126,7 @@ In this section, we will walk through a simple pipeline 
that reads from one Even
 4    MessageStream<KV<String, String>> eventhubInput = 
appDescriptor.getInputStream(inputDescriptor);
 5    OutputStream<KV<String, String>> eventhubOutput = 
appDescriptor.getOutputStream(outputDescriptor);
 
-    // Define the execution flow with the high-level API
+    // Define the execution flow with the High Level Streams API
 6    eventhubInput
 7        .map((message) -> {
 8          System.out.println("Received Key: " + message.getKey());

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/connectors/kafka.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/connectors/kafka.md 
b/docs/learn/documentation/versioned/connectors/kafka.md
index b71c736..9628130 100644
--- a/docs/learn/documentation/versioned/connectors/kafka.md
+++ b/docs/learn/documentation/versioned/connectors/kafka.md
@@ -24,9 +24,9 @@ Samza offers built-in integration with Apache Kafka for 
stream processing. A com
 
 The `hello-samza` project includes multiple examples on interacting with Kafka 
from your Samza jobs. Each example also includes instructions on how to run 
them and view results. 
 
-- [High-level API 
Example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/FilterExample.java)
 with a corresponding 
[tutorial](/learn/documentation/{{site.version}}/deployment/yarn.html#starting-your-application-on-yarn)
+- [High Level Streams API 
Example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/FilterExample.java)
 with a corresponding 
[tutorial](/learn/documentation/{{site.version}}/deployment/yarn.html#starting-your-application-on-yarn)
 
-- [Low-level API 
Example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/wikipedia/task/application/WikipediaParserTaskApplication.java)
 with a corresponding 
[tutorial](https://github.com/apache/samza-hello-samza#hello-samza)
+- [Low Level Task API 
Example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/wikipedia/task/application/WikipediaParserTaskApplication.java)
 with a corresponding 
[tutorial](https://github.com/apache/samza-hello-samza#hello-samza)
 
 
 ### Concepts
@@ -105,7 +105,7 @@ The above example configures Samza to ignore checkpointed 
offsets for `page-view
 
  
 
-### Code walkthrough: High-level API
+### Code walkthrough: High Level Streams API
 
 In this section, we walk through a complete example that reads from a Kafka 
topic, filters a few messages and writes them to another topic.
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/core-concepts/core-concepts.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/core-concepts/core-concepts.md 
b/docs/learn/documentation/versioned/core-concepts/core-concepts.md
index 479ebcb..b69de3d 100644
--- a/docs/learn/documentation/versioned/core-concepts/core-concepts.md
+++ b/docs/learn/documentation/versioned/core-concepts/core-concepts.md
@@ -55,7 +55,7 @@ A stream can have multiple producers that write data to it 
and multiple consumer
 
 A stream is sharded into multiple partitions for scaling how its data is 
processed. Each _partition_ is an ordered, replayable sequence of records. When 
a message is written to a stream, it ends up in one its partitions. Each 
message in a partition is uniquely identified by an _offset_. 
 
-Samza supports for pluggable systems that can implement the stream 
abstraction. As an example, Kafka implements a stream as a topic while another 
database might implement a stream as a sequence of updates to its tables.
+Samza supports pluggable systems that can implement the stream abstraction. As 
an example, Kafka implements a stream as a topic while a database might 
implement a stream as a sequence of updates to its tables.
 
 ## Stream Application
 A _stream application_ processes messages from input streams, transforms them 
and emits results to an output stream or a database. It is built by chaining 
multiple operators, each of which take in one or more streams and transform 
them.
@@ -63,19 +63,19 @@ A _stream application_ processes messages from input 
streams, transforms them an
 
![diagram-medium](/img/{{site.version}}/learn/documentation/core-concepts/stream-application.png)
 
 Samza offers three top-level APIs to help you build your stream applications: 
<br/>
-1. The [Samza Streams 
DSL](/learn/documentation/{{site.version}}/api/high-level-api.html),  which 
offers several built-in operators like map, filter etc. This is the recommended 
API for most use-cases. <br/>
-2. The [low-level 
API](/learn/documentation/{{site.version}}/api/low-level-api.html), which 
allows greater flexibility to define your processing-logic and offers for 
greater control <br/>
+1. The [High Level Streams 
API](/learn/documentation/{{site.version}}/api/high-level-api.html),  which 
offers several built-in operators like map, filter etc. This is the recommended 
API for most use-cases. <br/>
+2. The [Low Level Task 
API](/learn/documentation/{{site.version}}/api/low-level-api.html), which 
allows greater flexibility to define your processing-logic and offers greater 
control <br/>
 3. [Samza SQL](/learn/documentation/{{site.version}}/api/samza-sql.html), 
which offers a declarative SQL interface to create your applications <br/>
 
 ## State
-Samza supports for both stateless and stateful stream processing. _Stateless 
processing_, as the name implies, does not retain any state associated with the 
current message after it has been processed. A good example of this is to 
filter an incoming stream of user-records by a field (eg:userId) and write the 
filtered messages to their own stream. 
+Samza supports for both stateless and stateful stream processing. _Stateless 
processing_, as the name implies, does not retain any state associated with the 
current message after it has been processed. A good example of this is 
filtering an incoming stream of user-records by a field (eg:userId) and writing 
the filtered messages to their own stream. 
 
-In contrast, _stateful processing_ requires you to record some state about a 
message even after processing it. Consider the example of counting the number 
of unique users to a website every five minutes. This requires you to record 
state about each user seen thus far, for deduping later. Samza offers a 
fault-tolerant, scalable state-store for this purpose.
+In contrast, _stateful processing_ requires you to record some state about a 
message even after processing it. Consider the example of counting the number 
of unique users to a website every five minutes. This requires you to store 
information about each user seen thus far for de-duplication. Samza offers a 
fault-tolerant, scalable state-store for this purpose.
 
 ## Time
-Time is a fundamental concept in stream processing, especially how it is 
modeled and interpreted by the system. Samza supports two notions of dealing 
with time. By default, all built-in Samza operators use processing time. In 
processing time, the timestamp of a message is determined by when it is 
processed by the system. For example, an event generated by a sensor could be 
processed by Samza several milliseconds later. 
+Time is a fundamental concept in stream processing, especially in how it is 
modeled and interpreted by the system. Samza supports two notions of time. By 
default, all built-in Samza operators use processing time. In processing time, 
the timestamp of a message is determined by when it is processed by the system. 
For example, an event generated by a sensor could be processed by Samza several 
milliseconds later. 
 
-On the other hand, in event time, the timestamp of an event is determined by 
when it actually occurred in the source. For example, a sensor which generates 
an event could embed the time of occurrence as a part of the event itself. 
Samza provides event-time based processing by its integration with [Apache 
BEAM](https://beam.apache.org/documentation/runners/samza/).
+On the other hand, in event time, the timestamp of an event is determined by 
when it actually occurred at the source. For example, a sensor which generates 
an event could embed the time of occurrence as a part of the event itself. 
Samza provides event-time based processing by its integration with [Apache 
BEAM](https://beam.apache.org/documentation/runners/samza/).
 
 ## Processing guarantee
 Samza supports at-least once processing. As the name implies, this ensures 
that each message in the input stream is processed by the system at-least once. 
This guarantees no data-loss even when there are failures, thereby making Samza 
a practical choice for building fault-tolerant applications.

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/hadoop/overview.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/hadoop/overview.md 
b/docs/learn/documentation/versioned/hadoop/overview.md
index 0820127..3d8caaa 100644
--- a/docs/learn/documentation/versioned/hadoop/overview.md
+++ b/docs/learn/documentation/versioned/hadoop/overview.md
@@ -29,7 +29,7 @@ Samza provides a single set of APIs for both batch and stream 
processing. This u
 
 ### Multi-stage Batch Pipeline
 
-Complex data pipelines usually consist multiple stages, with data shuffled 
(repartitioned) between stages to enable key-based operations such as 
windowing, aggregation, and join. Samza [high-level 
API](/startup/preview/index.html) provides an operator named `partitionBy` to 
create such multi-stage pipelines. Internally, Samza creates a physical stream, 
called an “intermediate stream”, based on the system configured as in 
`job.default.system`. Samza repartitions the output of the previous stage by 
sending it to the intermediate stream with the appropriate partition count and 
partition key. It then feeds it to the next stage of the pipeline. The 
lifecycle of intermediate streams is completely managed by Samza so from the 
user perspective the data shuffling is automatic.
+Complex data pipelines usually consist multiple stages, with data shuffled 
(repartitioned) between stages to enable key-based operations such as 
windowing, aggregation, and join. Samza [High Level Streams 
API](/startup/preview/index.html) provides an operator named `partitionBy` to 
create such multi-stage pipelines. Internally, Samza creates a physical stream, 
called an “intermediate stream”, based on the system configured as in 
`job.default.system`. Samza repartitions the output of the previous stage by 
sending it to the intermediate stream with the appropriate partition count and 
partition key. It then feeds it to the next stage of the pipeline. The 
lifecycle of intermediate streams is completely managed by Samza so from the 
user perspective the data shuffling is automatic.
 
 For a single-stage pipeline, dealing with bounded data sets is 
straightforward: the system consumer “knows” the end of a particular 
partition, and it will emit end-of-stream token once a partition is complete. 
Samza will shut down the container when all its input partitions are complete.
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/index.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/index.html 
b/docs/learn/documentation/versioned/index.html
index 893e428..2d5446b 100644
--- a/docs/learn/documentation/versioned/index.html
+++ b/docs/learn/documentation/versioned/index.html
@@ -26,19 +26,15 @@ title: Documentation
 <h4>API</h4>
 
 <ul class="documentation-list">
-  <li><a href="api/low-level-api.html">Low-level API</a></li>
-  <li><a href="api/high-level-api.html">Streams DSL</a></li>
+  <li><a href="api/programming-model.html">API overview</a></li>
+  <li><a href="api/high-level-api.html">High Level Streams API</a></li>
+  <li><a href="api/low-level-api.html">Low Level Task API</a></li>
   <li><a href="api/table-api.html">Table API</a></li>
   <li><a href="api/samza-sql.html">Samza SQL</a></li>
   <li><a href="https://beam.apache.org/documentation/runners/samza/";>Apache 
BEAM</a></li>
-<!-- TODO comparisons pages
-  <li><a href="comparisons/aurora.html">Aurora</a></li>
-  <li><a href="comparisons/jms.html">JMS</a></li>
-  <li><a href="comparisons/s4.html">S4</a></li>
--->
 </ul>
 
-<h4>Deployment</h4>
+<h4>DEPLOYMENT</h4>
 
 <ul class="documentation-list">
   <li><a href="deployment/deployment-model.html">Deployment options</a></li>
@@ -46,7 +42,7 @@ title: Documentation
   <li><a href="deployment/standalone.html">Run as an embedded library</a></li>
 </ul>
 
-<h4>Connectors</h4>
+<h4>CONNECTORS</h4>
 
 <ul class="documentation-list">
   <li><a href="connectors/overview.html">Connectors overview</a></li>
@@ -56,7 +52,7 @@ title: Documentation
   <li><a href="connectors/kinesis.html">AWS Kinesis</a></li>
 </ul>
 
-<h4>Operations</h4>
+<h4>OPERATIONS</h4>
 
 <ul class="documentation-list">
   <li><a href="operations/monitoring.html">Monitoring</a></li>

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/jobs/samza-configurations.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/samza-configurations.md 
b/docs/learn/documentation/versioned/jobs/samza-configurations.md
index 5828969..e55446a 100644
--- a/docs/learn/documentation/versioned/jobs/samza-configurations.md
+++ b/docs/learn/documentation/versioned/jobs/samza-configurations.md
@@ -249,7 +249,7 @@ These properties define Samza's storage mechanism for 
efficient [stateful stream
 
 |Name|Default|Description|
 |--- |--- |--- |
-|stores.**_store-name_**.factory| |You can give a store any **_store-name_** 
except `default` (`default` is reserved for defining default store parameters), 
and use that name to get a reference to the store in your Samza application 
(call 
[TaskContext.getStore()](../api/javadocs/org/apache/samza/task/TaskContext.html#getStore(java.lang.String))
 in your task's 
[`init()`](../api/javadocs/org/apache/samza/task/InitableTask.html#init) method 
for the low-level API and in your application function's 
[`init()`](..api/javadocs/org/apache/samza/operators/functions/InitableFunction.html#init)
 method for the high level API. The value of this property is the 
fully-qualified name of a Java class that implements 
[`StorageEngineFactory`](../api/javadocs/org/apache/samza/storage/StorageEngineFactory.html).
 Samza currently ships with two storage engine implementations: 
<br><br>`org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory` 
<br>An on-disk storage engine with a key-value interface, 
 implemented using [RocksDB](http://rocksdb.org/). It supports fast 
random-access reads and writes, as well as range queries on keys. RocksDB can 
be configured with various additional tuning parameters.<br><br> 
`org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory`<br> 
In memory implementation of a key-value store. Uses 
`util.concurrent.ConcurrentSkipListMap` to store the keys in order.|
+|stores.**_store-name_**.factory| |You can give a store any **_store-name_** 
except `default` (`default` is reserved for defining default store parameters), 
and use that name to get a reference to the store in your Samza application 
(call 
[TaskContext.getStore()](../api/javadocs/org/apache/samza/task/TaskContext.html#getStore(java.lang.String))
 in your task's 
[`init()`](../api/javadocs/org/apache/samza/task/InitableTask.html#init) method 
for the Low Level Task API and in your application function's 
[`init()`](..api/javadocs/org/apache/samza/operators/functions/InitableFunction.html#init)
 method for the high level API. The value of this property is the 
fully-qualified name of a Java class that implements 
[`StorageEngineFactory`](../api/javadocs/org/apache/samza/storage/StorageEngineFactory.html).
 Samza currently ships with two storage engine implementations: 
<br><br>`org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory` 
<br>An on-disk storage engine with a key-value interf
 ace, implemented using [RocksDB](http://rocksdb.org/). It supports fast 
random-access reads and writes, as well as range queries on keys. RocksDB can 
be configured with various additional tuning parameters.<br><br> 
`org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory`<br> 
In memory implementation of a key-value store. Uses 
`util.concurrent.ConcurrentSkipListMap` to store the keys in order.|
 |stores.**_store-name_**.key.serde| |If the storage engine expects keys in the 
store to be simple byte arrays, this [serde](../container/serialization.html) 
allows the stream task to access the store using another object type as key. 
The value of this property must be a serde-name that is registered with 
serializers.registry.*.class. If this property is not set, keys are passed 
unmodified to the storage engine (and the changelog stream, if appropriate).|
 |stores.**_store-name_**.msg.serde| |If the storage engine expects values in 
the store to be simple byte arrays, this 
[serde](../container/serialization.html) allows the stream task to access the 
store using another object type as value. The value of this property must be a 
serde-name that is registered with serializers.registry.*.class. If this 
property is not set, values are passed unmodified to the storage engine (and 
the changelog stream, if appropriate).|
 |stores.**_store-name_**.changelog| |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 job.changelog.system 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).|

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/documentation/versioned/operations/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/operations/monitoring.md 
b/docs/learn/documentation/versioned/operations/monitoring.md
index d93e497..ec8430c 100644
--- a/docs/learn/documentation/versioned/operations/monitoring.md
+++ b/docs/learn/documentation/versioned/operations/monitoring.md
@@ -37,8 +37,8 @@ Like any other production software, it is critical to monitor 
the health of our
   + [A.3 Creating a Custom MetricsReporter](#customreporter)
 * [B. Metric Types in Samza](#metrictypes)
 * [C. Adding User-Defined Metrics](#userdefinedmetrics)
-  + [Low-level API](#lowlevelapi)
-  + [High-Level API](#highlevelapi)
+  + [Low Level Task API](#lowlevelapi)
+  + [High Level Streams API](#highlevelapi)
 * [D. Key Internal Samza Metrics](#keyinternalsamzametrics)
   + [D.1 Vital Metrics](#vitalmetrics)
   + [D.2 Store Metrics](#storemetrics)
@@ -232,7 +232,7 @@ To add a new metric, you can simply use the 
_MetricsRegistry_ in the provided Ta
 the init() method to register new metrics. The code snippets below show 
examples of registering and updating a user-defined
  Counter metric. Timers and gauges can similarly be used from within your task 
class.
 
-### <a name="lowlevelapi"></a> Low-level API
+### <a name="lowlevelapi"></a> Low Level Task API
 
 Simply have your task implement the InitableTask interface and access the 
MetricsRegistry from the TaskContext.
 
@@ -252,9 +252,9 @@ public class MyJavaStreamTask implements StreamTask, 
InitableTask {
 }
 ```
 
-### <a name="highlevelapi"></a> High-Level API
+### <a name="highlevelapi"></a> High Level Streams API
 
-In the high-level API, you can define a ContextManager and access the 
MetricsRegistry from the TaskContext, using which you can add and update your 
metrics.
+In the High Level Streams API, you can define a ContextManager and access the 
MetricsRegistry from the TaskContext, using which you can add and update your 
metrics.
 
 ```
 public class MyJavaStreamApp implements StreamApplication {

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/tutorials/versioned/hello-samza-high-level-code.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md 
b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md
index 1c06116..a881b51 100644
--- a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md
+++ b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md
@@ -35,7 +35,7 @@ cd hello-samza
 git checkout latest
 {% endhighlight %}
 
-This project already contains implementations of the wikipedia application 
using both the low-level task API and the high-level API. The low-level task 
implementations are in the `samza.examples.wikipedia.task` package. The 
high-level application implementation is in the 
`samza.examples.wikipedia.application` package.
+This project already contains implementations of the wikipedia application 
using both the Low Level Task API and the High Level Streams API. The Low Level 
Task API implementations are in the `samza.examples.wikipedia.task` package. 
The High Level Streams API implementation is in the 
`samza.examples.wikipedia.application` package.
 
 This tutorial will provide step by step instructions to recreate the existing 
wikipedia application.
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md 
b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
index cfe589a..5127fd6 100644
--- a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
+++ b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
@@ -18,9 +18,9 @@ title: Hello Samza High Level API - YARN Deployment
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-The [hello-samza](https://github.com/apache/samza-hello-samza) project is an 
example project designed to help you run your first Samza application. It has 
examples of applications using the low level task API as well as the high level 
API.
+The [hello-samza](https://github.com/apache/samza-hello-samza) project is an 
example project designed to help you run your first Samza application. It has 
examples of applications using the Low Level Task API as well as the High Level 
Streams API.
 
-This tutorial demonstrates a simple wikipedia application created with the 
high level API. The [Hello Samza tutorial] 
(/startup/hello-samza/{{site.version}}/index.html) is the low-level analog to 
this tutorial. It demonstrates the same logic but is created with the task API. 
The tutorials are designed to be as similar as possible. The primary 
differences are that with the high level API we accomplish the equivalent of 3 
separate low-level jobs with a single application, we skip the intermediate 
topics for simplicity, and we can visualize the execution plan after we start 
the application.
+This tutorial demonstrates a simple wikipedia application created with the 
High Level Streams API. The [Hello Samza tutorial] 
(/startup/hello-samza/{{site.version}}/index.html) is the Low Level Task API 
analog to this tutorial. It demonstrates the same logic but is created with the 
Low Level Task API. The tutorials are designed to be as similar as possible. 
The primary differences are that with the High Level Streams API we accomplish 
the equivalent of 3 separate Low Level Task API jobs with a single application, 
we skip the intermediate topics for simplicity, and we can visualize the 
execution plan after we start the application.
 
 ### Get the Code
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/tutorials/versioned/samza-event-hubs-standalone.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/samza-event-hubs-standalone.md 
b/docs/learn/tutorials/versioned/samza-event-hubs-standalone.md
index 53dbec3..b9fe533 100644
--- a/docs/learn/tutorials/versioned/samza-event-hubs-standalone.md
+++ b/docs/learn/tutorials/versioned/samza-event-hubs-standalone.md
@@ -18,7 +18,7 @@ title: Samza Event Hubs Connectors Example
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-The [hello-samza](https://github.com/apache/samza-hello-samza) project has an 
example that uses the Samza high-level API to consume and produce from [Event 
Hubs](../../documentation/versioned/connectors/eventhubs.html) using the 
Zookeeper deployment model.
+The [hello-samza](https://github.com/apache/samza-hello-samza) project has an 
example that uses the Samza High Level Streams API to consume and produce from 
[Event Hubs](../../documentation/versioned/connectors/eventhubs.html) using the 
Zookeeper deployment model.
 
 #### Get the Code
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/learn/tutorials/versioned/samza-sql.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/samza-sql.md 
b/docs/learn/tutorials/versioned/samza-sql.md
index f64aa06..71cfcc6 100644
--- a/docs/learn/tutorials/versioned/samza-sql.md
+++ b/docs/learn/tutorials/versioned/samza-sql.md
@@ -68,7 +68,7 @@ Below are some of the sql queries that you can execute using 
the samza-sql-conso
 
 # Running Samza SQL on YARN
 
-The [hello-samza](https://github.com/apache/samza-hello-samza) project is an 
example project designed to help you run your first Samza application. It has 
examples of applications using the low level task API, high level API as well 
as Samza SQL.
+The [hello-samza](https://github.com/apache/samza-hello-samza) project is an 
example project designed to help you run your first Samza application. It has 
examples of applications using the Low Level  Task API, High Level Streams API 
as well as Samza SQL.
 
 This tutorial demonstrates a simple Samza application that uses SQL to perform 
stream processing.
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/docs/startup/code-examples/versioned/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/code-examples/versioned/index.md 
b/docs/startup/code-examples/versioned/index.md
index ba1cc3e..384419d 100644
--- a/docs/startup/code-examples/versioned/index.md
+++ b/docs/startup/code-examples/versioned/index.md
@@ -36,7 +36,7 @@ These include:
 
 - The [Join 
example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/JoinExample.java])
 demonstrates how you can join a Kafka stream of page-views with a stream of 
ad-clicks
 
-- The [Stream-Table Join 
example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/RemoteTableJoinExample.java)
 demonstrates how the Samza Table API. It joins a Kafka stream with a remote 
dataset accessed through a REST service.
+- The [Stream-Table Join 
example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/RemoteTableJoinExample.java)
 demonstrates how to use the Samza Table API. It joins a Kafka stream with a 
remote dataset accessed through a REST service.
 
 - The 
[SessionWindow](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/SessionWindowExample.java)
 and 
[TumblingWindow](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/TumblingWindowExample.java)
 examples illustrate Samza's rich windowing and triggering capabilities.
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d034bbef/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java 
b/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
index 36003b3..148773f 100644
--- a/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
+++ b/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
@@ -19,12 +19,19 @@
 
 package org.apache.samza.task;
 
+import org.apache.samza.context.ApplicationContainerContext;
+import org.apache.samza.context.ApplicationTaskContext;
+
 /**
  * A ClosableTask augments {@link org.apache.samza.task.StreamTask}, allowing 
the method implementer to specify
  * code that will be called when the StreamTask is being shut down by the 
framework, providing to emit final metrics,
  * clean or close resources, etc.  The close method is not guaranteed to be 
called in event of crash or hard kill
  * of the process.
+ *
+ * Deprecated: It's recommended to manage the lifecycle of any runtime objects 
using
+ * {@link ApplicationContainerContext} and {@link ApplicationTaskContext} 
instead.
  */
+@Deprecated
 public interface ClosableTask {
   void close() throws Exception;
 }

Reply via email to