You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gearpump.apache.org by ma...@apache.org on 2017/07/28 02:20:58 UTC

[1/7] incubator-gearpump git commit: [GEARPUMP-311] refactor state management

Repository: incubator-gearpump
Updated Branches:
  refs/heads/state [created] db8abf99f


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
new file mode 100644
index 0000000..6d72e78
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state
+
+import java.io._
+import java.time.Instant
+import java.util
+import java.util.Map
+
+import com.google.common.collect.Table
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.refactor.coder._
+import org.apache.gearpump.streaming.refactor.state.api.StateInternals
+import org.apache.gearpump.streaming.refactor.state.heap.HeapStateInternalsFactory
+import org.apache.gearpump.streaming.state.impl.{CheckpointManager, PersistentStateConfig}
+import org.apache.gearpump.streaming.task.{Task, TaskContext, UpdateCheckpointClock}
+import org.apache.gearpump.streaming.transaction.api.CheckpointStoreFactory
+import org.apache.gearpump.util.LogUtil
+import org.apache.gearpump.{Message, TimeStamp}
+
+abstract class StatefulTask(taskContext: TaskContext, conf: UserConfig)
+  extends Task(taskContext, conf) {
+
+  import taskContext._
+
+  val checkpointStoreFactory = conf.getValue[CheckpointStoreFactory](
+    PersistentStateConfig.STATE_CHECKPOINT_STORE_FACTORY).get
+  val checkpointStore = checkpointStoreFactory.getCheckpointStore(
+    s"app$appId-task${taskId.processorId}_${taskId.index}")
+  val checkpointInterval = conf.getLong(PersistentStateConfig.STATE_CHECKPOINT_INTERVAL_MS).get
+  val checkpointManager = new CheckpointManager(checkpointInterval, checkpointStore)
+
+  var stateContext: RuntimeContext = null
+
+  var inited = false
+
+  // core state data
+  var encodedKeyStateMap: Map[String, Table[String, String, Array[Byte]]] = null
+
+  def open(runtimeContext: RuntimeContext): Unit = {}
+
+  def invoke(message: Message): Unit
+
+  def close(runtimeContext: RuntimeContext): Unit = {}
+
+  final override def onStart(startTime: Instant): Unit = {
+    // recover state from snapshot
+    LOG.info("[onStart] - recover from snapshot")
+    val timestamp = startTime.toEpochMilli
+    checkpointManager
+      .recover(timestamp)
+      .foreach(recoverState(timestamp, _))
+    reportCheckpointClock(timestamp)
+
+    inited = true
+    stateContext = new StreamingRuntimeContext(startTime)
+    open(stateContext)
+  }
+
+  final override def onNext(message: Message): Unit = {
+    checkpointManager.update(message.timestamp.toEpochMilli)
+    invoke(message)
+  }
+
+  override def onWatermarkProgress(watermark: Instant): Unit = {
+    if (checkpointManager.shouldCheckpoint(watermark.toEpochMilli)) {
+      checkpointManager.getCheckpointTime.foreach { checkpointTime =>
+        val serialized = snapshot
+        checkpointManager.checkpoint(checkpointTime, serialized)
+        reportCheckpointClock(checkpointTime)
+      }
+    }
+  }
+
+  final override def onStop(): Unit = {
+    LOG.info("[onStop] closing checkpoint manager")
+    close(stateContext)
+    checkpointManager.close()
+  }
+
+  private def reportCheckpointClock(timestamp: TimeStamp): Unit = {
+    LOG.debug("reportCheckpointClock at : {}", timestamp)
+    appMaster ! UpdateCheckpointClock(taskContext.taskId, timestamp)
+  }
+
+  private def recoverState(timestamp: TimeStamp, snapshot: Array[Byte]): Unit = {
+    LOG.info("call recoverState snapshot with timestamp : {}", timestamp)
+    var bis: Option[ByteArrayInputStream] = None
+    var oin: Option[ObjectInputStream] = None
+    try {
+      bis = Some(new ByteArrayInputStream(snapshot))
+      oin = Some(new ObjectInputStream(bis.get))
+      encodedKeyStateMap = oin.get.readObject().asInstanceOf[
+        Map[String, Table[String, String, Array[Byte]]]]
+    } catch {
+      case ex: IOException => throw new RuntimeException(ex)
+      case e: Exception => throw new RuntimeException(e)
+    } finally {
+      try {
+        if (oin.nonEmpty) oin.foreach(_.close())
+      } catch {
+        case ex: Exception => LOG.error("occurs exception when closing ObjectInputStream : {}", ex)
+      }
+      try {
+        if (bis.nonEmpty) bis.foreach(_.close())
+      } catch {
+        case ex: Exception =>
+          LOG.error("occurs exception when closing ByteArrayInputStream : {}", ex)
+      }
+    }
+  }
+
+  private def snapshot: Array[Byte] = {
+    LOG.info("do snapshot ")
+    var buffer: Option[ByteArrayOutputStream] = None
+    var oout: Option[ObjectOutputStream] = None
+    try {
+      buffer = Some(new ByteArrayOutputStream)
+      oout = Some(new ObjectOutputStream(buffer.get))
+      oout.get.writeObject(encodedKeyStateMap)
+      oout.get.flush()
+      buffer.get.toByteArray
+    } catch {
+      case ex: IOException => throw new RuntimeException(ex)
+      case e: Exception => throw new RuntimeException(e)
+    } finally {
+      try {
+        if (oout.nonEmpty) oout.foreach(_.close())
+      } catch {
+        case ex: Exception => LOG.error("occurs exception when closing ObjectOutputStream : {}", ex)
+      }
+      try {
+        if (buffer.nonEmpty) buffer.foreach(_.close())
+      } catch {
+        case ex: Exception =>
+          LOG.error("occurs exception when closing ByteArrayOutputStream : {}", ex)
+      }
+    }
+  }
+
+  private class StreamingRuntimeContext(startTime: Instant) extends RuntimeContext {
+
+    override def getStateInternals[KT](keyCoder: Coder[KT], key: KT): StateInternals = {
+      if (!inited) {
+        throw new RuntimeException(" please init state access object in `open` method! ")
+      }
+      if (encodedKeyStateMap == null) {
+        encodedKeyStateMap = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+      }
+
+      val factory = new HeapStateInternalsFactory[KT](keyCoder, encodedKeyStateMap)
+      factory.stateInternalsForKey(key)
+    }
+
+    override def getStartTime: Instant = startTime
+  }
+
+}


[5/7] incubator-gearpump git commit: [Gearpump 311] refactor state management

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuredCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuredCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuredCoder.java
new file mode 100644
index 0000000..3e299a6
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuredCoder.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.ByteArrayOutputStream;
+import java.util.Collections;
+import java.util.List;
+
+public abstract class StructuredCoder<T> extends Coder<T> {
+    protected StructuredCoder() {}
+
+    public List<? extends Coder<?>> getComponents() {
+        List<? extends Coder<?>> coderArguments = getCoderArguments();
+        if (coderArguments == null) {
+            return Collections.emptyList();
+        } else {
+            return coderArguments;
+        }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || this.getClass() != o.getClass()) {
+            return false;
+        }
+        StructuredCoder<?> that = (StructuredCoder<?>) o;
+        return this.getComponents().equals(that.getComponents());
+    }
+
+    @Override
+    public int hashCode() {
+        return getClass().hashCode() * 31 + getComponents().hashCode();
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        String s = getClass().getName();
+        builder.append(s.substring(s.lastIndexOf('.') + 1));
+
+        List<? extends Coder<?>> componentCoders = getComponents();
+        if (!componentCoders.isEmpty()) {
+            builder.append('(');
+            boolean first = true;
+            for (Coder<?> componentCoder : componentCoders) {
+                if (first) {
+                    first = false;
+                } else {
+                    builder.append(',');
+                }
+                builder.append(componentCoder.toString());
+            }
+            builder.append(')');
+        }
+        return builder.toString();
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return false;
+    }
+
+    @Override
+    public Object structuralValue(T value) {
+        if (value != null && consistentWithEquals()) {
+            return value;
+        } else {
+            try {
+                ByteArrayOutputStream os = new ByteArrayOutputStream();
+                encode(value, os);
+                return new StructuralByteArray(os.toByteArray());
+            } catch (Exception exn) {
+                throw new IllegalArgumentException(
+                        "Unable to encode element '" + value + "' with coder '" + this + "'.", exn);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarInt.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarInt.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarInt.java
new file mode 100644
index 0000000..bebc1e4
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarInt.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public class VarInt {
+
+    private static long convertIntToLongNoSignExtend(int v) {
+        return v & 0xFFFFFFFFL;
+    }
+
+    public static void encode(int v, OutputStream stream) throws IOException {
+        encode(convertIntToLongNoSignExtend(v), stream);
+    }
+
+    public static void encode(long v, OutputStream stream) throws IOException {
+        do {
+            // Encode next 7 bits + terminator bit
+            long bits = v & 0x7F;
+            v >>>= 7;
+            byte b = (byte) (bits | ((v != 0) ? 0x80 : 0));
+            stream.write(b);
+        } while (v != 0);
+    }
+
+    public static int decodeInt(InputStream stream) throws IOException {
+        long r = decodeLong(stream);
+        if (r < 0 || r >= 1L << 32) {
+            throw new IOException("varint overflow " + r);
+        }
+        return (int) r;
+    }
+
+    public static long decodeLong(InputStream stream) throws IOException {
+        long result = 0;
+        int shift = 0;
+        int b;
+        do {
+            // Get 7 bits from next byte
+            b = stream.read();
+            if (b < 0) {
+                if (shift == 0) {
+                    throw new EOFException();
+                } else {
+                    throw new IOException("varint not terminated");
+                }
+            }
+            long bits = b & 0x7F;
+            if (shift >= 64 || (shift == 63 && bits > 1)) {
+                // Out of range
+                throw new IOException("varint too long");
+            }
+            result |= bits << shift;
+            shift += 7;
+        } while ((b & 0x80) != 0);
+        return result;
+    }
+
+    public static int getLength(int v) {
+        return getLength(convertIntToLongNoSignExtend(v));
+    }
+
+    public static int getLength(long v) {
+        int result = 0;
+        do {
+            result++;
+            v >>>= 7;
+        } while (v != 0);
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarIntCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarIntCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarIntCoder.java
new file mode 100644
index 0000000..7dac822
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarIntCoder.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+
+public class VarIntCoder extends AtomicCoder<Integer> {
+
+    public static VarIntCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final VarIntCoder INSTANCE = new VarIntCoder();
+
+    private VarIntCoder() {}
+
+    @Override
+    public void encode(Integer value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Integer");
+        }
+        try {
+            VarInt.encode(value.intValue(), outStream);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public Integer decode(InputStream inStream)
+            throws CoderException {
+        try {
+            return VarInt.decodeInt(inStream);
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (Exception e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public void verifyDeterministic() {}
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Integer value) {
+        return true;
+    }
+
+    @Override
+    public long getEncodedElementByteSize(Integer value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Integer");
+        }
+        return VarInt.getLength(value.longValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarLongCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarLongCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarLongCoder.java
new file mode 100644
index 0000000..15af634
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VarLongCoder.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+import java.util.Collections;
+import java.util.List;
+
+public class VarLongCoder extends StructuredCoder<Long> {
+    public static VarLongCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final VarLongCoder INSTANCE = new VarLongCoder();
+
+    private VarLongCoder() {}
+
+    @Override
+    public void encode(Long value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Long");
+        }
+        try {
+            VarInt.encode(value.longValue(), outStream);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public Long decode(InputStream inStream)
+            throws CoderException {
+        try {
+            return VarInt.decodeLong(inStream);
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (Exception e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public void verifyDeterministic() {}
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Long value) {
+        return true;
+    }
+
+    @Override
+    public long getEncodedElementByteSize(Long value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Long");
+        }
+        return VarInt.getLength(value.longValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VoidCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VoidCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VoidCoder.java
new file mode 100644
index 0000000..f4d00f1
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/VoidCoder.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public class VoidCoder extends AtomicCoder<Void> {
+
+    public static VoidCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final VoidCoder INSTANCE = new VoidCoder();
+
+    private VoidCoder() {
+    }
+
+    @Override
+    public void encode(Void value, OutputStream outStream) {
+        // Nothing to write!
+    }
+
+    @Override
+    public Void decode(InputStream inStream) {
+        // Nothing to read!
+        return null;
+    }
+
+    @Override
+    public void verifyDeterministic() {
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Void value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(Void value) {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
index 17c93bd..e706f4f 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
@@ -20,10 +20,16 @@ package org.apache.gearpump.streaming.refactor.dsl.window.impl
 import org.apache.gearpump.Message
 import org.apache.gearpump.streaming.dsl.window.api.Trigger
 
+<<<<<<< HEAD:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
 trait ReduceFnRunner {
 
   def process(message: Message): Unit
 
   def onTrigger(trigger: Trigger): Unit
+=======
+trait State {
+
+  def clear: Unit
+>>>>>>> e6ce91c... [Gearpump 311] refactor state management:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/State.scala
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
index d0b84cb..6665766 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+<<<<<<< HEAD:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
 package org.apache.gearpump.streaming.refactor.sink
 
 import akka.actor.ActorSystem
@@ -33,4 +34,20 @@ object DataSinkProcessor {
     Processor[DataSinkTask](parallelism, description = description,
       taskConf.withValue[DataSink](DataSinkTask.DATA_SINK, dataSink))
   }
+=======
+package org.apache.gearpump.streaming.refactor.state
+
+import org.apache.gearpump.streaming.refactor.state.api.State
+
+trait StateTag[StateT <: State] extends Serializable {
+
+  def appendTo(sb: Appendable)
+
+  def getId: String
+
+  def getSpec: StateSpec[StateT]
+
+  def bind(binder: StateBinder): StateT
+
+>>>>>>> e6ce91c... [Gearpump 311] refactor state management:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTag.scala
 }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/InMemoryGlobalStateInternals.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/InMemoryGlobalStateInternals.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/InMemoryGlobalStateInternals.scala
new file mode 100644
index 0000000..4dbb07f
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/InMemoryGlobalStateInternals.scala
@@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state
+
+import java.util
+import java.util.Map.Entry
+import java.util.{ArrayList, HashSet, List, Set}
+import java.lang.Iterable
+
+import com.google.common.collect.{HashBasedTable, Table}
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.InMemoryGlobalStateInternals.InMemoryStateBinder
+import org.apache.gearpump.streaming.refactor.state.api._
+
+class InMemoryGlobalStateInternals[K] protected(key: K) extends StateInternals {
+
+  protected val inMemoryStateTable: InMemoryGlobalStateInternals.StateTable =
+    new InMemoryGlobalStateInternals.StateTable {
+      override def binderForNamespace(namespace: StateNamespace): StateBinder = {
+        new InMemoryStateBinder
+      }
+  }
+
+  override def getKey: Any = key
+
+  override def state[T <: State](namespace: StateNamespace, address: StateTag[T]): T =
+    inMemoryStateTable.get(namespace, address)
+
+}
+
+object InMemoryGlobalStateInternals {
+
+  abstract class StateTable {
+
+    val stateTable: Table[StateNamespace, StateTag[_], State] = HashBasedTable.create()
+
+    def get[StateT <: State](namespace: StateNamespace, tag: StateTag[StateT]): StateT = {
+      val storage: State = stateTable.get(namespace, tag)
+      if (storage != null) {
+        storage.asInstanceOf[StateT]
+      }
+
+      val typedStorage: StateT = tag.getSpec.bind(tag.getId, binderForNamespace(namespace))
+      stateTable.put(namespace, tag, typedStorage)
+      typedStorage
+    }
+
+    def clearNamespace(namespace: StateNamespace): Unit = stateTable.rowKeySet().remove(namespace)
+
+    def clear: Unit = stateTable.clear()
+
+    def values: Iterable[State] = stateTable.values().asInstanceOf[Iterable[State]]
+
+    def isNamespaceInUse(namespace: StateNamespace): Boolean = stateTable.containsRow(namespace)
+
+    def getTagsInUse(namespace: StateNamespace): java.util.Map[StateTag[_], State]
+      = stateTable.row(namespace)
+
+    def getNamespacesInUse(): java.util.Set[StateNamespace] = stateTable.rowKeySet()
+
+    def binderForNamespace(namespace: StateNamespace): StateBinder
+
+  }
+
+  class InMemoryStateBinder extends StateBinder {
+
+    override def bindValue[T](id: String, spec: StateSpec[ValueState[T]],
+        coder: Coder[T]): ValueState[T] = new InMemoryValueState[T]()
+
+    override def bindBag[T](id: String, spec: StateSpec[BagState[T]],
+        elemCoder: Coder[T]): BagState[T] = new InMemoryBagState[T]()
+
+    override def bindSet[T](id: String, spec: StateSpec[SetState[T]],
+        elemCoder: Coder[T]): SetState[T] = new InMemorySetState[T]()
+
+    override def bindMap[KeyT, ValueT](id: String, spec: StateSpec[MapState[KeyT, ValueT]],
+        mapKeyCoder: Coder[KeyT], mapValueCoder: Coder[ValueT]): MapState[KeyT, ValueT] =
+      new InMemoryMapState[KeyT, ValueT]()
+  }
+
+  trait InMemoryState[T <: InMemoryState[T]] {
+
+    def isCleared: Boolean
+
+    def copy: T
+
+  }
+
+  class InMemoryBagState[T] extends BagState[T] with InMemoryState[InMemoryBagState[T]] {
+
+    private var contents: List[T] = new ArrayList[T]
+
+    override def readLater: BagState[T] = this
+
+    override def isCleared: Boolean = contents.isEmpty
+
+    override def copy: InMemoryBagState[T] = {
+      val that: InMemoryBagState[T] = new InMemoryBagState[T]
+      that.contents.addAll(this.contents)
+      that
+    }
+
+    override def add(value: T): Unit = contents.add(value)
+
+    override def isEmpty: ReadableState[Boolean] = {
+      new ReadableState[Boolean] {
+        override def readLater: ReadableState[Boolean] = {
+          this
+        }
+
+        override def read: Boolean = {
+          contents.isEmpty
+        }
+      }
+    }
+
+    override def clear: Unit = contents = new ArrayList[T]
+
+    override def read: Iterable[T] = contents.asInstanceOf[Iterable[T]]
+
+  }
+
+  class InMemoryValueState[T] extends ValueState[T] with InMemoryState[InMemoryValueState[T]] {
+
+    private var cleared: Boolean = true
+    private var value: T = _
+
+    def write(input: T): Unit = {
+      cleared = false
+      this.value = input
+    }
+
+    def readLater: InMemoryValueState[T] = this
+
+    def isCleared: Boolean = cleared
+
+    def copy: InMemoryValueState[T] = {
+      val that: InMemoryValueState[T] = new InMemoryValueState[T]
+      if (!this.cleared) {
+        that.cleared = this.cleared
+        that.value = this.value
+      }
+
+      that
+    }
+
+    def clear: Unit = {
+      value = null.asInstanceOf[T]
+      cleared = true
+    }
+
+    def read: T = value
+
+  }
+
+  class InMemoryMapState[K, V] extends MapState[K, V] with InMemoryState[InMemoryMapState[K, V]] {
+
+    private var contents: util.Map[K, V] = new util.HashMap[K, V]
+
+    override def put(key: K, value: V): Unit = contents.put(key, value)
+
+    override def putIfAbsent(key: K, value: V): ReadableState[V] = {
+      var v: V = contents.get(key)
+      if (v == null) {
+        v = contents.put(key, value)
+      }
+
+      ReadableStates.immediate(v)
+    }
+
+    override def remove(key: K): Unit = contents.remove(key)
+
+    override def get(key: K): ReadableState[V] = ReadableStates.immediate(contents.get(key))
+
+    override def keys: ReadableState[Iterable[K]] =
+      ReadableStates.immediate(contents.keySet().asInstanceOf[Iterable[K]])
+
+    override def values: ReadableState[Iterable[V]] =
+      ReadableStates.immediate(contents.values().asInstanceOf[Iterable[V]])
+
+    override def entries: ReadableState[Iterable[Entry[K, V]]] =
+      ReadableStates.immediate(contents.entrySet().asInstanceOf[Iterable[util.Map.Entry[K, V]]])
+
+    override def isCleared: Boolean = contents.isEmpty
+
+    override def copy: InMemoryMapState[K, V] = {
+      val that: InMemoryMapState[K, V] = new InMemoryMapState
+      that.contents.putAll(this.contents)
+      that
+    }
+
+    override def clear: Unit = contents = new util.HashMap[K, V]
+
+  }
+
+  class InMemorySetState[T] extends SetState[T] with InMemoryState[InMemorySetState[T]] {
+
+    private var contents: Set[T] = new HashSet[T]
+
+    override def contains(t: T): ReadableState[Boolean] =
+      ReadableStates.immediate(contents.contains(t))
+
+    override def addIfAbsent(t: T): ReadableState[Boolean] = {
+      val alreadyContained: Boolean = contents.contains(t)
+      contents.add(t)
+      ReadableStates.immediate(!alreadyContained)
+    }
+
+    override def remove(t: T): Unit = contents.remove(t)
+
+    override def readLater: SetState[T] = this
+
+    override def isCleared: Boolean = contents.isEmpty
+
+    override def copy: InMemorySetState[T] = {
+      val that: InMemorySetState[T] = new InMemorySetState[T]
+      that.contents.addAll(this.contents)
+      that
+    }
+
+    override def add(value: T): Unit = contents.add(value)
+
+    override def isEmpty: ReadableState[Boolean] = {
+      new ReadableState[Boolean] {
+
+        override def readLater: ReadableState[Boolean] = this
+
+        override def read: Boolean = contents.isEmpty
+      }
+    }
+
+    override def clear: Unit = contents = new HashSet[T]
+
+    override def read: Iterable[T] = contents.asInstanceOf[Iterable[T]]
+  }
+
+}
+
+object ReadableStates {
+
+  def immediate[T](value: T): ReadableState[T] = {
+    new ReadableState[T] {
+      override def readLater: ReadableState[T] = {
+        this
+      }
+
+      override def read: T = {
+        value
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
index c387960..8832aee 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
@@ -23,10 +23,16 @@ import java.time.Instant
 import org.apache.gearpump.streaming.refactor.coder.Coder
 import org.apache.gearpump.streaming.refactor.state.api.StateInternals
 
+<<<<<<< HEAD:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
 /**
  *
  */
 trait RuntimeContext {
+=======
+trait StateSpec[StateT <: State] extends Serializable {
+
+  def bind(id: String, binder: StateBinder): StateT
+>>>>>>> e6ce91c... [Gearpump 311] refactor state management:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpec.scala
 
   def getStateInternals[KT](keyCoder: Coder[KT], key: KT): StateInternals
 

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateBinder.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateBinder.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateBinder.scala
new file mode 100644
index 0000000..db39142
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateBinder.scala
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state
+
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.api.{BagState, MapState, SetState, ValueState}
+
+trait StateBinder {
+
+  def bindValue[T](id: String, spec: StateSpec[ValueState[T]], coder: Coder[T]): ValueState[T]
+
+  def bindBag[T](id: String, spec: StateSpec[BagState[T]], elemCoder: Coder[T]): BagState[T]
+
+  def bindSet[T](id: String, spec: StateSpec[SetState[T]], elemCoder: Coder[T]): SetState[T]
+
+  def bindMap[KeyT, ValueT](id: String, spec: StateSpec[MapState[KeyT, ValueT]],
+      mapKeyCoder: Coder[KeyT], mapValueCoder: Coder[ValueT]): MapState[KeyT, ValueT]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespace.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespace.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespace.scala
new file mode 100644
index 0000000..dbc2320
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespace.scala
@@ -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.gearpump.streaming.refactor.state
+
+trait StateNamespace {
+
+  def stringKey: String
+
+  def appendTo(sb: Appendable): Unit
+
+  def getCacheKey: Object
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespaces.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespaces.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespaces.scala
new file mode 100644
index 0000000..c2cba51
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateNamespaces.scala
@@ -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.gearpump.streaming.refactor.state
+
+import java.util.Objects
+
+object StateNamespaces {
+
+  def global: StateNamespace = {
+    new GlobalNameSpace
+  }
+
+  private object NameSpace extends Enumeration {
+    type NameSpace = Value
+    val GLOBAL, WINDOW, WINDOW_AND_TRIGGER = Value
+  }
+
+  class GlobalNameSpace extends StateNamespace {
+
+    private val GLOBAL_STRING: String = "/"
+
+    override def stringKey: String = {
+      GLOBAL_STRING
+    }
+
+    override def appendTo(sb: Appendable): Unit = {
+      sb.append(GLOBAL_STRING)
+    }
+
+    override def getCacheKey: AnyRef = {
+      GLOBAL_STRING
+    }
+
+    override def equals(obj: Any): Boolean = {
+      obj == this || obj.isInstanceOf[GlobalNameSpace]
+    }
+
+    override def hashCode(): Int = {
+      Objects.hash(NameSpace.GLOBAL)
+    }
+  }
+
+  // TODO : implement WindowNamespace & WindowAndTriggerNamespace
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpecs.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpecs.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpecs.scala
new file mode 100644
index 0000000..f056915
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpecs.scala
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state
+
+import java.util.Objects
+
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.api.{BagState, MapState, SetState, ValueState}
+
+object StateSpecs {
+
+  private class ValueStateSpec[T](coder: Coder[T]) extends StateSpec[ValueState[T]] {
+
+    var aCoder: Coder[T] = coder
+
+    override def bind(id: String, binder: StateBinder): ValueState[T] = {
+      binder.bindValue(id, this, aCoder)
+    }
+
+    override def offerCoders(coders: Array[Coder[ValueState[T]]]): Unit = {
+      if (this.aCoder == null) {
+        if (coders(0) != null) {
+          this.aCoder = coders(0).asInstanceOf[Coder[T]]
+        }
+      }
+    }
+
+    override def finishSpecifying: Unit = {
+      if (aCoder == null) throw new IllegalStateException(
+        "Unable to infer a coder for ValueState and no Coder"
+        + " was specified. Please set a coder by either invoking"
+        + " StateSpecs.value(Coder<T> valueCoder) or by registering the coder in the"
+        + " Pipeline's CoderRegistry.")
+    }
+
+    override def equals(obj: Any): Boolean = {
+      var result = false
+      if (obj == this) result = true
+
+      if (!(obj.isInstanceOf[ValueStateSpec[T]])) result = false
+
+      val that: ValueStateSpec[_] = obj.asInstanceOf[ValueStateSpec[_]]
+      result = Objects.equals(this.aCoder, that.aCoder)
+      result
+    }
+
+    override def hashCode(): Int = {
+      Objects.hashCode(this.aCoder)
+    }
+  }
+
+  private class BagStateSpec[T](coder: Coder[T]) extends StateSpec[BagState[T]] {
+
+    private implicit var elemCoder = coder
+
+    override def bind(id: String, binder: StateBinder): BagState[T] =
+      binder.bindBag(id, this, elemCoder)
+
+    override def offerCoders(coders: Array[Coder[BagState[T]]]): Unit = {
+      if (this.elemCoder == null) {
+        if (coders(0) != null) {
+          this.elemCoder = coders(0).asInstanceOf[Coder[T]]
+        }
+      }
+    }
+
+    override def finishSpecifying: Unit = {
+      if (elemCoder == null) {
+        throw new IllegalStateException("Unable to infer a coder for BagState and no Coder"
+          + " was specified. Please set a coder by either invoking"
+          + " StateSpecs.bag(Coder<T> elemCoder) or by registering the coder in the"
+          + " Pipeline's CoderRegistry.");
+      }
+    }
+
+    override def equals(obj: Any): Boolean = {
+      var result = false
+      if (obj == this) result = true
+
+      if (!obj.isInstanceOf[BagStateSpec[_]]) result = false
+
+      val that = obj.asInstanceOf[BagStateSpec[_]]
+      result = Objects.equals(this.elemCoder, that.elemCoder)
+      result
+    }
+
+    override def hashCode(): Int = Objects.hash(getClass, elemCoder)
+  }
+
+  private class MapStateSpec[K, V](keyCoder: Coder[K], valueCoder: Coder[V])
+    extends StateSpec[MapState[K, V]] {
+
+    private implicit var kCoder = keyCoder
+    private implicit var vCoder = valueCoder
+
+    override def bind(id: String, binder: StateBinder): MapState[K, V] =
+      binder.bindMap(id, this, keyCoder, valueCoder)
+
+    override def offerCoders(coders: Array[Coder[MapState[K, V]]]): Unit = {
+      if (this.kCoder == null) {
+        if (coders(0) != null) {
+          this.kCoder = coders(0).asInstanceOf[Coder[K]]
+        }
+      }
+
+      if (this.vCoder == null) {
+        if (coders(1) != null) {
+          this.vCoder = coders(1).asInstanceOf[Coder[V]]
+        }
+      }
+    }
+
+    override def finishSpecifying: Unit = {
+      if (keyCoder == null || valueCoder == null) {
+        throw new IllegalStateException("Unable to infer a coder for MapState and no Coder"
+          + " was specified. Please set a coder by either invoking"
+          + " StateSpecs.map(Coder<K> keyCoder, Coder<V> valueCoder) or by registering the"
+          + " coder in the Pipeline's CoderRegistry.");
+      }
+    }
+
+    override def hashCode(): Int = Objects.hash(getClass, kCoder, vCoder)
+
+    override def equals(obj: Any): Boolean = {
+      var result = false
+      if (obj == this) result = true
+
+      if (!obj.isInstanceOf[MapStateSpec[_, _]]) result = false
+
+      implicit var that = obj.asInstanceOf[MapStateSpec[_, _]]
+      result = Objects.equals(this.kCoder, that.vCoder) && Objects.equals(this.vCoder, that.vCoder)
+      result
+    }
+  }
+
+  private class SetStateSpec[T](coder: Coder[T]) extends StateSpec[SetState[T]] {
+
+    private implicit var elemCoder = coder
+
+    override def bind(id: String, binder: StateBinder): SetState[T] =
+      binder.bindSet(id, this, elemCoder)
+
+    override def offerCoders(coders: Array[Coder[SetState[T]]]): Unit = {
+      if (this.elemCoder == null) {
+        if (coders(0) != null) {
+          this.elemCoder = coders(0).asInstanceOf[Coder[T]]
+        }
+      }
+    }
+
+    override def finishSpecifying: Unit = {
+      if (elemCoder == null) {
+        throw new IllegalStateException("Unable to infer a coder for SetState and no Coder"
+          + " was specified. Please set a coder by either invoking"
+          + " StateSpecs.set(Coder<T> elemCoder) or by registering the coder in the"
+          + " Pipeline's CoderRegistry.");
+      }
+    }
+
+    override def equals(obj: Any): Boolean = {
+      var result = false
+      if (obj == this) result = true
+
+      if (!obj.isInstanceOf[SetStateSpec[_]]) result = false
+
+      implicit var that = obj.asInstanceOf[SetStateSpec[_]]
+      result = Objects.equals(this.elemCoder, that.elemCoder)
+      result
+    }
+
+    override def hashCode(): Int = Objects.hash(getClass, elemCoder)
+  }
+
+  def value[T]: StateSpec[ValueState[T]] = new ValueStateSpec[T](null)
+
+  def value[T](valueCoder: Coder[T]): StateSpec[ValueState[T]] = {
+    if (valueCoder == null) {
+      throw new NullPointerException("valueCoder should not be null. Consider value() instead")
+    }
+
+    new ValueStateSpec[T](valueCoder)
+  }
+
+  def bag[T]: StateSpec[BagState[T]] = new BagStateSpec[T](null)
+
+  def bag[T](elemCoder: Coder[T]): StateSpec[BagState[T]] = new BagStateSpec[T](elemCoder)
+
+  def set[T]: StateSpec[SetState[T]] = new SetStateSpec[T](null)
+
+  def set[T](elemCoder: Coder[T]): StateSpec[SetState[T]] = new SetStateSpec[T](elemCoder)
+
+  def map[K, V]: StateSpec[MapState[K, V]] = new MapStateSpec[K, V](null, null)
+
+  def map[K, V](keyCoder: Coder[K], valueCoder: Coder[V]): StateSpec[MapState[K, V]] =
+    new MapStateSpec[K, V](keyCoder, valueCoder)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTags.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTags.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTags.scala
new file mode 100644
index 0000000..cbd050a
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTags.scala
@@ -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.gearpump.streaming.refactor.state
+
+import java.util.Objects
+
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.StateTags.StateKind.StateKind
+import org.apache.gearpump.streaming.refactor.state.api._
+
+object StateTags {
+
+  object StateKind extends Enumeration {
+    type StateKind = Value
+    val SYSTEM = Value("s")
+    val USER = Value("u")
+  }
+
+  private trait SystemStateTag[StateT <: State] {
+    def asKind(kind: StateKind): StateTag[StateT]
+  }
+
+  def tagForSpec[StateT <: State](id: String, spec: StateSpec[StateT]): StateTag[StateT] =
+    new SimpleStateTag[StateT](new StructureId(id), spec)
+
+  def value[T](id: String, valueCoder: Coder[T]): StateTag[ValueState[T]] =
+    new SimpleStateTag[ValueState[T]](new StructureId(id), StateSpecs.value(valueCoder))
+
+  def bag[T](id: String, elemCoder: Coder[T]): StateTag[BagState[T]] =
+    new SimpleStateTag[BagState[T]](new StructureId(id), StateSpecs.bag(elemCoder))
+
+  def set[T](id: String, elemCoder: Coder[T]): StateTag[SetState[T]] =
+    new SimpleStateTag[SetState[T]](new StructureId(id), StateSpecs.set(elemCoder))
+
+  def map[K, V](id: String, keyCoder: Coder[K], valueCoder: Coder[V]): StateTag[MapState[K, V]] =
+    new SimpleStateTag[MapState[K, V]](new StructureId(id), StateSpecs.map(keyCoder, valueCoder))
+
+  private class SimpleStateTag[StateT <: State](id: StructureId, spec: StateSpec[StateT])
+    extends StateTag[StateT] with SystemStateTag[StateT] {
+
+    val aSpec: StateSpec[StateT] = spec
+    val aId: StructureId = id
+
+    override def appendTo(sb: Appendable): Unit = aId.appendTo(sb)
+
+
+    override def getId: String = id.getRawId
+
+    override def getSpec: StateSpec[StateT] = aSpec
+
+    override def bind(binder: StateBinder): StateT = aSpec.bind(aId.getRawId, binder)
+
+    override def asKind(kind: StateKind): StateTag[StateT] =
+      new SimpleStateTag[StateT](aId.asKind(kind), aSpec)
+
+    override def hashCode(): Int = Objects.hash(getClass, getId, getSpec)
+
+    override def equals(obj: Any): Boolean = {
+      if (!(obj.isInstanceOf[SimpleStateTag[_]])) false
+
+      val otherTag: SimpleStateTag[_] = obj.asInstanceOf[SimpleStateTag[_]]
+      Objects.equals(getId, otherTag.getId) && Objects.equals(getSpec, otherTag.getSpec)
+    }
+  }
+
+  private class StructureId(kind: StateKind, rawId: String) extends Serializable {
+
+    private val k: StateKind = kind
+    private val r: String = rawId
+
+    def this(rawId: String) {
+      this(StateKind.USER, rawId)
+    }
+
+    def asKind(kind: StateKind): StructureId = new StructureId(kind, r)
+
+    def appendTo(sb: Appendable): Unit = sb.append(k.toString).append(r)
+
+    def getRawId: String = r
+
+    override def hashCode(): Int = Objects.hash(k, r)
+
+    override def equals(obj: Any): Boolean = {
+      if (obj == this) true
+
+      if (!(obj.isInstanceOf[StructureId])) false
+
+      val that : StructureId = obj.asInstanceOf[StructureId]
+      Objects.equals(k, that.k) && Objects.equals(r, that.r)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
index 6d72e78..0f94052 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
@@ -34,6 +34,13 @@ import org.apache.gearpump.streaming.transaction.api.CheckpointStoreFactory
 import org.apache.gearpump.util.LogUtil
 import org.apache.gearpump.{Message, TimeStamp}
 
+<<<<<<< HEAD
+=======
+object StatefulTask {
+  val LOG = LogUtil.getLogger(getClass)
+}
+
+>>>>>>> e6ce91c... [Gearpump 311] refactor state management
 abstract class StatefulTask(taskContext: TaskContext, conf: UserConfig)
   extends Task(taskContext, conf) {
 
@@ -53,7 +60,11 @@ abstract class StatefulTask(taskContext: TaskContext, conf: UserConfig)
   // core state data
   var encodedKeyStateMap: Map[String, Table[String, String, Array[Byte]]] = null
 
+<<<<<<< HEAD
   def open(runtimeContext: RuntimeContext): Unit = {}
+=======
+  def open: Unit = {}
+>>>>>>> e6ce91c... [Gearpump 311] refactor state management
 
   def invoke(message: Message): Unit
 

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/BagState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/BagState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/BagState.scala
new file mode 100644
index 0000000..38d918e
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/BagState.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.api
+
+import java.lang.Iterable
+
+trait BagState[T] extends GroupingState[T, Iterable[T]] {
+
+  def readLater: BagState[T]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/CombiningState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/CombiningState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/CombiningState.scala
new file mode 100644
index 0000000..640cc9e
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/CombiningState.scala
@@ -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.gearpump.streaming.refactor.state.api
+
+trait CombiningState[InputT, AccumT, OutputT] extends GroupingState[InputT, OutputT] {
+
+  def getAccum: AccumT
+
+  def addAccum(accumT: AccumT)
+
+  def mergeAccumulators(accumulators: Iterable[AccumT]): AccumT
+
+  def readLater: CombiningState[InputT, AccumT, OutputT]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/GroupingState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/GroupingState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/GroupingState.scala
new file mode 100644
index 0000000..2f8939a
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/GroupingState.scala
@@ -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.gearpump.streaming.refactor.state.api
+
+trait GroupingState[InputT, OutputT] extends ReadableState[OutputT] with State {
+
+  def add(value: InputT): Unit
+
+  def isEmpty: ReadableState[Boolean]
+
+  def readLater: GroupingState[InputT, OutputT]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/MapState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/MapState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/MapState.scala
new file mode 100644
index 0000000..25de704
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/MapState.scala
@@ -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.gearpump.streaming.refactor.state.api
+
+import java.lang.Iterable
+
+trait MapState[K, V] extends State {
+
+  def put(key : K, value : V): Unit
+
+  def putIfAbsent(key : K, value : V): ReadableState[V]
+
+  def remove(key : K): Unit
+
+  def get(key : K): ReadableState[V]
+
+  def keys: ReadableState[Iterable[K]]
+
+  def values: ReadableState[Iterable[V]]
+
+  def entries: ReadableState[Iterable[java.util.Map.Entry[K, V]]]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ReadableState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ReadableState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ReadableState.scala
new file mode 100644
index 0000000..f6f4d98
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ReadableState.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.api
+
+trait ReadableState[T] {
+
+  def read: T
+
+  def readLater: ReadableState[T]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/SetState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/SetState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/SetState.scala
new file mode 100644
index 0000000..e1990b2
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/SetState.scala
@@ -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.gearpump.streaming.refactor.state.api
+
+import java.lang.Iterable
+
+trait SetState[T] extends GroupingState[T, Iterable[T]]{
+
+  def contains(t: T): ReadableState[Boolean]
+
+  def addIfAbsent(t: T): ReadableState[Boolean]
+
+  def remove(t: T): Unit
+
+  def readLater: SetState[T]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternals.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternals.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternals.scala
new file mode 100644
index 0000000..e3a136d
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternals.scala
@@ -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.gearpump.streaming.refactor.state.api
+
+import org.apache.gearpump.streaming.refactor.state.{StateNamespace, StateTag}
+
+trait StateInternals {
+
+  def getKey: Any
+
+  def state[T <: State](namespace: StateNamespace, address: StateTag[T]): T
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternalsFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternalsFactory.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternalsFactory.scala
new file mode 100644
index 0000000..215528c
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/StateInternalsFactory.scala
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.api
+
+trait StateInternalsFactory[K] extends Serializable {
+
+  def stateInternalsForKey(key: K): StateInternals
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ValueState.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ValueState.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ValueState.scala
new file mode 100644
index 0000000..3555ec4
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/ValueState.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.api
+
+trait ValueState[T] extends ReadableState[T] with State {
+
+  def write(input : T): Unit
+
+  def readLater: ValueState[T]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternals.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternals.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternals.scala
new file mode 100644
index 0000000..12b6e42
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternals.scala
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.heap
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream}
+import java.lang.Iterable
+import java.util
+import java.util.Map.Entry
+import java.util._
+import java.util.Objects
+
+import com.google.common.collect.Table
+import org.apache.gearpump.streaming.refactor.coder.{Coder, ListCoder, MapCoder, SetCoder}
+import org.apache.gearpump.streaming.refactor.state.{StateBinder, StateNamespace, StateSpec, StateTag}
+import org.apache.gearpump.streaming.refactor.state.api._
+import org.apache.gearpump.util.LogUtil
+
+class HeapStateInternals[K](key: K, stateTable: Table[String, String, Array[Byte]])
+  extends StateInternals {
+
+  val LOG = LogUtil.getLogger(getClass)
+
+  private class HeapStateBinder(namespace: StateNamespace, address: StateTag[_])
+    extends StateBinder {
+
+    private val ns: StateNamespace = namespace
+    private val addr: StateTag[_] = address
+
+    override def bindValue[T](id: String, spec: StateSpec[ValueState[T]],
+        coder: Coder[T]): ValueState[T] =
+      new HeapValueState[T](ns, addr.asInstanceOf[StateTag[ValueState[T]]], coder)
+
+    override def bindBag[T](id: String, spec: StateSpec[BagState[T]],
+        elemCoder: Coder[T]): BagState[T] =
+      new HeapBagState[T](ns, addr.asInstanceOf[StateTag[BagState[T]]], elemCoder)
+
+    override def bindSet[T](id: String, spec: StateSpec[SetState[T]],
+        elemCoder: Coder[T]): SetState[T] =
+      new HeapSetState[T](ns, addr.asInstanceOf[StateTag[SetState[T]]], elemCoder)
+
+    override def bindMap[KeyT, ValueT](id: String, spec: StateSpec[MapState[KeyT, ValueT]],
+        mapKeyCoder: Coder[KeyT], mapValueCoder: Coder[ValueT]): MapState[KeyT, ValueT] =
+      new HeapMapState[KeyT, ValueT](ns,
+        addr.asInstanceOf[StateTag[MapState[KeyT, ValueT]]], mapKeyCoder, mapValueCoder)
+
+  }
+
+  override def getKey: Any = key
+
+  override def state[T <: State](namespace: StateNamespace, address: StateTag[T]): T =
+    address.bind(new HeapStateBinder(namespace, address))
+
+  private class AbstractState[T](namespace: StateNamespace, address: StateTag[_ <: State],
+      coder: Coder[T]) {
+
+    protected val ns: StateNamespace = namespace
+    protected val addr: StateTag[_ <: State] = address
+    protected val c: Coder[T] = coder
+
+    protected def readValue: T = {
+      var value: T = null.asInstanceOf[T]
+      val buf: Array[Byte] = stateTable.get(ns.stringKey, addr.getId)
+      if (buf != null) {
+        val is: InputStream = new ByteArrayInputStream(buf)
+        try {
+          value = c.decode(is)
+        } catch {
+          case ex: Exception => throw new RuntimeException(ex)
+        }
+      }
+
+      value
+    }
+
+    def writeValue(input: T): Unit = {
+      val output: ByteArrayOutputStream = new ByteArrayOutputStream();
+      try {
+        c.encode(input, output)
+        stateTable.put(ns.stringKey, addr.getId, output.toByteArray)
+      } catch {
+        case ex: Exception => throw new RuntimeException(ex)
+      }
+    }
+
+    def clear: Unit = stateTable.remove(ns.stringKey, addr.getId)
+
+    override def hashCode(): Int = Objects.hash(ns, addr)
+
+    override def equals(obj: Any): Boolean = {
+      if (obj == this) true
+
+      if (null == obj || getClass != obj.getClass) false
+
+      val that: AbstractState[_] = obj.asInstanceOf[AbstractState[_]]
+      Objects.equals(ns, that.ns) && Objects.equals(addr, that.addr)
+    }
+  }
+
+  private class HeapValueState[T](namespace: StateNamespace,
+      address: StateTag[ValueState[T]], coder: Coder[T])
+      extends AbstractState[T](namespace, address, coder) with ValueState[T] {
+
+    override def write(input: T): Unit = writeValue(input)
+
+    override def readLater: ValueState[T] = this
+
+    override def read: T = readValue
+  }
+
+  private class HeapMapState[MapKT, MapVT](namespace: StateNamespace,
+      address: StateTag[MapState[MapKT, MapVT]], mapKCoder: Coder[MapKT], mapVCoder: Coder[MapVT])
+      extends AbstractState[Map[MapKT, MapVT]](
+        namespace, address, MapCoder.of(mapKCoder, mapVCoder))
+      with MapState[MapKT, MapVT] {
+
+    private def readMap: Map[MapKT, MapVT] = {
+      implicit var map = super.readValue
+      if (map == null || map.size() == 0) {
+        map = new util.HashMap[MapKT, MapVT]
+      }
+
+      map
+    }
+
+    override def put(key: MapKT, value: MapVT): Unit = {
+      implicit var map = readMap
+      map.put(key, value)
+      super.writeValue(map)
+    }
+
+    override def putIfAbsent(key: MapKT, value: MapVT): ReadableState[MapVT] = {
+      implicit var map = readMap
+      implicit val previousVal = map.putIfAbsent(key, value)
+      super.writeValue(map)
+      new ReadableState[MapVT] {
+
+        override def readLater: ReadableState[MapVT] = this
+
+        override def read: MapVT = previousVal
+      }
+    }
+
+    override def remove(key: MapKT): Unit = {
+      implicit var map = readMap
+      map.remove(key)
+      super.writeValue(map)
+    }
+
+    override def get(key: MapKT): ReadableState[MapVT] = {
+      implicit var map = readMap
+      new ReadableState[MapVT] {
+
+        override def read: MapVT = map.get(key)
+
+        override def readLater: ReadableState[MapVT] = this
+      }
+    }
+
+    override def keys: ReadableState[Iterable[MapKT]] = {
+      implicit val map = readMap
+      new ReadableState[Iterable[MapKT]] {
+
+        override def readLater: ReadableState[Iterable[MapKT]] = this
+
+        override def read: Iterable[MapKT] = map.keySet()
+      }
+    }
+
+    override def values: ReadableState[Iterable[MapVT]] = {
+      implicit val map = readMap
+      new ReadableState[Iterable[MapVT]] {
+
+        override def readLater: ReadableState[Iterable[MapVT]] = this
+
+        override def read: Iterable[MapVT] = map.values()
+      }
+    }
+
+    override def entries: ReadableState[Iterable[Entry[MapKT, MapVT]]] = {
+      implicit var map = readMap
+      new ReadableState[Iterable[Entry[MapKT, MapVT]]] {
+
+        override def readLater: ReadableState[Iterable[Entry[MapKT, MapVT]]] = this
+
+        override def read: Iterable[Entry[MapKT, MapVT]] = map.entrySet()
+      }
+    }
+
+    override def clear: Unit = {
+      implicit var map = readMap
+      map.clear()
+      super.writeValue(map)
+    }
+}
+
+  private class HeapBagState[T](namespace: StateNamespace,
+      address: StateTag[BagState[T]], coder: Coder[T])
+      extends AbstractState[List[T]](namespace, address, ListCoder.of(coder)) with BagState[T] {
+
+    override def readLater: BagState[T] = this
+
+    override def add(input: T): Unit = {
+      val value: List[T] = read
+      value.add(input)
+      writeValue(value)
+    }
+
+    override def isEmpty: ReadableState[Boolean] = {
+      new ReadableState[Boolean] {
+
+        override def readLater: ReadableState[Boolean] = this
+
+        override def read: Boolean = stateTable.get(ns.stringKey, addr.getId) == null
+      }
+    }
+
+    override def read: List[T] = {
+      var value: List[T] = super.readValue
+      if (value == null || value.size() == 0) {
+        value = new ArrayList[T]
+      }
+
+      value
+    }
+  }
+
+  private class HeapSetState[T](namespace: StateNamespace,
+      address: StateTag[SetState[T]], coder: Coder[T])
+      extends AbstractState[Set[T]](namespace, address, SetCoder.of(coder)) with SetState[T] {
+
+    override def contains(t: T): ReadableState[Boolean] = {
+      implicit val set = read
+      new ReadableState[Boolean] {
+
+        override def readLater: ReadableState[Boolean] = this
+
+        override def read: Boolean = set.contains(t)
+      }
+    }
+
+    override def addIfAbsent(t: T): ReadableState[Boolean] = {
+      implicit val set = read
+      val success = set.add(t)
+      super.writeValue(set)
+      new ReadableState[Boolean] {
+
+        override def readLater: ReadableState[Boolean] = this
+
+        override def read: Boolean = success
+      }
+    }
+
+    override def remove(t: T): Unit = {
+      implicit var set = read
+      set.remove(t)
+      writeValue(set)
+    }
+
+    override def readLater: SetState[T] = this
+
+    override def add(value: T): Unit = {
+      implicit var set = read
+      set.add(value)
+      writeValue(set)
+    }
+
+    override def isEmpty: ReadableState[Boolean] = {
+      implicit val set = read
+      new ReadableState[Boolean] {
+
+        override def readLater: ReadableState[Boolean] = this
+
+        override def read: Boolean = set.isEmpty
+      }
+    }
+
+    override def read: Set[T] = {
+      var value: Set[T] = super.readValue
+      if (value == null || value.size() == 0) {
+        value = new util.HashSet[T]()
+      }
+
+      value
+    }
+  }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsFactory.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsFactory.scala
new file mode 100644
index 0000000..db20d66
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsFactory.scala
@@ -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.gearpump.streaming.refactor.state.heap
+
+import org.apache.gearpump.streaming.refactor.coder.{Coder, CoderException, CoderUtils}
+import org.apache.gearpump.streaming.refactor.state.api.{StateInternals, StateInternalsFactory}
+import java.util._
+
+import com.google.common.collect.{HashBasedTable, Table}
+import org.apache.gearpump.util.LogUtil
+
+class HeapStateInternalsFactory[K](keyCoder: Coder[K],
+    map: Map[String, Table[String, String, Array[Byte]]])
+    extends StateInternalsFactory[K] with Serializable {
+
+  private val LOG = LogUtil.getLogger(getClass)
+
+  private val kc: Coder[K] = keyCoder
+  private val perKeyState: Map[String, Table[String, String, Array[Byte]]] = map
+
+  def getKeyCoder: Coder[K] = {
+    this.kc
+  }
+
+  override def stateInternalsForKey(key: K): StateInternals = {
+    var keyBytes: Option[Array[Byte]] = None
+      if (key != null) {
+        keyBytes = Some(CoderUtils.encodeToByteArray(kc, key))
+      }
+
+    if (keyBytes.isEmpty) {
+      throw new RuntimeException("key bytes is null or empty, encode key occurs a error")
+    }
+
+    val keyBased64Str = Base64.getEncoder.encodeToString(keyBytes.get)
+    var stateTable: Table[String, String, Array[Byte]] = perKeyState.get(keyBased64Str)
+    if (stateTable == null) {
+      LOG.info("stateTable is null, will create!")
+      stateTable = HashBasedTable.create()
+      perKeyState.put(keyBased64Str, stateTable)
+    }
+
+    new HeapStateInternals[K](key, stateTable)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsProxy.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsProxy.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsProxy.scala
new file mode 100644
index 0000000..2f85dd9
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsProxy.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.heap
+
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.{StateNamespace, StateTag}
+import org.apache.gearpump.streaming.refactor.state.api.{State, StateInternals, StateInternalsFactory}
+
+class HeapStateInternalsProxy[K](heapStateInternalsFactory: HeapStateInternalsFactory[K])
+  extends StateInternals with Serializable {
+
+  private val factory: HeapStateInternalsFactory[K] = heapStateInternalsFactory
+
+  @transient
+  private var currentKey: K = _
+
+  def getFactory: StateInternalsFactory[K] = {
+    factory
+  }
+
+  def getKeyCoder: Coder[K] = {
+    factory.getKeyCoder
+  }
+
+  override def getKey: K = {
+    currentKey
+  }
+
+  def setKey(key: K): Unit = {
+    currentKey = key
+  }
+
+  override def state[T <: State](namespace: StateNamespace, address: StateTag[T]): T = {
+    factory.stateInternalsForKey(currentKey).state(namespace, address)
+  }
+}


[4/7] incubator-gearpump git commit: [Gearpump 311] refactor state management

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/test/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsSpec.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsSpec.scala
new file mode 100644
index 0000000..f11299e
--- /dev/null
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/refactor/state/heap/HeapStateInternalsSpec.scala
@@ -0,0 +1,484 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.heap
+
+import java.util
+import java.util.{Iterator, Map}
+
+import com.google.common.collect.Table
+import org.apache.gearpump.streaming.refactor.coder.StringUtf8Coder
+import org.apache.gearpump.streaming.refactor.state.api.{BagState, SetState, ValueState}
+import org.apache.gearpump.streaming.refactor.state.{StateNamespaces, StateTags}
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.prop.PropertyChecks
+import org.scalatest.{Matchers, PropSpec}
+
+class HeapStateInternalsSpec
+  extends PropSpec with PropertyChecks with Matchers with MockitoSugar {
+
+  property("HeapStateInternalsProxy should return correct key coder") {
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory: HeapStateInternalsFactory[String] =
+      new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+    val proxy: HeapStateInternalsProxy[String] = new HeapStateInternalsProxy[String](factory)
+
+    factory.getKeyCoder shouldBe StringUtf8Coder.of
+  }
+
+  // region value state
+  property("test heap value state: write heap state should equals read state") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val value = "hello world"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+    val stateInternals = factory.stateInternalsForKey(key)
+    val valueState = stateInternals.state[ValueState[String]](namespace,
+      StateTags.value(stateId, StringUtf8Coder.of))
+
+    valueState.write(value)
+    valueState.read shouldBe value
+  }
+
+  property("test heap value state: write heap state should not equals read state " +
+    "for different state id") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val newStateId = "02"
+    implicit val value = "hello world"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+    val stateInternals = factory.stateInternalsForKey(key)
+    val valueState = stateInternals.state[ValueState[String]](namespace,
+      StateTags.value(stateId, StringUtf8Coder.of))
+
+    valueState.write(value)
+
+    val newValueState = stateInternals.state[ValueState[String]](namespace,
+      StateTags.value(newStateId, StringUtf8Coder.of))
+    newValueState.read shouldNot be(value)
+  }
+
+  property("test heap value state: write heap state should equals read state " +
+    "for different key") {
+    implicit val key = "key"
+    implicit val newKey = "newKey"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val value = "hello world"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val valueState = stateInternals.state[ValueState[String]](namespace,
+      StateTags.value(stateId, StringUtf8Coder.of))
+
+    val newStateInternals = factory.stateInternalsForKey(newKey)
+    val newValueState = newStateInternals.state[ValueState[String]](namespace,
+      StateTags.value(stateId, StringUtf8Coder.of))
+
+    valueState.write(value)
+    newValueState.read shouldNot be(value)
+  }
+  // endregion
+
+  // region bag state
+  property("test heap Bag state: write heap state should equals read state") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val bagValue1 = "bagValue1"
+    implicit val bagValue2 = "bagValue2"
+    implicit val stateId = "01"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val bagState = stateInternals.state[BagState[String]](namespace,
+      StateTags.bag(stateId, StringUtf8Coder.of))
+
+    bagState.add(bagValue1)
+    bagState.add(bagValue2)
+
+    val bagIterator: Iterator[String] = bagState.read.iterator()
+
+    implicit var counter = 0
+
+    while (bagIterator.hasNext) {
+      counter += 1
+      if (counter == 1) {
+        bagIterator.next() shouldBe bagValue1
+      }
+      if (counter == 2) {
+        bagIterator.next() shouldBe bagValue2
+      }
+    }
+
+    counter shouldBe 2
+  }
+
+  property("test heap Bag state: write heap state should not equal read state with " +
+    "different key") {
+    implicit val key = "key"
+    implicit val newKey = "newKey"
+    implicit val namespace = StateNamespaces.global
+    implicit val bagValue1 = "bagValue1"
+    implicit val bagValue2 = "bagValue2"
+    implicit val stateId = "01"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val newStateInternals = factory.stateInternalsForKey(newKey)
+
+    val bagState = stateInternals.state[BagState[String]](namespace,
+      StateTags.bag(stateId, StringUtf8Coder.of))
+    val newBagState = newStateInternals.state[BagState[String]](namespace,
+      StateTags.bag(stateId, StringUtf8Coder.of))
+
+    bagState.add(bagValue1)
+    bagState.add(bagValue2)
+
+    val newBagIterator: Iterator[String] = newBagState.read.iterator()
+
+    implicit var counter = 0
+
+    while (newBagIterator.hasNext) {
+      counter += 1
+      newBagIterator.next()
+    }
+
+    counter shouldBe 0
+  }
+
+  property("test heap Bag state: write heap state should not equal read state " +
+    "with different stateId") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val bagValue1 = "bagValue1"
+    implicit val bagValue2 = "bagValue2"
+    implicit val stateId = "01"
+    implicit val newStateId = "02"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val bagState = stateInternals.state[BagState[String]](namespace,
+      StateTags.bag(stateId, StringUtf8Coder.of))
+    val newBagState = stateInternals.state[BagState[String]](namespace,
+      StateTags.bag(newStateId, StringUtf8Coder.of))
+
+    bagState.add(bagValue1)
+    bagState.add(bagValue2)
+
+    val newBagIterator: Iterator[String] = newBagState.read.iterator()
+
+    implicit var counter = 0
+
+    while (newBagIterator.hasNext) {
+      counter += 1
+      newBagIterator.next()
+    }
+
+    counter shouldBe 0
+  }
+  // endregion
+
+  // region set state
+  property("test heap set state, generic methods") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val setValue1 = "setValue1"
+    implicit val setValue2 = "setValue2"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val setState = stateInternals.state[SetState[String]](namespace,
+      StateTags.set(stateId, StringUtf8Coder.of))
+
+    setState.add(setValue1)
+    setState.add(setValue2)
+
+    implicit var setStateIterator = setState.read.iterator()
+
+    implicit var counter = 0
+    while (setStateIterator.hasNext) {
+      counter += 1
+      setStateIterator.next()
+    }
+
+    counter shouldBe 2
+
+    setState.addIfAbsent(setValue2).read shouldBe false
+
+    setStateIterator = setState.read.iterator()
+
+    counter = 0
+    while (setStateIterator.hasNext) {
+      counter += 1
+      setStateIterator.next()
+    }
+
+    counter shouldBe 2
+
+    setState.contains(setValue1).read shouldBe true
+    setState.contains("setValue03").read shouldBe false
+
+    setState.isEmpty.read shouldBe false
+
+    setState.remove(setValue1)
+    setState.remove(setValue2)
+
+    setState.isEmpty.read shouldBe true
+  }
+
+  property("test heap set state, write state should not equal read state " +
+    "with different key") {
+    implicit val key = "key"
+    implicit val newKey = "newKey"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val setValue1 = "setValue1"
+    implicit val setValue2 = "setValue2"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val newStateInternals = factory.stateInternalsForKey(newKey)
+
+    val setState = stateInternals.state[SetState[String]](namespace,
+      StateTags.set(stateId, StringUtf8Coder.of))
+    val newSetState = newStateInternals.state(namespace,
+      StateTags.set(stateId, StringUtf8Coder.of))
+
+    setState.add(setValue1)
+    setState.add(setValue2)
+
+    implicit val newSetStateIterator = newSetState.read.iterator()
+
+    var counter = 0
+    while (newSetStateIterator.hasNext) {
+      counter += 1
+      newSetStateIterator.next()
+    }
+
+    counter shouldBe 0
+  }
+
+  property("test heap set state, write state shuold not equal read state " +
+    "with different state id") {
+    implicit val key = "key"
+    implicit val newKey = "newKey"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val newStateId = "02"
+    implicit val setValue1 = "setValue1"
+    implicit val setValue2 = "setValue2"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+
+    val setState = stateInternals.state(namespace, StateTags.set(stateId, StringUtf8Coder.of))
+    val newSetState = stateInternals.state(namespace,
+      StateTags.set(newStateId, StringUtf8Coder.of))
+
+    setState.add(setValue1)
+    setState.addIfAbsent(setValue2)
+
+    implicit val setStateIterator = newSetState.read.iterator()
+
+    var counter = 0
+    while (setStateIterator.hasNext) {
+      counter += 1
+      setStateIterator.next()
+    }
+
+    counter shouldBe 0
+  }
+  // endregion
+
+  // region map state
+  property("test map state, generic methods") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val mapStateKey1 = "mapKey01"
+    implicit val mapStateValue1 = "mapValue01"
+    implicit val mapStateKey2 = "mapKey02"
+    implicit val mapStateValue2 = "mapValue02"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val mapState = stateInternals.state(namespace,
+      StateTags.map(stateId, StringUtf8Coder.of, StringUtf8Coder.of))
+
+    mapState.put(mapStateKey1, mapStateValue1)
+
+    implicit var mapKeysIterator = mapState.keys.read.iterator()
+
+    mapState.putIfAbsent(mapStateKey1, mapStateValue2).read shouldBe mapStateValue1
+
+    var counter = 0
+    while (mapKeysIterator.hasNext) {
+      counter += 1
+      mapKeysIterator.next()
+    }
+
+    counter shouldBe 1
+
+    counter = 0
+    implicit val mapValuesIterator = mapState.values.read.iterator()
+    while (mapValuesIterator.hasNext) {
+      counter += 1
+      mapValuesIterator.next()
+    }
+
+    counter shouldBe 1
+
+    counter = 0
+    implicit val mapEntriesIterator = mapState.entries.read.iterator()
+    while (mapEntriesIterator.hasNext) {
+      counter += 1
+      mapEntriesIterator.next()
+    }
+
+    counter shouldBe 1
+
+    mapState.get(mapStateKey1).read shouldBe mapStateValue1
+    mapState.get("test01").read shouldBe null
+
+    mapState.remove(mapStateKey1)
+
+    counter = 0
+    mapKeysIterator = mapState.keys.read.iterator()
+    while (mapKeysIterator.hasNext) {
+      counter += 1
+      mapKeysIterator.next()
+    }
+
+    counter shouldBe 0
+
+    mapState.putIfAbsent(mapStateKey2, mapStateValue2)
+
+    counter = 0
+    mapKeysIterator = mapState.keys.read.iterator()
+    while (mapKeysIterator.hasNext) {
+      counter += 1
+      mapKeysIterator.next()
+    }
+
+    counter shouldBe 1
+
+    mapState.clear
+
+    counter = 0
+    mapKeysIterator = mapState.keys.read.iterator()
+    while (mapKeysIterator.hasNext) {
+      counter += 1
+      mapKeysIterator.next()
+    }
+
+    counter shouldBe 0
+  }
+
+  property("test map state, write state should not equal read state " +
+    "with different key") {
+    implicit val key = "key"
+    implicit val newKey = "newKey"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val mapStateKey1 = "mapKey01"
+    implicit val mapStateValue1 = "mapValue01"
+    implicit val mapStateKey2 = "mapKey02"
+    implicit val mapStateValue2 = "mapValue02"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+    val newStateInternals = factory.stateInternalsForKey(newKey)
+
+    val mapState = stateInternals.state(namespace,
+      StateTags.map(stateId, StringUtf8Coder.of, StringUtf8Coder.of))
+    val newMapState = newStateInternals.state(namespace,
+      StateTags.map(stateId, StringUtf8Coder.of, StringUtf8Coder.of))
+
+    mapState.put(mapStateKey1, mapStateValue1)
+
+    mapState.get(mapStateKey1).read shouldBe mapStateValue1
+    newMapState.get(mapStateKey1).read shouldNot be(mapStateValue1)
+  }
+
+  property("test map state, write state should not equal read state " +
+    "with different state id") {
+    implicit val key = "key"
+    implicit val namespace = StateNamespaces.global
+    implicit val stateId = "01"
+    implicit val newStateId = "02"
+    implicit val mapStateKey1 = "mapKey01"
+    implicit val mapStateValue1 = "mapValue01"
+    implicit val mapStateKey2 = "mapKey02"
+    implicit val mapStateValue2 = "mapValue02"
+
+    val map: Map[String, Table[String, String, Array[Byte]]]
+    = new util.HashMap[String, Table[String, String, Array[Byte]]]()
+    val factory = new HeapStateInternalsFactory[String](StringUtf8Coder.of, map)
+
+    val stateInternals = factory.stateInternalsForKey(key)
+
+    val mapState = stateInternals.state(namespace,
+      StateTags.map(stateId, StringUtf8Coder.of, StringUtf8Coder.of))
+    val newMapState = stateInternals.state(namespace,
+      StateTags.map(newStateId, StringUtf8Coder.of, StringUtf8Coder.of))
+
+    mapState.put(mapStateKey1, mapStateValue1)
+
+    mapState.get(mapStateKey1).read shouldBe mapStateValue1
+    newMapState.get(mapStateKey1).read shouldNot be(mapStateValue1)
+  }
+  // endregion
+
+}


[6/7] incubator-gearpump git commit: [Gearpump 311] refactor state management

Posted by ma...@apache.org.
[Gearpump 311] refactor state management


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/7068699d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/7068699d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/7068699d

Branch: refs/heads/state
Commit: 7068699d3e865f4d5bd49bcfb59f2cd4115ec125
Parents: b9f1086
Author: vinoyang <ya...@gmail.com>
Authored: Wed Jun 28 11:41:15 2017 +0800
Committer: vinoyang <ya...@gmail.com>
Committed: Sun Jul 23 09:59:05 2017 +0800

----------------------------------------------------------------------
 .../streaming/refactor/coder/AtomicCoder.java   |  48 ++
 .../refactor/coder/BigEndianIntegerCoder.java   |  83 ++++
 .../refactor/coder/BigEndianLongCoder.java      |  83 ++++
 .../refactor/coder/BigIntegerCoder.java         |  74 +++
 .../BufferedElementCountingOutputStream.java    | 125 +++++
 .../refactor/coder/ByteArrayCoder.java          |  94 ++++
 .../streaming/refactor/coder/ByteCoder.java     |  88 ++++
 .../streaming/refactor/coder/Coder.java         | 134 +++++
 .../refactor/coder/CoderException.java          |  32 ++
 .../streaming/refactor/coder/CoderUtils.java    | 286 +++++++++++
 .../streaming/refactor/coder/DoubleCoder.java   |  86 ++++
 .../ElementByteSizeObservableIterable.java      |  45 ++
 .../ElementByteSizeObservableIterator.java      |  30 ++
 .../refactor/coder/ElementByteSizeObserver.java |  66 +++
 .../streaming/refactor/coder/IterableCoder.java |  41 ++
 .../refactor/coder/IterableLikeCoder.java       | 238 +++++++++
 .../refactor/coder/IteratorObserver.java        |  56 +++
 .../streaming/refactor/coder/ListCoder.java     |  47 ++
 .../streaming/refactor/coder/MapCoder.java      | 138 ++++++
 .../streaming/refactor/coder/SetCoder.java      |  49 ++
 .../refactor/coder/StringUtf8Coder.java         |  99 ++++
 .../refactor/coder/StructuralByteArray.java     |  55 +++
 .../refactor/coder/StructuredCoder.java         |  95 ++++
 .../streaming/refactor/coder/VarInt.java        |  91 ++++
 .../streaming/refactor/coder/VarIntCoder.java   |  82 ++++
 .../streaming/refactor/coder/VarLongCoder.java  |  88 ++++
 .../streaming/refactor/coder/VoidCoder.java     |  66 +++
 .../dsl/window/impl/ReduceFnRunner.scala        |   6 +
 .../refactor/sink/DataSinkProcessor.scala       |  17 +
 .../state/InMemoryGlobalStateInternals.scala    | 269 +++++++++++
 .../refactor/state/RuntimeContext.scala         |   6 +
 .../streaming/refactor/state/StateBinder.scala  |  35 ++
 .../refactor/state/StateNamespace.scala         |  29 ++
 .../refactor/state/StateNamespaces.scala        |  61 +++
 .../streaming/refactor/state/StateSpecs.scala   | 213 ++++++++
 .../streaming/refactor/state/StateTags.scala    | 109 +++++
 .../streaming/refactor/state/StatefulTask.scala |  11 +
 .../streaming/refactor/state/api/BagState.scala |  27 ++
 .../refactor/state/api/CombiningState.scala     |  31 ++
 .../refactor/state/api/GroupingState.scala      |  29 ++
 .../streaming/refactor/state/api/MapState.scala |  39 ++
 .../refactor/state/api/ReadableState.scala      |  27 ++
 .../streaming/refactor/state/api/SetState.scala |  33 ++
 .../refactor/state/api/StateInternals.scala     |  29 ++
 .../state/api/StateInternalsFactory.scala       |  25 +
 .../refactor/state/api/ValueState.scala         |  27 ++
 .../state/heap/HeapStateInternals.scala         | 305 ++++++++++++
 .../state/heap/HeapStateInternalsFactory.scala  |  62 +++
 .../state/heap/HeapStateInternalsProxy.scala    |  52 ++
 .../state/heap/HeapStateInternalsSpec.scala     | 484 +++++++++++++++++++
 50 files changed, 4345 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/AtomicCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/AtomicCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/AtomicCoder.java
new file mode 100644
index 0000000..e152b48
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/AtomicCoder.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.gearpump.streaming.refactor.coder;
+
+import java.util.Collections;
+import java.util.List;
+
+public abstract class AtomicCoder<T> extends StructuredCoder<T>  {
+
+    @Override
+    public void verifyDeterministic() {}
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public final List<? extends Coder<?>> getComponents() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public final boolean equals(Object other) {
+        return other != null && this.getClass().equals(other.getClass());
+    }
+
+    @Override
+    public final int hashCode() {
+        return this.getClass().hashCode();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianIntegerCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianIntegerCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianIntegerCoder.java
new file mode 100644
index 0000000..27ec539
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianIntegerCoder.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+
+public class BigEndianIntegerCoder extends AtomicCoder<Integer> {
+
+    public static BigEndianIntegerCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final BigEndianIntegerCoder INSTANCE = new BigEndianIntegerCoder();
+
+    private BigEndianIntegerCoder() {}
+
+    @Override
+    public void encode(Integer value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Integer");
+        }
+
+        try {
+            new DataOutputStream(outStream).writeInt(value);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public Integer decode(InputStream inStream)
+            throws CoderException {
+        try {
+            return new DataInputStream(inStream).readInt();
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public void verifyDeterministic() {}
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Integer value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(Integer value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Integer");
+        }
+        return 4;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianLongCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianLongCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianLongCoder.java
new file mode 100644
index 0000000..c788729
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigEndianLongCoder.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+
+public class BigEndianLongCoder extends AtomicCoder<Long> {
+
+    public static BigEndianLongCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final BigEndianLongCoder INSTANCE = new BigEndianLongCoder();
+
+    private BigEndianLongCoder() {}
+
+    @Override
+    public void encode(Long value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Long");
+        }
+        try {
+            new DataOutputStream(outStream).writeLong(value);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public Long decode(InputStream inStream)
+            throws CoderException {
+        try {
+            return new DataInputStream(inStream).readLong();
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public void verifyDeterministic() {
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Long value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(Long value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Long");
+        }
+        return 8;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigIntegerCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigIntegerCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigIntegerCoder.java
new file mode 100644
index 0000000..4a65992
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BigIntegerCoder.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.gearpump.streaming.refactor.coder;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigInteger;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BigIntegerCoder extends AtomicCoder<BigInteger> {
+
+    public static BigIntegerCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final BigIntegerCoder INSTANCE = new BigIntegerCoder();
+    private static final ByteArrayCoder BYTE_ARRAY_CODER = ByteArrayCoder.of();
+
+    private BigIntegerCoder() {
+    }
+
+    @Override
+    public void encode(BigInteger value, OutputStream outStream)
+            throws CoderException {
+        checkNotNull(value, String.format("cannot encode a null %s", BigInteger.class.getSimpleName()));
+        BYTE_ARRAY_CODER.encode(value.toByteArray(), outStream);
+    }
+
+    @Override
+    public BigInteger decode(InputStream inStream)
+            throws CoderException {
+        return new BigInteger(BYTE_ARRAY_CODER.decode(inStream));
+    }
+
+    @Override
+    public void verifyDeterministic() {
+        BYTE_ARRAY_CODER.verifyDeterministic();
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(BigInteger value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(BigInteger value) {
+        checkNotNull(value, String.format("cannot encode a null %s", BigInteger.class.getSimpleName()));
+        return BYTE_ARRAY_CODER.getEncodedElementByteSize(value.toByteArray());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BufferedElementCountingOutputStream.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BufferedElementCountingOutputStream.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BufferedElementCountingOutputStream.java
new file mode 100644
index 0000000..119e6eb
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/BufferedElementCountingOutputStream.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+public class BufferedElementCountingOutputStream extends OutputStream {
+    public static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
+    private final ByteBuffer buffer;
+    private final OutputStream os;
+    private boolean finished;
+    private long count;
+
+    public BufferedElementCountingOutputStream(OutputStream os) {
+        this(os, DEFAULT_BUFFER_SIZE);
+    }
+
+    BufferedElementCountingOutputStream(OutputStream os, int bufferSize) {
+        this.buffer = ByteBuffer.allocate(bufferSize);
+        this.os = os;
+        this.finished = false;
+        this.count = 0;
+    }
+
+    public void finish() throws IOException {
+        if (finished) {
+            return;
+        }
+        flush();
+        // Finish the stream by stating that there are 0 elements that follow.
+        VarInt.encode(0, os);
+        finished = true;
+    }
+
+    public void markElementStart() throws IOException {
+        if (finished) {
+            throw new IOException("Stream has been finished. Can not add any more elements.");
+        }
+        count++;
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        if (finished) {
+            throw new IOException("Stream has been finished. Can not write any more data.");
+        }
+        if (count == 0) {
+            os.write(b);
+            return;
+        }
+
+        if (buffer.hasRemaining()) {
+            buffer.put((byte) b);
+        } else {
+            outputBuffer();
+            os.write(b);
+        }
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        if (finished) {
+            throw new IOException("Stream has been finished. Can not write any more data.");
+        }
+        if (count == 0) {
+            os.write(b, off, len);
+            return;
+        }
+
+        if (buffer.remaining() >= len) {
+            buffer.put(b, off, len);
+        } else {
+            outputBuffer();
+            os.write(b, off, len);
+        }
+    }
+
+    @Override
+    public void flush() throws IOException {
+        if (finished) {
+            return;
+        }
+        outputBuffer();
+        os.flush();
+    }
+
+    @Override
+    public void close() throws IOException {
+        finish();
+        os.close();
+    }
+
+    // Output the buffer if it contains any data.
+    private void outputBuffer() throws IOException {
+        if (count > 0) {
+            VarInt.encode(count, os);
+            // We are using a heap based buffer and not a direct buffer so it is safe to access
+            // the underlying array.
+            os.write(buffer.array(), buffer.arrayOffset(), buffer.position());
+            buffer.clear();
+            // The buffer has been flushed so we must write to the underlying stream until
+            // we learn of the next element. We reset the count to zero marking that we should
+            // not use the buffer.
+            count = 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteArrayCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteArrayCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteArrayCoder.java
new file mode 100644
index 0000000..6b1af05
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteArrayCoder.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import com.google.common.io.ByteStreams;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public class ByteArrayCoder extends AtomicCoder<byte[]> {
+
+    public static ByteArrayCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final ByteArrayCoder INSTANCE = new ByteArrayCoder();
+
+    private ByteArrayCoder() {
+    }
+
+    @Override
+    public void encode(byte[] value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null byte[]");
+        }
+
+        try {
+            VarInt.encode(value.length, outStream);
+            outStream.write(value);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public byte[] decode(InputStream inStream)
+            throws CoderException {
+        byte[] value = null;
+        try {
+            int length = VarInt.decodeInt(inStream);
+            if (length < 0) {
+                throw new CoderException("invalid length " + length);
+            }
+            value = new byte[length];
+
+            ByteStreams.readFully(inStream, value);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+        return value;
+    }
+
+    @Override
+    public void verifyDeterministic() {
+    }
+
+    @Override
+    public Object structuralValue(byte[] value) {
+        return new StructuralByteArray(value);
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(byte[] value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(byte[] value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null byte[]");
+        }
+        return VarInt.getLength(value.length) + value.length;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteCoder.java
new file mode 100644
index 0000000..e3cb7e4
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ByteCoder.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+
+public class ByteCoder extends AtomicCoder<Byte> {
+
+    public static ByteCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final ByteCoder INSTANCE = new ByteCoder();
+
+    private ByteCoder() {
+    }
+
+    @Override
+    public void encode(Byte value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Byte");
+        }
+        try {
+            outStream.write(value.byteValue());
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public Byte decode(InputStream inStream) throws CoderException {
+        try {
+            // value will be between 0-255, -1 for EOF
+            int value = inStream.read();
+            if (value == -1) {
+                throw new EOFException("EOF encountered decoding 1 byte from input stream");
+            }
+            return (byte) value;
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public void verifyDeterministic() {
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Byte value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(Byte value) {
+        if (value == null) {
+            throw new CoderException("cannot estimate size for unsupported null value");
+        }
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
new file mode 100644
index 0000000..e1999ed
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Objects;
+import com.google.common.io.ByteStreams;
+import com.google.common.io.CountingOutputStream;
+
+import java.io.*;
+import java.util.Arrays;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public abstract class Coder<T> implements Serializable {
+
+    public abstract void encode(T value, OutputStream outStream)
+            throws CoderException;
+
+    public abstract T decode(InputStream inStream) throws CoderException;
+
+    public abstract List<? extends Coder<?>> getCoderArguments();
+
+    public abstract void verifyDeterministic() throws Coder.NonDeterministicException;
+
+    public static void verifyDeterministic(Coder<?> target, String message, Iterable<Coder<?>> coders)
+            throws NonDeterministicException {
+        for (Coder<?> coder : coders) {
+            try {
+                coder.verifyDeterministic();
+            } catch (NonDeterministicException e) {
+                throw new NonDeterministicException(target, message, e);
+            }
+        }
+    }
+
+    public static void verifyDeterministic(Coder<?> target, String message, Coder<?>... coders)
+            throws NonDeterministicException {
+        verifyDeterministic(target, message, Arrays.asList(coders));
+    }
+
+    public boolean consistentWithEquals() {
+        return false;
+    }
+
+    public Object structuralValue(T value) {
+        if (value != null && consistentWithEquals()) {
+            return value;
+        } else {
+            try {
+                ByteArrayOutputStream os = new ByteArrayOutputStream();
+                encode(value, os);
+                return new StructuralByteArray(os.toByteArray());
+            } catch (Exception exn) {
+                throw new IllegalArgumentException(
+                        "Unable to encode element '" + value + "' with coder '" + this + "'.", exn);
+            }
+        }
+    }
+
+    public boolean isRegisterByteSizeObserverCheap(T value) {
+        return false;
+    }
+
+    public void registerByteSizeObserver(T value, ElementByteSizeObserver observer) {
+        observer.update(getEncodedElementByteSize(value));
+    }
+
+    protected long getEncodedElementByteSize(T value) {
+        try (CountingOutputStream os = new CountingOutputStream(ByteStreams.nullOutputStream())) {
+            encode(value, os);
+            return os.getCount();
+        } catch (Exception exn) {
+            throw new IllegalArgumentException(
+                    "Unable to encode element '" + value + "' with coder '" + this + "'.", exn);
+        }
+    }
+
+    public static class NonDeterministicException extends RuntimeException {
+        private Coder<?> coder;
+        private List<String> reasons;
+
+        public NonDeterministicException(
+                Coder<?> coder, String reason, NonDeterministicException e) {
+            this(coder, Arrays.asList(reason), e);
+        }
+
+        public NonDeterministicException(Coder<?> coder, String reason) {
+            this(coder, Arrays.asList(reason), null);
+        }
+
+        public NonDeterministicException(Coder<?> coder, List<String> reasons) {
+            this(coder, reasons, null);
+        }
+
+        public NonDeterministicException(
+                Coder<?> coder,
+                List<String> reasons,
+                NonDeterministicException cause) {
+            super(cause);
+            checkArgument(reasons.size() > 0, "Reasons must not be empty.");
+            this.reasons = reasons;
+            this.coder = coder;
+        }
+
+        public Iterable<String> getReasons() {
+            return reasons;
+        }
+
+        @Override
+        public String getMessage() {
+            return String.format("%s is not deterministic because:%n  %s",
+                    coder, Joiner.on("%n  ").join(reasons));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderException.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderException.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderException.java
new file mode 100644
index 0000000..8213e42
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderException.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gearpump.streaming.refactor.coder;
+
+public class CoderException extends RuntimeException {
+    public CoderException(String message) {
+        super(message);
+    }
+
+    public CoderException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public CoderException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderUtils.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderUtils.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderUtils.java
new file mode 100644
index 0000000..2126c48
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/CoderUtils.java
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import com.google.common.io.BaseEncoding;
+
+import java.io.*;
+import java.lang.ref.SoftReference;
+
+public final class CoderUtils {
+    private CoderUtils() {
+    }  // Non-instantiable
+
+    private static ThreadLocal<SoftReference<ExposedByteArrayOutputStream>>
+            threadLocalOutputStream = new ThreadLocal<>();
+
+    private static ThreadLocal<Boolean> threadLocalOutputStreamInUse = new ThreadLocal<Boolean>() {
+        @Override
+        protected Boolean initialValue() {
+            return false;
+        }
+    };
+
+    public static <T> byte[] encodeToByteArray(Coder<T> coder, T value)
+            throws CoderException {
+        if (threadLocalOutputStreamInUse.get()) {
+            // encodeToByteArray() is called recursively and the thread local stream is in use,
+            // allocating a new one.
+            ByteArrayOutputStream stream = new ExposedByteArrayOutputStream();
+            encodeToSafeStream(coder, value, stream);
+            return stream.toByteArray();
+        } else {
+            threadLocalOutputStreamInUse.set(true);
+            try {
+                ByteArrayOutputStream stream = getThreadLocalOutputStream();
+                encodeToSafeStream(coder, value, stream);
+                return stream.toByteArray();
+            } finally {
+                threadLocalOutputStreamInUse.set(false);
+            }
+        }
+    }
+
+    private static <T> void encodeToSafeStream(
+            Coder<T> coder, T value, OutputStream stream) throws CoderException {
+        try {
+            coder.encode(value, new UnownedOutputStream(stream));
+        } catch (CoderException exn) {
+            throw new IllegalArgumentException(
+                    "Forbidden IOException when writing to OutputStream", exn);
+        }
+    }
+
+    public static <T> T decodeFromByteArray(
+            Coder<T> coder, byte[] encodedValue) throws CoderException {
+        try (ExposedByteArrayInputStream stream = new ExposedByteArrayInputStream(encodedValue)) {
+            T result = decodeFromSafeStream(coder, stream);
+            if (stream.available() != 0) {
+                throw new CoderException(
+                        stream.available() + " unexpected extra bytes after decoding " + result);
+            }
+            return result;
+        }
+    }
+
+    private static <T> T decodeFromSafeStream(
+            Coder<T> coder, InputStream stream) throws CoderException {
+        try {
+            return coder.decode(new UnownedInputStream(stream));
+        } catch (CoderException exn) {
+            throw new IllegalArgumentException(
+                    "Forbidden IOException when reading from InputStream", exn);
+        }
+    }
+
+    private static ByteArrayOutputStream getThreadLocalOutputStream() {
+        SoftReference<ExposedByteArrayOutputStream> refStream = threadLocalOutputStream.get();
+        ExposedByteArrayOutputStream stream = refStream == null ? null : refStream.get();
+        if (stream == null) {
+            stream = new ExposedByteArrayOutputStream();
+            threadLocalOutputStream.set(new SoftReference<>(stream));
+        }
+        stream.reset();
+        return stream;
+    }
+
+    public static <T> T clone(Coder<T> coder, T value) throws CoderException {
+        return decodeFromByteArray(coder, encodeToByteArray(coder, value));
+    }
+
+    public static <T> String encodeToBase64(Coder<T> coder, T value)
+            throws CoderException {
+        byte[] rawValue = encodeToByteArray(coder, value);
+        return BaseEncoding.base64Url().omitPadding().encode(rawValue);
+    }
+
+    public static <T> T decodeFromBase64(Coder<T> coder, String encodedValue) throws CoderException {
+        return decodeFromSafeStream(
+                coder,
+                new ByteArrayInputStream(BaseEncoding.base64Url().omitPadding().decode(encodedValue)));
+    }
+
+    public static class ExposedByteArrayOutputStream extends ByteArrayOutputStream {
+
+        private byte[] swappedBuffer;
+
+        private boolean isFallback = false;
+
+        private void fallback() {
+            isFallback = true;
+            if (swappedBuffer != null) {
+                // swappedBuffer != null means buf is actually provided by the caller of writeAndOwn(),
+                // while swappedBuffer is the original buffer.
+                // Recover the buffer and copy the bytes from buf.
+                byte[] tempBuffer = buf;
+                count = 0;
+                buf = swappedBuffer;
+                super.write(tempBuffer, 0, tempBuffer.length);
+                swappedBuffer = null;
+            }
+        }
+
+        public void writeAndOwn(byte[] b) throws IOException {
+            if (b.length == 0) {
+                return;
+            }
+            if (count == 0) {
+                // Optimized first-time whole write.
+                // The original buffer will be swapped to swappedBuffer, while the input b is used as buf.
+                swappedBuffer = buf;
+                buf = b;
+                count = b.length;
+            } else {
+                fallback();
+                super.write(b);
+            }
+        }
+
+        @Override
+        public void write(byte[] b, int off, int len) {
+            fallback();
+            super.write(b, off, len);
+        }
+
+        @Override
+        public void write(int b) {
+            fallback();
+            super.write(b);
+        }
+
+        @Override
+        public byte[] toByteArray() {
+            // Note: count == buf.length is not a correct criteria to "return buf;", because the internal
+            // buf may be reused after reset().
+            if (!isFallback && count > 0) {
+                return buf;
+            } else {
+                return super.toByteArray();
+            }
+        }
+
+        @Override
+        public void reset() {
+            if (count == 0) {
+                return;
+            }
+            count = 0;
+            if (isFallback) {
+                isFallback = false;
+            } else {
+                buf = swappedBuffer;
+                swappedBuffer = null;
+            }
+        }
+    }
+
+    public static class ExposedByteArrayInputStream extends ByteArrayInputStream {
+
+        public ExposedByteArrayInputStream(byte[] buf) {
+            super(buf);
+        }
+
+        public byte[] readAll() throws IOException {
+            if (pos == 0 && count == buf.length) {
+                pos = count;
+                return buf;
+            }
+            byte[] ret = new byte[count - pos];
+            super.read(ret);
+            return ret;
+        }
+
+        @Override
+        public void close() {
+            try {
+                super.close();
+            } catch (IOException exn) {
+                throw new RuntimeException("Unexpected IOException closing ByteArrayInputStream", exn);
+            }
+        }
+    }
+
+    public static class UnownedOutputStream extends FilterOutputStream {
+        public UnownedOutputStream(OutputStream delegate) {
+            super(delegate);
+        }
+
+        @Override
+        public void close() throws IOException {
+            throw new UnsupportedOperationException("Caller does not own the underlying output stream "
+                    + " and should not call close().");
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return obj instanceof UnownedOutputStream
+                    && ((UnownedOutputStream) obj).out.equals(out);
+        }
+
+        @Override
+        public int hashCode() {
+            return out.hashCode();
+        }
+
+    }
+
+    public static class UnownedInputStream extends FilterInputStream {
+        public UnownedInputStream(InputStream delegate) {
+            super(delegate);
+        }
+
+        @Override
+        public void close() throws IOException {
+            throw new UnsupportedOperationException("Caller does not own the underlying input stream "
+                    + " and should not call close().");
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return obj instanceof UnownedInputStream
+                    && ((UnownedInputStream) obj).in.equals(in);
+        }
+
+        @Override
+        public int hashCode() {
+            return in.hashCode();
+        }
+
+        @SuppressWarnings("UnsynchronizedOverridesSynchronized")
+        @Override
+        public void mark(int readlimit) {
+            throw new UnsupportedOperationException("Caller does not own the underlying input stream "
+                    + " and should not call mark().");
+        }
+
+        @Override
+        public boolean markSupported() {
+            return false;
+        }
+
+        @SuppressWarnings("UnsynchronizedOverridesSynchronized")
+        @Override
+        public void reset() throws IOException {
+            throw new UnsupportedOperationException("Caller does not own the underlying input stream "
+                    + " and should not call reset().");
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/DoubleCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/DoubleCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/DoubleCoder.java
new file mode 100644
index 0000000..981bee2
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/DoubleCoder.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+
+public class DoubleCoder extends AtomicCoder<Double> {
+
+    public static DoubleCoder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final DoubleCoder INSTANCE = new DoubleCoder();
+
+    private DoubleCoder() {
+    }
+
+    @Override
+    public void encode(Double value, OutputStream outStream)
+            throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Double");
+        }
+        try {
+            new DataOutputStream(outStream).writeDouble(value);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public Double decode(InputStream inStream)
+            throws CoderException {
+        try {
+            return new DataInputStream(inStream).readDouble();
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public void verifyDeterministic() {
+        throw new NonDeterministicException(this,
+                "Floating point encodings are not guaranteed to be deterministic.");
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(Double value) {
+        return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(Double value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null Double");
+        }
+        return 8;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterable.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterable.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterable.java
new file mode 100644
index 0000000..29b4aa5
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterable.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Observer;
+
+public abstract class ElementByteSizeObservableIterable<
+        V, InputT extends ElementByteSizeObservableIterator<V>>
+        implements Iterable<V> {
+    private List<Observer> observers = new ArrayList<>();
+
+    protected abstract InputT createIterator();
+
+    public void addObserver(Observer observer) {
+        observers.add(observer);
+    }
+
+    @Override
+    public InputT iterator() {
+        InputT iterator = createIterator();
+        for (Observer observer : observers) {
+            iterator.addObserver(observer);
+        }
+        observers.clear();
+        return iterator;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterator.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterator.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterator.java
new file mode 100644
index 0000000..946882b
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObservableIterator.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.gearpump.streaming.refactor.coder;
+
+import java.util.Iterator;
+import java.util.Observable;
+
+public abstract class ElementByteSizeObservableIterator<V>
+        extends Observable implements Iterator<V> {
+    protected final void notifyValueReturned(long byteSize) {
+        setChanged();
+        notifyObservers(byteSize);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObserver.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObserver.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObserver.java
new file mode 100644
index 0000000..5464067
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ElementByteSizeObserver.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.util.Observable;
+import java.util.Observer;
+
+public abstract class ElementByteSizeObserver implements Observer {
+    private boolean isLazy = false;
+    private long totalSize = 0;
+    private double scalingFactor = 1.0;
+
+    public ElementByteSizeObserver() {
+    }
+
+    protected abstract void reportElementSize(long elementByteSize);
+
+    public void setLazy() {
+        isLazy = true;
+    }
+
+    public boolean getIsLazy() {
+        return isLazy;
+    }
+
+    public void update(Object obj) {
+        update(null, obj);
+    }
+
+    public void setScalingFactor(double scalingFactor) {
+        this.scalingFactor = scalingFactor;
+    }
+
+    @Override
+    public void update(Observable obs, Object obj) {
+        if (obj instanceof Long) {
+            totalSize += scalingFactor * (Long) obj;
+        } else if (obj instanceof Integer) {
+            totalSize += scalingFactor * (Integer) obj;
+        } else {
+            throw new AssertionError("unexpected parameter object");
+        }
+    }
+
+    public void advance() {
+        reportElementSize(totalSize);
+        totalSize = 0;
+        isLazy = false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableCoder.java
new file mode 100644
index 0000000..d069068
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableCoder.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.util.List;
+
+public class IterableCoder<T> extends IterableLikeCoder<T, Iterable<T>> {
+
+    public static <T> IterableCoder<T> of(Coder<T> elemCoder) {
+        return new IterableCoder<>(elemCoder);
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+    // Internal operations below here.
+
+    @Override
+    protected final Iterable<T> decodeToIterable(List<T> decodedElements) {
+        return decodedElements;
+    }
+
+    protected IterableCoder(Coder<T> elemCoder) {
+        super(elemCoder, "Iterable");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableLikeCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableLikeCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableLikeCoder.java
new file mode 100644
index 0000000..5bb6c66
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IterableLikeCoder.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.io.*;
+import java.util.*;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public abstract class IterableLikeCoder<T, IterableT extends Iterable<T>>
+        extends StructuredCoder<IterableT> {
+    public Coder<T> getElemCoder() {
+        return elementCoder;
+    }
+
+    protected abstract IterableT decodeToIterable(List<T> decodedElements);
+
+    /////////////////////////////////////////////////////////////////////////////
+    // Internal operations below here.
+
+    private final Coder<T> elementCoder;
+    private final String iterableName;
+
+    protected IterableLikeCoder(Coder<T> elementCoder, String  iterableName) {
+        checkArgument(elementCoder != null, "element Coder for IterableLikeCoder must not be null");
+        checkArgument(iterableName != null, "iterable name for IterableLikeCoder must not be null");
+        this.elementCoder = elementCoder;
+        this.iterableName = iterableName;
+    }
+
+    @Override
+    public void encode(IterableT iterable, OutputStream outStream) {
+        if (iterable == null) {
+            throw new CoderException("cannot encode a null " + iterableName);
+        }
+        DataOutputStream dataOutStream = new DataOutputStream(outStream);
+        try {
+            if (iterable instanceof Collection) {
+                // We can know the size of the Iterable.  Use an encoding with a
+                // leading size field, followed by that many elements.
+                Collection<T> collection = (Collection<T>) iterable;
+                dataOutStream.writeInt(collection.size());
+                for (T elem : collection) {
+                    elementCoder.encode(elem, dataOutStream);
+                }
+            } else {
+                // We don't know the size without traversing it so use a fixed size buffer
+                // and encode as many elements as possible into it before outputting the size followed
+                // by the elements.
+                dataOutStream.writeInt(-1);
+                BufferedElementCountingOutputStream countingOutputStream =
+                        new BufferedElementCountingOutputStream(dataOutStream);
+                for (T elem : iterable) {
+                    countingOutputStream.markElementStart();
+                    elementCoder.encode(elem, countingOutputStream);
+                }
+                countingOutputStream.finish();
+            }
+            // Make sure all our output gets pushed to the underlying outStream.
+            dataOutStream.flush();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public IterableT decode(InputStream inStream) {
+        try {
+            DataInputStream dataInStream = new DataInputStream(inStream);
+            int size = dataInStream.readInt();
+            if (size >= 0) {
+                List<T> elements = new ArrayList<>(size);
+                for (int i = 0; i < size; i++) {
+                    elements.add(elementCoder.decode(dataInStream));
+                }
+                return decodeToIterable(elements);
+            }
+            List<T> elements = new ArrayList<>();
+            // We don't know the size a priori.  Check if we're done with
+            // each block of elements.
+            long count = VarInt.decodeLong(dataInStream);
+            while (count > 0L) {
+                elements.add(elementCoder.decode(dataInStream));
+                --count;
+                if (count == 0L) {
+                    count = VarInt.decodeLong(dataInStream);
+                }
+            }
+            return decodeToIterable(elements);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+        return Arrays.asList(elementCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws Coder.NonDeterministicException {
+        throw new NonDeterministicException(this,
+                "IterableLikeCoder can not guarantee deterministic ordering.");
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(
+            IterableT iterable) {
+        return iterable instanceof ElementByteSizeObservableIterable;
+    }
+
+    @Override
+    public void registerByteSizeObserver(
+            IterableT iterable, ElementByteSizeObserver observer) {
+        if (iterable == null) {
+            throw new CoderException("cannot encode a null Iterable");
+        }
+
+        if (iterable instanceof ElementByteSizeObservableIterable) {
+            observer.setLazy();
+            ElementByteSizeObservableIterable<?, ?> observableIterable =
+                    (ElementByteSizeObservableIterable<?, ?>) iterable;
+            observableIterable.addObserver(
+                    new IteratorObserver(observer, iterable instanceof Collection));
+        } else {
+            if (iterable instanceof Collection) {
+                // We can know the size of the Iterable.  Use an encoding with a
+                // leading size field, followed by that many elements.
+                Collection<T> collection = (Collection<T>) iterable;
+                observer.update(4L);
+                for (T elem : collection) {
+                    elementCoder.registerByteSizeObserver(elem, observer);
+                }
+            } else {
+                // TODO: (BEAM-1537) Update to use an accurate count depending on size and count,
+                // currently we are under estimating the size by up to 10 bytes per block of data since we
+                // are not encoding the count prefix which occurs at most once per 64k of data and is upto
+                // 10 bytes long. Since we include the total count we can upper bound the underestimate
+                // to be 10 / 65536 ~= 0.0153% of the actual size.
+                observer.update(4L);
+                long count = 0;
+                for (T elem : iterable) {
+                    count += 1;
+                    elementCoder.registerByteSizeObserver(elem, observer);
+                }
+                if (count > 0) {
+                    // Update the length based upon the number of counted elements, this helps
+                    // eliminate the case where all the elements are encoded in the first block and
+                    // it is quite short (e.g. Long.MAX_VALUE nulls encoded with VoidCoder).
+                    observer.update(VarInt.getLength(count));
+                }
+                // Update with the terminator byte.
+                observer.update(1L);
+            }
+        }
+    }
+
+    private class IteratorObserver implements Observer {
+        private final ElementByteSizeObserver outerObserver;
+        private final boolean countable;
+
+        public IteratorObserver(ElementByteSizeObserver outerObserver,
+                                boolean countable) {
+            this.outerObserver = outerObserver;
+            this.countable = countable;
+
+            if (countable) {
+                // Additional 4 bytes are due to size.
+                outerObserver.update(4L);
+            } else {
+                // Additional 5 bytes are due to size = -1 (4 bytes) and
+                // hasNext = false (1 byte).
+                outerObserver.update(5L);
+            }
+        }
+
+        @Override
+        public void update(Observable obs, Object obj) {
+            if (!(obj instanceof Long)) {
+                throw new AssertionError("unexpected parameter object");
+            }
+
+            if (countable) {
+                outerObserver.update(obs, obj);
+            } else {
+                // Additional 1 byte is due to hasNext = true flag.
+                outerObserver.update(obs, 1 + (long) obj);
+            }
+        }
+    }
+
+    public static abstract class ElementByteSizeObservableIterable<
+            V, InputT extends ElementByteSizeObservableIterator<V>>
+            implements Iterable<V> {
+        private List<Observer> observers = new ArrayList<>();
+
+        protected abstract InputT createIterator();
+
+        public void addObserver(Observer observer) {
+            observers.add(observer);
+        }
+
+        @Override
+        public InputT iterator() {
+            InputT iterator = createIterator();
+            for (Observer observer : observers) {
+                iterator.addObserver(observer);
+            }
+            observers.clear();
+            return iterator;
+        }
+    }
+
+    public static abstract class ElementByteSizeObservableIterator<V>
+            extends Observable implements Iterator<V> {
+        protected final void notifyValueReturned(long byteSize) {
+            setChanged();
+            notifyObservers(byteSize);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IteratorObserver.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IteratorObserver.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IteratorObserver.java
new file mode 100644
index 0000000..f1be5fb
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/IteratorObserver.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import java.util.Observable;
+import java.util.Observer;
+
+public class IteratorObserver implements Observer {
+    private final ElementByteSizeObserver outerObserver;
+    private final boolean countable;
+
+    public IteratorObserver(ElementByteSizeObserver outerObserver,
+                            boolean countable) {
+        this.outerObserver = outerObserver;
+        this.countable = countable;
+
+        if (countable) {
+            // Additional 4 bytes are due to size.
+            outerObserver.update(4L);
+        } else {
+            // Additional 5 bytes are due to size = -1 (4 bytes) and
+            // hasNext = false (1 byte).
+            outerObserver.update(5L);
+        }
+    }
+
+    @Override
+    public void update(Observable obs, Object obj) {
+        if (!(obj instanceof Long)) {
+            throw new AssertionError("unexpected parameter object");
+        }
+
+        if (countable) {
+            outerObserver.update(obs, obj);
+        } else {
+            // Additional 1 byte is due to hasNext = true flag.
+            outerObserver.update(obs, 1 + (long) obj);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ListCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ListCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ListCoder.java
new file mode 100644
index 0000000..3858ec6
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/ListCoder.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.gearpump.streaming.refactor.coder;
+
+import java.util.List;
+
+public class ListCoder<T> extends IterableLikeCoder<T, List<T>> {
+
+    public static <T> ListCoder<T> of(Coder<T> elemCoder) {
+        return new ListCoder<>(elemCoder);
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+    // Internal operations below here.
+
+    @Override
+    protected final List<T> decodeToIterable(List<T> decodedElements) {
+        return decodedElements;
+    }
+
+    protected ListCoder(Coder<T> elemCoder) {
+        super(elemCoder, "List");
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+        verifyDeterministic(this, "ListCoder.elemCoder must be deterministic",
+                (Iterable<Coder<?>>)getElemCoder());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/MapCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/MapCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/MapCoder.java
new file mode 100644
index 0000000..66b983c
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/MapCoder.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import com.google.common.collect.Maps;
+
+import java.io.*;
+import java.util.*;
+
+public class MapCoder<K, V> extends StructuredCoder<Map<K, V>> {
+
+    public static <K, V> MapCoder<K, V> of(
+            Coder<K> keyCoder,
+            Coder<V> valueCoder) {
+        return new MapCoder<>(keyCoder, valueCoder);
+    }
+
+    public Coder<K> getKeyCoder() {
+        return keyCoder;
+    }
+
+    public Coder<V> getValueCoder() {
+        return valueCoder;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private Coder<K> keyCoder;
+    private Coder<V> valueCoder;
+
+    private MapCoder(Coder<K> keyCoder, Coder<V> valueCoder) {
+        this.keyCoder = keyCoder;
+        this.valueCoder = valueCoder;
+    }
+
+    @Override
+    public void encode(Map<K, V> map, OutputStream outStream) throws CoderException  {
+        if (map == null) {
+            throw new CoderException("cannot encode a null Map");
+        }
+        DataOutputStream dataOutStream = new DataOutputStream(outStream);
+
+        int size = map.size();
+        try {
+            dataOutStream.writeInt(size);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        if (size == 0) {
+            return;
+        }
+
+        // Since we handled size == 0 above, entry is guaranteed to exist before and after loop
+        Iterator<Map.Entry<K, V>> iterator = map.entrySet().iterator();
+        Map.Entry<K, V> entry = iterator.next();
+        while (iterator.hasNext()) {
+            keyCoder.encode(entry.getKey(), outStream);
+            valueCoder.encode(entry.getValue(), outStream);
+            entry = iterator.next();
+        }
+
+        keyCoder.encode(entry.getKey(), outStream);
+        valueCoder.encode(entry.getValue(), outStream);
+        // no flush needed as DataOutputStream does not buffer
+    }
+
+    @Override
+    public Map<K, V> decode(InputStream inStream) {
+        DataInputStream dataInStream = new DataInputStream(inStream);
+        int size = 0;
+        try {
+            size = dataInStream.readInt();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        if (size == 0) {
+            return Collections.emptyMap();
+        }
+
+        Map<K, V> retval = Maps.newHashMapWithExpectedSize(size);
+        for (int i = 0; i < size - 1; ++i) {
+            K key = keyCoder.decode(inStream);
+            V value = valueCoder.decode(inStream);
+            retval.put(key, value);
+        }
+
+        K key = keyCoder.decode(inStream);
+        V value = valueCoder.decode(inStream);
+        retval.put(key, value);
+        return retval;
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+        return Arrays.asList(keyCoder, valueCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+        throw new NonDeterministicException(this,
+                "Ordering of entries in a Map may be non-deterministic.");
+    }
+
+    @Override
+    public void registerByteSizeObserver(
+            Map<K, V> map, ElementByteSizeObserver observer) {
+        observer.update(4L);
+        if (map.isEmpty()) {
+            return;
+        }
+        Iterator<Map.Entry<K, V>> entries = map.entrySet().iterator();
+        Map.Entry<K, V> entry = entries.next();
+        while (entries.hasNext()) {
+            keyCoder.registerByteSizeObserver(entry.getKey(), observer);
+            valueCoder.registerByteSizeObserver(entry.getValue(), observer);
+            entry = entries.next();
+        }
+        keyCoder.registerByteSizeObserver(entry.getKey(), observer);
+        valueCoder.registerByteSizeObserver(entry.getValue(), observer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/SetCoder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/SetCoder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/SetCoder.java
new file mode 100644
index 0000000..4147732
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/SetCoder.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.gearpump.streaming.refactor.coder;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class SetCoder<T> extends IterableLikeCoder<T, Set<T>> {
+
+    public static <T> SetCoder<T> of(Coder<T> elementCoder) {
+        return new SetCoder<>(elementCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+        throw new NonDeterministicException(this,
+                "Ordering of elements in a set may be non-deterministic.");
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+    // Internal operations below here.
+
+    @Override
+    protected final Set<T> decodeToIterable(List<T> decodedElements) {
+        return new HashSet<>(decodedElements);
+    }
+
+    protected SetCoder(Coder<T> elemCoder) {
+        super(elemCoder, "Set");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StringUtf8Coder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StringUtf8Coder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StringUtf8Coder.java
new file mode 100644
index 0000000..73ea8eb
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StringUtf8Coder.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import com.google.common.base.Utf8;
+
+import java.io.*;
+import java.nio.charset.StandardCharsets;
+
+public class StringUtf8Coder extends AtomicCoder<String> {
+
+    public static StringUtf8Coder of() {
+        return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final StringUtf8Coder INSTANCE = new StringUtf8Coder();
+
+    private static void writeString(String value, DataOutputStream dos)
+            throws IOException {
+        byte[] bytes = value.getBytes(StandardCharsets.UTF_8);
+        VarInt.encode(bytes.length, dos);
+        dos.write(bytes);
+    }
+
+    private static String readString(DataInputStream dis) throws IOException {
+        int len = VarInt.decodeInt(dis);
+        if (len < 0) {
+            throw new CoderException("Invalid encoded string length: " + len);
+        }
+        byte[] bytes = new byte[len];
+        dis.readFully(bytes);
+        return new String(bytes, StandardCharsets.UTF_8);
+    }
+
+    private StringUtf8Coder() {
+    }
+
+    @Override
+    public void encode(String value, OutputStream outStream) throws CoderException {
+        if (value == null) {
+            throw new CoderException("cannot encode a null String");
+        }
+        try {
+            writeString(value, new DataOutputStream(outStream));
+        } catch (IOException e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public String decode(InputStream inStream)
+            throws CoderException {
+        try {
+            return readString(new DataInputStream(inStream));
+        } catch (EOFException | UTFDataFormatException exn) {
+            // These exceptions correspond to decoding problems, so change
+            // what kind of exception they're branded as.
+            throw new CoderException(exn);
+        } catch (Exception e) {
+            throw new CoderException(e);
+        }
+    }
+
+    @Override
+    public void verifyDeterministic() {
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+        return true;
+    }
+
+    @Override
+    public long getEncodedElementByteSize(String value) {
+        if (value == null) {
+            throw new CoderException("cannot encode a null String");
+        }
+        int size = Utf8.encodedLength(value);
+        return VarInt.getLength(size) + size;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/7068699d/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuralByteArray.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuralByteArray.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuralByteArray.java
new file mode 100644
index 0000000..6a371f6
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/StructuralByteArray.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.coder;
+
+import com.google.common.io.BaseEncoding;
+
+import java.util.Arrays;
+
+public class StructuralByteArray {
+    byte[] value;
+
+    public StructuralByteArray(byte[] value) {
+        this.value = value;
+    }
+
+    public byte[] getValue() {
+        return value;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o instanceof StructuralByteArray) {
+            StructuralByteArray that = (StructuralByteArray) o;
+            return Arrays.equals(this.value, that.value);
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return Arrays.hashCode(value);
+    }
+
+    @Override
+    public String toString() {
+        return "base64:" + BaseEncoding.base64().encode(value);
+    }
+}


[2/7] incubator-gearpump git commit: [GEARPUMP-311] refactor state management

Posted by ma...@apache.org.
[GEARPUMP-311] refactor state management


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/fe410304
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/fe410304
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/fe410304

Branch: refs/heads/state
Commit: fe4103046ae3bddacce2cd7a17270a994a494dbd
Parents: 50316ce
Author: vinoyang <ya...@gmail.com>
Authored: Fri Jul 21 20:32:10 2017 +0800
Committer: vinoyang <ya...@gmail.com>
Committed: Sat Jul 22 17:16:56 2017 +0800

----------------------------------------------------------------------
 .../state/refactor/ProduceProcessor.scala       |  53 +++++
 .../examples/state/refactor/SumProcessor.scala  |  92 ++++++++
 .../wordcountjava/dsl/refactor/WordCount.java   | 211 +++++++++++++++++++
 .../wordcount/dsl/refactor/WordCount.scala      | 151 +++++++++++++
 project/BuildExamples.scala                     |  36 +++-
 .../streaming/refactor/javaapi/Processor.java   | 143 +++++++++++++
 .../api/functions/MapWithStateFunction.scala    |  55 +++++
 .../refactor/dsl/javaapi/JavaStreamApp.scala    |  49 +++++
 .../functions/FlatMapWithStateFunction.scala    |  48 +++++
 .../streaming/refactor/dsl/plan/Planner.scala   |  90 ++++++++
 .../dsl/plan/functions/FunctionRunner.scala     | 135 ++++++++++++
 .../refactor/dsl/scalaapi/StreamApp.scala       | 116 ++++++++++
 .../functions/FlatMapWithStateFunction.scala    | 120 +++++++++++
 .../refactor/dsl/task/GroupByTask.scala         |  89 ++++++++
 .../streaming/refactor/dsl/task/TaskUtil.scala  |  65 ++++++
 .../refactor/dsl/task/TransformTask.scala       |  55 +++++
 .../dsl/window/impl/ReduceFnRunner.scala        |  29 +++
 .../refactor/dsl/window/impl/WindowRunner.scala | 160 ++++++++++++++
 .../refactor/sink/DataSinkProcessor.scala       |  36 ++++
 .../streaming/refactor/sink/DataSinkTask.scala  |  52 +++++
 .../refactor/source/DataSourceProcessor.scala   |  55 +++++
 .../refactor/source/DataSourceTask.scala        |  91 ++++++++
 .../refactor/state/RuntimeContext.scala         |  35 +++
 .../streaming/refactor/state/StatefulTask.scala | 174 +++++++++++++++
 24 files changed, 2136 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/ProduceProcessor.scala
----------------------------------------------------------------------
diff --git a/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/ProduceProcessor.scala b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/ProduceProcessor.scala
new file mode 100644
index 0000000..11f99e2
--- /dev/null
+++ b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/ProduceProcessor.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.examples.state.refactor
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StatefulTask}
+import org.apache.gearpump.streaming.task.TaskContext
+
+/**
+ *  a produce processor for generating a specific num sequence
+ */
+class ProduceProcessor(taskContext: TaskContext, conf: UserConfig)
+  extends StatefulTask(taskContext, conf) {
+
+  override def open(runtimeContext: RuntimeContext): Unit = {}
+
+  override def invoke(message: Message): Unit = {
+    message.value match {
+      case msgBytes: Array[Byte] => {
+        val msgStr = new String(msgBytes)
+        LOG.info("got total sequence num : {}", msgStr)
+
+        val n: Int = Integer.valueOf(msgStr)
+        var sumResult: Long = 0
+        for (i <- 1 to n) {
+          taskContext.output(Message(String.valueOf(i).getBytes))
+          sumResult = sumResult + i
+        }
+
+        LOG.info(" total sum result : {}", sumResult)
+      }
+    }
+  }
+
+  override def close(runtimeContext: RuntimeContext): Unit = {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/SumProcessor.scala
----------------------------------------------------------------------
diff --git a/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/SumProcessor.scala b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/SumProcessor.scala
new file mode 100644
index 0000000..438b337
--- /dev/null
+++ b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/refactor/SumProcessor.scala
@@ -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.gearpump.streaming.examples.state.refactor
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.refactor.coder._
+import org.apache.gearpump.streaming.refactor.state.api.{StateInternals, ValueState}
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StateNamespaces, StateTags, StatefulTask}
+import org.apache.gearpump.streaming.task.TaskContext
+
+/**
+ *  a sum processor for continues sum message from kafka
+ *  it is a example for using state api and verifying state exactly-once guarantee
+ */
+class SumProcessor(taskContext: TaskContext, conf: UserConfig)
+  extends StatefulTask(taskContext, conf) {
+
+  private implicit val valueStateTag = "tag1"
+  private implicit val counterStateTag = "tag2"
+
+  private var stateInternals: Option[StateInternals] = None
+  private var valueState: Option[ValueState[java.lang.Long]] = None
+  private var counterState: Option[ValueState[java.lang.Long]] = None
+
+  override def open(stateContext: RuntimeContext): Unit = {
+    stateInternals = Some(stateContext.getStateInternals(StringUtf8Coder.of, "partitionedKey"))
+    valueState = Some(
+      stateInternals.get.state(
+        StateNamespaces.global, StateTags.value(valueStateTag, VarLongCoder.of))
+    )
+
+    counterState = Some(
+      stateInternals.get.state(
+        StateNamespaces.global, StateTags.value(counterStateTag, VarLongCoder.of))
+    )
+
+    // init
+    if (valueState.get.read == null) {
+      LOG.info("[open] value state current is null, init it to 0")
+      valueState.get.write(0L)
+    } else {
+      LOG.info("[open] load from snapshot value state current is : {}", valueState.get.read)
+    }
+
+    if (counterState.get.read == null) {
+      LOG.info("[open] counter state current is null, init it to 0")
+      counterState.get.write(0L)
+    } else {
+      LOG.info("[open] load from snapshot counter state current is : {}", counterState.get.read)
+    }
+  }
+
+  override def invoke(message: Message): Unit = {
+    message.value match {
+      case numberByte: Array[Byte] => {
+        val number = new String(numberByte)
+        val oldVal = valueState.get.read
+        valueState.get.write(oldVal + java.lang.Long.valueOf(number))
+
+        val oldCounter = counterState.get.read
+        counterState.get.write(oldCounter + 1)
+      }
+
+      case other => LOG.error("received unsupported message {}", other)
+    }
+
+    if (counterState.get.read % 1000000 == 0) {
+      LOG.info("counter state is : {}", counterState.get.read)
+      LOG.info("value state is : {}", valueState.get.read)
+    }
+  }
+
+  override def close(stateContext: RuntimeContext): Unit = {}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/refactor/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/refactor/WordCount.java b/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/refactor/WordCount.java
new file mode 100644
index 0000000..900c35a
--- /dev/null
+++ b/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/refactor/WordCount.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.examples.wordcountjava.dsl.refactor;
+
+import com.typesafe.config.Config;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.gearpump.DefaultMessage;
+import org.apache.gearpump.Message;
+import org.apache.gearpump.cluster.ClusterConfig;
+import org.apache.gearpump.cluster.UserConfig;
+import org.apache.gearpump.cluster.client.ClientContext;
+import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction;
+import org.apache.gearpump.streaming.hadoop.HadoopCheckpointStoreFactory;
+import org.apache.gearpump.streaming.hadoop.lib.rotation.FileSizeRotation;
+import org.apache.gearpump.streaming.refactor.coder.StringUtf8Coder;
+import org.apache.gearpump.streaming.refactor.coder.VarLongCoder;
+import org.apache.gearpump.streaming.refactor.dsl.api.functions.MapWithStateFunction;
+import org.apache.gearpump.streaming.refactor.dsl.javaapi.JavaStream;
+import org.apache.gearpump.streaming.dsl.javaapi.functions.GroupByFunction;
+import org.apache.gearpump.streaming.refactor.dsl.javaapi.JavaStreamApp;
+import org.apache.gearpump.streaming.refactor.dsl.javaapi.functions.FlatMapWithStateFunction;
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext;
+import org.apache.gearpump.streaming.refactor.state.StateNamespaces;
+import org.apache.gearpump.streaming.refactor.state.StateTags;
+import org.apache.gearpump.streaming.refactor.state.api.StateInternals;
+import org.apache.gearpump.streaming.refactor.state.api.ValueState;
+import org.apache.gearpump.streaming.source.DataSource;
+import org.apache.gearpump.streaming.state.impl.PersistentStateConfig;
+import org.apache.gearpump.streaming.task.TaskContext;
+import org.apache.hadoop.conf.Configuration;
+import scala.Tuple2;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Iterator;
+
+/** Java version of WordCount with high level DSL API */
+public class WordCount {
+
+  public static void main(String[] args) throws InterruptedException {
+    main(ClusterConfig.defaultConfig(), args);
+  }
+
+  public static void main(Config akkaConf, String[] args) throws InterruptedException {
+    ClientContext context = new ClientContext(akkaConf);
+
+    Configuration hadoopConfig = new Configuration();
+    HadoopCheckpointStoreFactory checkpointStoreFactory = new HadoopCheckpointStoreFactory(
+            "MessageConsume", hadoopConfig,
+            // Rotates on 1MB
+            new FileSizeRotation(1000000)
+    );
+    UserConfig taskConfig = UserConfig.empty()
+            .withBoolean(PersistentStateConfig.STATE_CHECKPOINT_ENABLE(), true)
+            .withLong(PersistentStateConfig.STATE_CHECKPOINT_INTERVAL_MS(), 1000L)
+            .withValue(PersistentStateConfig.STATE_CHECKPOINT_STORE_FACTORY(),
+                    checkpointStoreFactory,
+                    context.system()
+            );
+
+    JavaStreamApp app = new JavaStreamApp("JavaDSL", context, taskConfig);
+
+    JavaStream<String> sentence = app.source(new StringSource("This is a good start, bingo!! bingo!!"),
+        1, UserConfig.empty(), "source");
+
+    JavaStream<String> words = sentence.flatMapWithState(new StatefulSplitFunction(), "flatMap");
+
+    JavaStream<Tuple2<String, Integer>> ones = words.mapWithState(new StatefulMapFunction(), "map");
+
+    JavaStream<Tuple2<String, Integer>> groupedOnes = ones.groupBy(new TupleKey(), 1, "groupBy");
+
+    JavaStream<Tuple2<String, Integer>> wordcount = groupedOnes.reduce(new Count(), "reduce");
+
+    wordcount.log();
+
+    app.submit().waitUntilFinish();
+    context.close();
+  }
+
+  private static class StringSource implements DataSource {
+
+    private final String str;
+
+    StringSource(String str) {
+      this.str = str;
+    }
+
+    @Override
+    public void open(TaskContext context, Instant startTime) {
+    }
+
+    @Override
+    public Message read() {
+      return new DefaultMessage(str, Instant.now());
+    }
+
+    @Override
+    public void close() {
+    }
+
+    @Override
+    public Instant getWatermark() {
+      return Instant.now();
+    }
+  }
+
+  private static class StatefulSplitFunction extends FlatMapWithStateFunction<String, String> {
+
+    private static final Log logger = LogFactory.getLog(StatefulSplitFunction.class);
+
+    private String counterStateTag = "tag1";
+
+    private StateInternals stateInternal;
+    private ValueState<Long> counterState;
+
+    @Override
+    public void setup(RuntimeContext runtimeContext) {
+      logger.info("StatefulSplitFunction setup.");
+      stateInternal = runtimeContext.getStateInternals(StringUtf8Coder.of(), "partitionedKey");
+
+      counterState = stateInternal.state(StateNamespaces.global(), StateTags.value(counterStateTag, VarLongCoder.of()));
+
+      if (counterState.read() == null) {
+        counterState.write(0L);
+      }
+    }
+
+    @Override
+    public Iterator<String> flatMap(String s) {
+      long oldVal = counterState.read();
+      logger.info("old value in flatMap : " + oldVal);
+      counterState.write(oldVal + 1);
+
+      return Arrays.asList(s.split("\\s+")).iterator();
+    }
+
+    @Override
+    public void teardown(RuntimeContext runtimeContext) {
+      logger.info("StatefulSplitFunction teardown.");
+    }
+  }
+
+  private static class StatefulMapFunction extends MapWithStateFunction<String, Tuple2<String, Integer>> {
+
+    private static final Log logger = LogFactory.getLog(StatefulMapFunction.class);
+
+    private String counterStateTag = "tag2";
+
+    private StateInternals stateInternal;
+    private ValueState<Long> counterState;
+
+    @Override
+    public void setup(RuntimeContext runtimeContext) {
+      logger.info("StatefulMapFunction setup.");
+      stateInternal = runtimeContext.getStateInternals(StringUtf8Coder.of(), "partitionedKey");
+
+      counterState = stateInternal.state(StateNamespaces.global(), StateTags.value(counterStateTag, VarLongCoder.of()));
+
+      if (counterState.read() == null) {
+        counterState.write(0L);
+      }
+    }
+
+    @Override
+    public Tuple2<String, Integer> map(String s) {
+      long oldVal = counterState.read();
+      logger.info("old value in map method : " + oldVal);
+      counterState.write(oldVal + 1);
+
+      return new Tuple2<>(s, 1);
+    }
+
+    @Override
+    public void teardown(RuntimeContext runtimeContext) {
+      logger.info("StatefulMapFunction teardown.");
+    }
+  }
+
+  private static class Count extends ReduceFunction<Tuple2<String, Integer>> {
+
+    @Override
+    public Tuple2<String, Integer> reduce(Tuple2<String, Integer> t1, Tuple2<String, Integer> t2) {
+      return new Tuple2<>(t1._1(), t1._2() + t2._2());
+    }
+  }
+
+  private static class TupleKey extends GroupByFunction<Tuple2<String, Integer>, String> {
+
+    @Override
+    public String groupBy(Tuple2<String, Integer> tuple) {
+      return tuple._1();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/refactor/WordCount.scala
----------------------------------------------------------------------
diff --git a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/refactor/WordCount.scala b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/refactor/WordCount.scala
new file mode 100644
index 0000000..a9919e2
--- /dev/null
+++ b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/refactor/WordCount.scala
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.examples.wordcount.dsl.refactor
+
+import org.apache.commons.logging.{Log, LogFactory}
+import org.apache.gearpump.streaming.refactor.coder.{StringUtf8Coder, VarLongCoder}
+import org.apache.gearpump.streaming.refactor.dsl.api.functions.MapWithStateFunction
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.StreamApp
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.functions.FlatMapWithStateFunction
+import org.apache.gearpump.streaming.refactor.state.api.{StateInternals, ValueState}
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StateNamespaces, StateTags}
+import org.apache.hadoop.conf.Configuration
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.cluster.client.ClientContext
+import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
+import org.apache.gearpump.streaming.hadoop.HadoopCheckpointStoreFactory
+import org.apache.gearpump.streaming.hadoop.lib.rotation.FileSizeRotation
+import org.apache.gearpump.streaming.state.impl.PersistentStateConfig
+import org.apache.gearpump.streaming.{Processor, StreamApplication}
+import org.apache.gearpump.util.{AkkaApp, Graph}
+
+/**
+ *
+ */
+object WordCount extends AkkaApp with ArgumentsParser {
+
+  override val options: Array[(String, CLIOption[Any])] = Array.empty
+
+  override def main(akkaConf: Config, args: Array[String]): Unit = {
+    val context = ClientContext(akkaConf)
+
+    val hadoopConfig = new Configuration
+    val checkpointStoreFactory = new HadoopCheckpointStoreFactory("MessageConsume", hadoopConfig,
+      // Rotates on 1MB
+      new FileSizeRotation(1000000))
+    val taskConfig = UserConfig.empty
+      .withBoolean(PersistentStateConfig.STATE_CHECKPOINT_ENABLE, true)
+      .withLong(PersistentStateConfig.STATE_CHECKPOINT_INTERVAL_MS, 1000L)
+      .withValue(PersistentStateConfig.STATE_CHECKPOINT_STORE_FACTORY,
+        checkpointStoreFactory)(context.system)
+
+    val app = StreamApp("dsl", context, taskConfig)
+    val data = "This is a good start, bingo!! bingo!!"
+    app.source(data.lines.toList, 1, "source").
+      // word => (word, count)
+      flatMapWithState(new StatefulFlatMapFunction(), "a stateful flatmap udf").
+      mapWithState(new StatefulMapFunction(), "").
+      // (word, count1), (word, count2) => (word, count1 + count2)
+      groupByKey().sum.log
+
+    context.submit(app).waitUntilFinish()
+    context.close()
+  }
+
+
+  private class StatefulFlatMapFunction
+    extends FlatMapWithStateFunction[String, String] {
+
+    private val logger: Log = LogFactory.getLog(getClass)
+
+    private implicit val counterStateTag = "tag1"
+
+    private var stateInternals: Option[StateInternals] = None
+    private var counterState: Option[ValueState[java.lang.Long]] = None
+
+    override def setup(runtimeContext: RuntimeContext): Unit = {
+      logger.info("StatefulFlatMapFunction setup.")
+      stateInternals = Some(runtimeContext.getStateInternals(StringUtf8Coder.of, "partitionedKey"))
+
+      counterState = Some(
+        stateInternals.get.state(
+          StateNamespaces.global, StateTags.value(counterStateTag, VarLongCoder.of))
+      )
+
+      // init
+      if (counterState.get.read == null) {
+        counterState.get.write(0L)
+      }
+    }
+
+
+    override def flatMap(t: String): TraversableOnce[String] = {
+      val oldVal = counterState.get.read
+      logger.info("old value in flatmap : " + oldVal)
+      counterState.get.write(oldVal + 1)
+
+      t.split("[\\s]+")
+    }
+
+    override def teardown(runtimeContext: RuntimeContext): Unit = {
+      logger.info("StatefulFlatMapFunction teardown.")
+    }
+
+  }
+
+  private class StatefulMapFunction
+    extends MapWithStateFunction[String, (String, Int)] {
+
+    private val logger: Log = LogFactory.getLog(getClass)
+
+    private implicit val counterStateTag = "tag2"
+
+    private var stateInternals: Option[StateInternals] = None
+    private var counterState: Option[ValueState[java.lang.Long]] = None
+
+    override def setup(runtimeContext: RuntimeContext): Unit = {
+      logger.info("StatefulMapFunction setup.")
+      stateInternals = Some(runtimeContext.getStateInternals(StringUtf8Coder.of, "partitionedKey"))
+
+      counterState = Some(
+        stateInternals.get.state(
+          StateNamespaces.global, StateTags.value(counterStateTag, VarLongCoder.of))
+      )
+
+      // init
+      if (counterState.get.read == null) {
+        counterState.get.write(0L)
+      }
+    }
+
+    override def map(t: String): (String, Int) = {
+      val oldVal = counterState.get.read
+      logger.info("old value in map : " + oldVal)
+      counterState.get.write(oldVal + 1)
+
+      (t, 1)
+    }
+
+    override def teardown(runtimeContext: RuntimeContext): Unit = {
+      logger.info("StatefulMapFunction teardown.")
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/project/BuildExamples.scala
----------------------------------------------------------------------
diff --git a/project/BuildExamples.scala b/project/BuildExamples.scala
index d7390b5..57210c0 100644
--- a/project/BuildExamples.scala
+++ b/project/BuildExamples.scala
@@ -62,14 +62,42 @@ object BuildExamples extends sbt.Build {
   lazy val wordcountJava = Project(
     id = "gearpump-examples-wordcountjava",
     base = file("examples/streaming/wordcount-java"),
-    settings = exampleSettings("org.apache.gearpump.streaming.examples.wordcountjava.WordCount")
-  ).dependsOn(core % "provided", streaming % "test->test; provided")
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.wordcountjava.WordCount") ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
+            exclude("org.mortbay.jetty", "jetty-util")
+            exclude("org.mortbay.jetty", "jetty")
+            exclude("org.fusesource.leveldbjni", "leveldbjni-all")
+            exclude("tomcat", "jasper-runtime")
+            exclude("commons-beanutils", "commons-beanutils-core")
+            exclude("commons-beanutils", "commons-beanutils")
+            exclude("asm", "asm")
+            exclude("org.ow2.asm", "asm"),
+          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided", external_hadoopfs)
 
   lazy val wordcount = Project(
     id = "gearpump-examples-wordcount",
     base = file("examples/streaming/wordcount"),
-    settings = exampleSettings("org.apache.gearpump.streaming.examples.wordcount.WordCount")
-  ).dependsOn(core % "provided", streaming % "test->test; provided")
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.wordcount.WordCount") ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
+            exclude("org.mortbay.jetty", "jetty-util")
+            exclude("org.mortbay.jetty", "jetty")
+            exclude("org.fusesource.leveldbjni", "leveldbjni-all")
+            exclude("tomcat", "jasper-runtime")
+            exclude("commons-beanutils", "commons-beanutils-core")
+            exclude("commons-beanutils", "commons-beanutils")
+            exclude("asm", "asm")
+            exclude("org.ow2.asm", "asm"),
+          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided", external_hadoopfs)
 
   lazy val sol = Project(
     id = "gearpump-examples-sol",

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/java/org/apache/gearpump/streaming/refactor/javaapi/Processor.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/javaapi/Processor.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/javaapi/Processor.java
new file mode 100644
index 0000000..7d85b09
--- /dev/null
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/javaapi/Processor.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.javaapi;
+
+import akka.actor.ActorSystem;
+import org.apache.gearpump.cluster.UserConfig;
+import org.apache.gearpump.streaming.javaapi.Task;
+import org.apache.gearpump.streaming.sink.DataSink;
+import org.apache.gearpump.streaming.refactor.sink.DataSinkProcessor;
+import org.apache.gearpump.streaming.refactor.sink.DataSinkTask;
+import org.apache.gearpump.streaming.source.DataSource;
+import org.apache.gearpump.streaming.refactor.source.DataSourceProcessor;
+import org.apache.gearpump.streaming.refactor.source.DataSourceTask;
+
+/**
+ * Java version of Processor
+ *
+ * See {@link org.apache.gearpump.streaming.Processor}
+ */
+public class Processor<T extends org.apache.gearpump.streaming.task.Task> implements org.apache.gearpump.streaming.Processor<T> {
+  private Class<T> _taskClass;
+  private int _parallelism = 1;
+  private String _description = "";
+  private UserConfig _userConf = UserConfig.empty();
+
+  public Processor(Class<T> taskClass) {
+    this._taskClass = taskClass;
+  }
+
+  public Processor(Class<T> taskClass, int parallelism) {
+    this._taskClass = taskClass;
+    this._parallelism = parallelism;
+  }
+
+  /**
+   * Creates a Sink Processor
+   *
+   * @param dataSink    the data sink itself
+   * @param parallelism the parallelism of this processor
+   * @param description the description for this processor
+   * @param taskConf    the configuration for this processor
+   * @param system      actor system
+   * @return the new created sink processor
+   */
+  public static Processor<DataSinkTask> sink(DataSink dataSink, int parallelism, String description, UserConfig taskConf, ActorSystem system) {
+    org.apache.gearpump.streaming.Processor<DataSinkTask> p = DataSinkProcessor.apply(dataSink, parallelism, description, taskConf, system);
+    return new Processor(p);
+  }
+
+  /**
+   * Creates a Source Processor
+   *
+   * @param source      the data source itself
+   * @param parallelism the parallelism of this processor
+   * @param description the description of this processor
+   * @param taskConf    the configuration of this processor
+   * @param system      actor system
+   * @return the new created source processor
+   */
+  public static Processor<DataSourceTask> source(DataSource source, int parallelism, String description, UserConfig taskConf, ActorSystem system) {
+    org.apache.gearpump.streaming.Processor<DataSourceTask<Object, Object>> p =
+        DataSourceProcessor.apply(source, parallelism, description, taskConf, system);
+    return new Processor(p);
+  }
+
+  public Processor(org.apache.gearpump.streaming.Processor<T> processor) {
+    this._taskClass = (Class) (processor.taskClass());
+    this._parallelism = processor.parallelism();
+    this._description = processor.description();
+    this._userConf = processor.taskConf();
+  }
+
+  /**
+   * Creates a general processor with user specified task logic.
+   *
+   * @param taskClass    task implementation class of this processor (shall be a derived class from {@link Task}
+   * @param parallelism, how many initial tasks you want to use
+   * @param description, some text to describe this processor
+   * @param taskConf,    Processor specific configuration
+   */
+  public Processor(Class<T> taskClass, int parallelism, String description, UserConfig taskConf) {
+    this._taskClass = taskClass;
+    this._parallelism = parallelism;
+    this._description = description;
+    this._userConf = taskConf;
+  }
+
+  public Processor<T> withParallelism(int parallel) {
+    return new Processor<T>(_taskClass, parallel, _description, _userConf);
+  }
+
+  public Processor<T> withDescription(String desc) {
+    return new Processor<T>(_taskClass, _parallelism, desc, _userConf);
+  }
+
+  public Processor<T> withConfig(UserConfig conf) {
+    return new Processor<T>(_taskClass, _parallelism, _description, conf);
+  }
+
+  @Override
+  public int parallelism() {
+    return _parallelism;
+  }
+
+  @Override
+  public UserConfig taskConf() {
+    return _userConf;
+  }
+
+  @Override
+  public String description() {
+    return _description;
+  }
+
+  @Override
+  public Class<? extends org.apache.gearpump.streaming.task.Task> taskClass() {
+    return _taskClass;
+  }
+
+  /**
+   * reference equal
+   */
+  @Override
+  public boolean equals(Object obj) {
+    return (this == obj);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/api/functions/MapWithStateFunction.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/api/functions/MapWithStateFunction.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/api/functions/MapWithStateFunction.scala
new file mode 100644
index 0000000..6724ab5
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/api/functions/MapWithStateFunction.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.dsl.api.functions
+
+import org.apache.gearpump.streaming.dsl.api.functions.MapFunction
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext
+
+object MapWithStateFunction {
+
+  def apply[T, R](fn: T => R): MapWithStateFunction[T, R] = {
+    new MapWithStateFunction[T, R] {
+      override def map(t: T): R = {
+        fn(t)
+      }
+    }
+  }
+
+}
+
+/**
+ *  map function support state
+ */
+abstract class MapWithStateFunction[T, R] extends MapFunction[T, R] {
+
+  final override def setup(): Unit = {
+    throw new UnsupportedOperationException("please call or override " +
+      "setup(runtimeContext: RuntimeContext) .")
+  }
+
+  final override def teardown(): Unit = {
+    throw new UnsupportedOperationException("please call or override " +
+      "teardown(runtimeContext: RuntimeContext) ")
+  }
+
+  def setup(runtimeContext: RuntimeContext): Unit = {}
+
+  def teardown(runtimeContext: RuntimeContext): Unit = {}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStreamApp.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStreamApp.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStreamApp.scala
new file mode 100644
index 0000000..5a88c17
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStreamApp.scala
@@ -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.gearpump.streaming.refactor.dsl.javaapi
+
+import java.util.Collection
+
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.cluster.client.{ClientContext, RunningApplication}
+import org.apache.gearpump.streaming.dsl.scalaapi.CollectionDataSource
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.StreamApp
+import org.apache.gearpump.streaming.source.DataSource
+
+import scala.collection.JavaConverters._
+
+class JavaStreamApp(name: String, context: ClientContext, userConfig: UserConfig) {
+
+  private val streamApp = StreamApp(name, context, userConfig)
+
+  def source[T](collection: Collection[T], parallelism: Int,
+      conf: UserConfig, description: String): JavaStream[T] = {
+    val dataSource = new CollectionDataSource(collection.asScala.toSeq)
+    source(dataSource, parallelism, conf, description)
+  }
+
+  def source[T](dataSource: DataSource, parallelism: Int,
+      conf: UserConfig, description: String): JavaStream[T] = {
+    new JavaStream[T](streamApp.source(dataSource, parallelism, conf, description))
+  }
+
+  def submit(): RunningApplication = {
+    context.submit(streamApp)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/functions/FlatMapWithStateFunction.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/functions/FlatMapWithStateFunction.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/functions/FlatMapWithStateFunction.scala
new file mode 100644
index 0000000..4ed48ab
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/functions/FlatMapWithStateFunction.scala
@@ -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.gearpump.streaming.refactor.dsl.javaapi.functions
+
+import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext
+
+/**
+ * Transforms one input into zero or more outputs of possibly different types.
+ * This Java version of FlatMapFunction returns a java.util.Iterator.
+ *
+ * @param T Input value type
+ * @param R Output value type
+ */
+abstract class FlatMapWithStateFunction[T, R] extends FlatMapFunction[T, R] {
+
+  def flatMap(t: T): java.util.Iterator[R]
+
+  final override def setup(): Unit = {
+    throw new UnsupportedOperationException("please call or override " +
+      "setup(runtimeContext: RuntimeContext) ")
+  }
+
+  final override def teardown(): Unit = {
+    throw new UnsupportedOperationException("please call or override " +
+      "teardown(runtimeContext: RuntimeContext) ")
+  }
+
+  def setup(runtimeContext: RuntimeContext): Unit = {}
+
+  def teardown(runtimeContext: RuntimeContext): Unit = {}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/Planner.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/Planner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/Planner.scala
new file mode 100644
index 0000000..03cdf43
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/Planner.scala
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.dsl.plan
+
+import akka.actor.ActorSystem
+import org.apache.gearpump.streaming.Processor
+import org.apache.gearpump.streaming.partitioner.{CoLocationPartitioner, GroupByPartitioner, HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.refactor.dsl.plan.functions._
+import org.apache.gearpump.streaming.task.Task
+import org.apache.gearpump.util.Graph
+
+class Planner {
+
+  /**
+   * Converts Dag of Op to Dag of TaskDescription. TaskDescription is part of the low
+   * level Graph API.
+   */
+  def plan(dag: Graph[Op, OpEdge])
+    (implicit system: ActorSystem): Graph[Processor[_ <: Task], _ <: Partitioner] = {
+
+    val graph = optimize(dag)
+    graph.mapEdge { (_, edge, node2) =>
+      edge match {
+        case Shuffle =>
+          node2 match {
+            case op: GroupByOp[_, _] =>
+              new GroupByPartitioner(op.groupBy)
+            case _ => new HashPartitioner
+          }
+        case Direct =>
+          new CoLocationPartitioner
+      }
+    }.mapVertex(_.toProcessor)
+  }
+
+  private def optimize(dag: Graph[Op, OpEdge])
+    (implicit system: ActorSystem): Graph[Op, OpEdge] = {
+    val graph = dag.copy
+    val nodes = graph.topologicalOrderWithCirclesIterator.toList.reverse
+    for (node <- nodes) {
+      val outGoingEdges = graph.outgoingEdgesOf(node)
+      for (edge <- outGoingEdges) {
+        merge(graph, edge._1, edge._3)
+      }
+    }
+    graph
+  }
+
+  private def merge(graph: Graph[Op, OpEdge], node1: Op, node2: Op)
+    (implicit system: ActorSystem): Unit = {
+    if (graph.outDegreeOf(node1) == 1 &&
+      graph.inDegreeOf(node2) == 1 &&
+      // For processor node, we don't allow it to merge with downstream operators
+      !node1.isInstanceOf[ProcessorOp[_ <: Task]] &&
+      !node2.isInstanceOf[ProcessorOp[_ <: Task]]) {
+      val (_, edge, _) = graph.outgoingEdgesOf(node1).head
+      if (edge == Direct) {
+        val chainedOp = node1.chain(node2)
+        graph.addVertex(chainedOp)
+        for (incomingEdge <- graph.incomingEdgesOf(node1)) {
+          graph.addEdge(incomingEdge._1, incomingEdge._2, chainedOp)
+        }
+
+        for (outgoingEdge <- graph.outgoingEdgesOf(node2)) {
+          graph.addEdge(chainedOp, outgoingEdge._2, outgoingEdge._3)
+        }
+
+        // Remove the old vertex
+        graph.removeVertex(node1)
+        graph.removeVertex(node2)
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/functions/FunctionRunner.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/functions/FunctionRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/functions/FunctionRunner.scala
new file mode 100644
index 0000000..27f4265
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/functions/FunctionRunner.scala
@@ -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.gearpump.streaming.refactor.dsl.plan.functions
+
+import org.apache.gearpump.streaming.dsl.api.functions.FoldFunction
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.functions.FlatMapWithStateFunction
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext
+
+/**
+ * Interface to invoke SerializableFunction methods
+ *
+ * @param IN input value type
+ * @param OUT output value type
+ */
+sealed trait FunctionRunner[IN, OUT] extends java.io.Serializable {
+
+  def setup(runtimeContext: RuntimeContext): Unit = {}
+
+  def process(value: IN): TraversableOnce[OUT]
+
+  def finish(): TraversableOnce[OUT] = None
+
+  def teardown(runtimeContext: RuntimeContext): Unit = {}
+
+  def description: String
+}
+
+case class AndThen[IN, MIDDLE, OUT](first: FunctionRunner[IN, MIDDLE],
+    second: FunctionRunner[MIDDLE, OUT])
+  extends FunctionRunner[IN, OUT] {
+
+  override def setup(runtimeContext: RuntimeContext): Unit = {
+    first.setup(runtimeContext)
+    second.setup(runtimeContext)
+  }
+
+  override def process(value: IN): TraversableOnce[OUT] = {
+    first.process(value).flatMap(second.process)
+  }
+
+  override def finish(): TraversableOnce[OUT] = {
+    val firstResult = first.finish().flatMap(second.process)
+    if (firstResult.isEmpty) {
+      second.finish()
+    } else {
+      firstResult
+    }
+  }
+
+  override def teardown(runtimeContext: RuntimeContext): Unit = {
+    first.teardown(runtimeContext)
+    second.teardown(runtimeContext)
+  }
+
+  override def description: String = {
+    Option(first.description).flatMap { description =>
+      Option(second.description).map(description + "." + _)
+    }.orNull
+  }
+}
+
+class FlatMapper[IN, OUT](fn: FlatMapFunction[IN, OUT], val description: String)
+  extends FunctionRunner[IN, OUT] {
+
+  override def setup(runtimeContext: RuntimeContext): Unit = {
+    if (fn.isInstanceOf[FlatMapWithStateFunction[IN, OUT]]) {
+      fn.asInstanceOf[FlatMapWithStateFunction[IN, OUT]].setup(runtimeContext)
+    } else {
+      fn.setup()
+    }
+  }
+
+  override def process(value: IN): TraversableOnce[OUT] = {
+    fn.flatMap(value)
+  }
+
+  override def teardown(runtimeContext: RuntimeContext): Unit = {
+    if (fn.isInstanceOf[FlatMapWithStateFunction[IN, OUT]]) {
+      fn.asInstanceOf[FlatMapWithStateFunction[IN, OUT]].teardown(runtimeContext)
+    } else {
+      fn.teardown()
+    }
+  }
+}
+
+class FoldRunner[T, A](fn: FoldFunction[T, A], val description: String)
+  extends FunctionRunner[T, A] {
+
+  private var state: Option[A] = None
+
+  override def setup(runtimeContext: RuntimeContext): Unit = {
+    // TODO
+    fn.setup()
+    state = Option(fn.init)
+  }
+
+  override def process(value: T): TraversableOnce[A] = {
+    state = state.map(fn.fold(_, value))
+    None
+  }
+
+  override def finish(): TraversableOnce[A] = {
+    state
+  }
+
+  override def teardown(runtimeContext: RuntimeContext): Unit = {
+    // TODO
+    state = None
+    fn.teardown()
+  }
+}
+
+class DummyRunner[T] extends FunctionRunner[T, T] {
+
+  override def process(value: T): TraversableOnce[T] = Option(value)
+
+  override def description: String = ""
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/StreamApp.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/StreamApp.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/StreamApp.scala
new file mode 100644
index 0000000..a8dd727
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/StreamApp.scala
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.dsl.scalaapi
+
+import java.time.Instant
+
+import akka.actor.ActorSystem
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.cluster.client.ClientContext
+import org.apache.gearpump.streaming.StreamApplication
+import org.apache.gearpump.streaming.refactor.dsl.plan._
+import org.apache.gearpump.streaming.refactor.dsl.plan.functions.{DataSourceOp, Op, OpEdge}
+import org.apache.gearpump.streaming.source.{DataSource, Watermark}
+import org.apache.gearpump.streaming.task.TaskContext
+import org.apache.gearpump.util.Graph
+
+import scala.language.implicitConversions
+
+/**
+ * Example:
+ * {{{
+ * val data = "This is a good start, bingo!! bingo!!"
+ * app.fromCollection(data.lines.toList).
+ * // word => (word, count)
+ * flatMap(line => line.split("[\\s]+")).map((_, 1)).
+ * // (word, count1), (word, count2) => (word, count1 + count2)
+ * groupBy(kv => kv._1).reduce(sum(_, _))
+ *
+ * val appId = context.submit(app)
+ * context.close()
+ * }}}
+ *
+ * @param name name of app
+ */
+class StreamApp(
+    name: String, system: ActorSystem, userConfig: UserConfig,
+    private val graph: Graph[Op, OpEdge]) {
+
+  def this(name: String, system: ActorSystem, userConfig: UserConfig) = {
+    this(name, system, userConfig, Graph.empty[Op, OpEdge])
+  }
+
+  def plan(): StreamApplication = {
+    implicit val actorSystem = system
+    val planner = new Planner
+    val dag = planner.plan(graph)
+    StreamApplication(name, dag, userConfig)
+  }
+}
+
+object StreamApp {
+  def apply(name: String, context: ClientContext, userConfig: UserConfig = UserConfig.empty)
+    : StreamApp = {
+    new StreamApp(name, context.system, userConfig)
+  }
+
+  implicit def streamAppToApplication(streamApp: StreamApp): StreamApplication = {
+    streamApp.plan()
+  }
+
+  implicit class Source(app: StreamApp) extends java.io.Serializable {
+
+    def source[T](dataSource: DataSource, parallelism: Int = 1,
+        conf: UserConfig = UserConfig.empty, description: String = "source"): Stream[T] = {
+      implicit val sourceOp = DataSourceOp(dataSource, parallelism, description, conf)
+      app.graph.addVertex(sourceOp)
+      new Stream[T](app.graph, sourceOp)
+    }
+
+    def source[T](seq: Seq[T], parallelism: Int, description: String): Stream[T] = {
+      this.source(new CollectionDataSource[T](seq), parallelism, UserConfig.empty, description)
+    }
+  }
+}
+
+/** A test message source which generated message sequence repeatedly. */
+class CollectionDataSource[T](seq: Seq[T]) extends DataSource {
+  private lazy val iterator: Iterator[T] = seq.iterator
+
+  override def open(context: TaskContext, startTime: Instant): Unit = {}
+
+  override def read(): Message = {
+    if (iterator.hasNext) {
+      Message(iterator.next(), Instant.now())
+    } else {
+      null
+    }
+  }
+
+  override def close(): Unit = {}
+
+  override def getWatermark: Instant = {
+    if (iterator.hasNext) {
+      Instant.now()
+    } else {
+      Watermark.MAX
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/functions/FlatMapWithStateFunction.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/functions/FlatMapWithStateFunction.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/functions/FlatMapWithStateFunction.scala
new file mode 100644
index 0000000..cb878d6
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/functions/FlatMapWithStateFunction.scala
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gearpump.streaming.refactor.dsl.scalaapi.functions
+
+import org.apache.gearpump.streaming.dsl.api.functions.{FilterFunction}
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
+import org.apache.gearpump.streaming.refactor.dsl.javaapi.functions.{FlatMapWithStateFunction => JFlatMapWithStateFunction}
+import org.apache.gearpump.streaming.refactor.dsl.api.functions.MapWithStateFunction
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext
+
+import scala.collection.JavaConverters._
+
+object FlatMapWithStateFunction {
+
+  def apply[T, R](fn: JFlatMapWithStateFunction[T, R]): FlatMapWithStateFunction[T, R] = {
+    new FlatMapWithStateFunction[T, R] {
+
+      override def setup(runtimeContext: RuntimeContext): Unit = {
+        fn.setup(runtimeContext)
+      }
+
+      override def flatMap(t: T): TraversableOnce[R] = {
+        fn.flatMap(t).asScala
+      }
+
+
+      override def teardown(runtimeContext: RuntimeContext): Unit = {
+        fn.teardown(runtimeContext)
+      }
+    }
+  }
+
+//  def apply[T, R](fn: T => TraversableOnce[R]): FlatMapWithStateFunction[T, R] = {
+//    new FlatMapWithStateFunction[T, R] {
+//      override def flatMap(t: T): TraversableOnce[R] = {
+//        fn(t)
+//      }
+//    }
+//  }
+
+  def apply[T, R](fn: MapWithStateFunction[T, R]): FlatMapWithStateFunction[T, R] = {
+    new FlatMapWithStateFunction[T, R] {
+
+      override def setup(runtimeContext: RuntimeContext): Unit = {
+        fn.setup(runtimeContext)
+      }
+
+      override def flatMap(t: T): TraversableOnce[R] = {
+        Option(fn.map(t))
+      }
+
+      override def teardown(runtimeContext: RuntimeContext): Unit = {
+        fn.teardown(runtimeContext)
+      }
+    }
+  }
+
+  def apply[T, R](fn: FilterFunction[T]): FlatMapWithStateFunction[T, T] = {
+    new FlatMapWithStateFunction[T, T] {
+
+      override def setup(runtimeContext: RuntimeContext): Unit = {
+        // TODO
+        fn.setup()
+      }
+
+      override def flatMap(t: T): TraversableOnce[T] = {
+        if (fn.filter(t)) {
+          Option(t)
+        } else {
+          None
+        }
+      }
+
+      override def teardown(runtimeContext: RuntimeContext): Unit = {
+        // TODO
+        fn.teardown()
+      }
+    }
+  }
+}
+
+/**
+ * Transforms one input into zero or more outputs of possibly different types.
+ * This Scala version of FlatMapFunction returns a TraversableOnce.
+ *
+ * @param T Input value type
+ * @param R Output value type
+ */
+abstract class FlatMapWithStateFunction[T, R] extends FlatMapFunction[T, R] {
+
+  final override def setup(): Unit = {
+    throw new UnsupportedOperationException("please call or override " +
+      "setup(runtimeContext: RuntimeContext)")
+  }
+
+  final override def teardown(): Unit = {
+    throw new UnsupportedOperationException("please call or override " +
+      " teardown(runtimeContext: RuntimeContext)")
+  }
+
+  def setup(runtimeContext: RuntimeContext): Unit = {}
+
+  def teardown(runtimeContext: RuntimeContext): Unit = {}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/GroupByTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/GroupByTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/GroupByTask.scala
new file mode 100644
index 0000000..16f8cc9
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/GroupByTask.scala
@@ -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.gearpump.streaming.refactor.dsl.task
+
+import java.time.Instant
+import java.util.function.Consumer
+
+import com.gs.collections.impl.map.mutable.UnifiedMap
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.Constants.{GEARPUMP_STREAMING_GROUPBY_FUNCTION, GEARPUMP_STREAMING_OPERATOR}
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{TimestampedValue, WindowRunner}
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StatefulTask}
+import org.apache.gearpump.streaming.source.Watermark
+import org.apache.gearpump.streaming.task.{Task, TaskContext, TaskUtil}
+
+/**
+ * Processes messages in groups as defined by groupBy function.
+ */
+class GroupByTask[IN, GROUP, OUT](
+    groupBy: IN => GROUP,
+    taskContext: TaskContext,
+    userConfig: UserConfig) extends StatefulTask(taskContext, userConfig) {
+
+  private var runtimeContext: RuntimeContext = null
+
+  def this(context: TaskContext, conf: UserConfig) = {
+    this(
+      conf.getValue[IN => GROUP](GEARPUMP_STREAMING_GROUPBY_FUNCTION)(context.system).get,
+      context, conf
+    )
+  }
+
+  private val groups: UnifiedMap[GROUP, WindowRunner[IN, OUT]] =
+    new UnifiedMap[GROUP, WindowRunner[IN, OUT]]
+
+
+  override def open(runtimeContext: RuntimeContext): Unit = {
+    this.runtimeContext = runtimeContext
+  }
+
+  override def invoke(message: Message): Unit = {
+    val input = message.value.asInstanceOf[IN]
+    val group = groupBy(input)
+
+    if (!groups.containsKey(group)) {
+      groups.put(group,
+        userConfig.getValue[WindowRunner[IN, OUT]](
+          GEARPUMP_STREAMING_OPERATOR)(taskContext.system).get)
+    }
+
+    groups.get(group).process(TimestampedValue(message.value.asInstanceOf[IN],
+      message.timestamp))
+  }
+
+  override def onWatermarkProgress(watermark: Instant): Unit = {
+    if (groups.isEmpty && watermark == Watermark.MAX) {
+      taskContext.updateWatermark(Watermark.MAX)
+    } else {
+      groups.values.forEach(new Consumer[WindowRunner[IN, OUT]] {
+        override def accept(runner: WindowRunner[IN, OUT]): Unit = {
+          TaskUtil.trigger(watermark, runner, taskContext, runtimeContext)
+        }
+      })
+    }
+
+    super.onWatermarkProgress(watermark)
+  }
+
+  override def close(runtimeContext: RuntimeContext): Unit = {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TaskUtil.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TaskUtil.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TaskUtil.scala
new file mode 100644
index 0000000..fa474ec
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TaskUtil.scala
@@ -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.gearpump.streaming.refactor.dsl.task
+
+import java.time.Instant
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{TimestampedValue, WindowRunner}
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext
+import org.apache.gearpump.streaming.task._
+
+object TaskUtil {
+
+  /**
+   * Resolves a classname to a Task class.
+   *
+   * @param className  the class name to resolve
+   * @return resolved class
+   */
+  def loadClass(className: String): Class[_ <: Task] = {
+    val loader = Thread.currentThread().getContextClassLoader()
+    loader.loadClass(className).asSubclass(classOf[Task])
+  }
+
+  def trigger[IN, OUT](watermark: Instant, runner: WindowRunner[IN, OUT],
+      context: TaskContext, runtimeContext: RuntimeContext): Unit = {
+    val triggeredOutputs = runner.trigger(watermark, runtimeContext)
+    context.updateWatermark(triggeredOutputs.watermark)
+    triggeredOutputs.outputs.foreach { case TimestampedValue(v, t) =>
+      context.output(Message(v, t))
+    }
+  }
+
+  /**
+   * @return t1 if t1 is not larger than t2 and t2 otherwise
+   */
+  def min(t1: Instant, t2: Instant): Instant = {
+    if (t1.isAfter(t2)) t2
+    else t1
+  }
+
+  /**
+   * @return t1 if t1 is not smaller than t2 and t2 otherwise
+   */
+  def max(t1: Instant, t2: Instant): Instant = {
+    if (t2.isBefore(t1)) t1
+    else t2
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TransformTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TransformTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TransformTask.scala
new file mode 100644
index 0000000..b7dc895
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/task/TransformTask.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gearpump.streaming.refactor.dsl.task
+
+import java.time.Instant
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.Constants._
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{TimestampedValue, WindowRunner}
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StatefulTask}
+import org.apache.gearpump.streaming.task.{TaskContext}
+
+class TransformTask[IN, OUT](
+    runner: WindowRunner[IN, OUT],
+    taskContext: TaskContext, userConf: UserConfig) extends StatefulTask(taskContext, userConf) {
+
+  private var runtimeContext: RuntimeContext = null
+
+  def this(context: TaskContext, conf: UserConfig) = {
+    this(
+      conf.getValue[WindowRunner[IN, OUT]](GEARPUMP_STREAMING_OPERATOR)(context.system).get,
+      context, conf
+    )
+  }
+
+  override def open(stateContext: RuntimeContext): Unit = {
+    this.runtimeContext = stateContext
+  }
+
+  override def invoke(msg: Message): Unit = {
+    runner.process(TimestampedValue(msg.value.asInstanceOf[IN], msg.timestamp))
+  }
+
+  override def onWatermarkProgress(watermark: Instant): Unit = {
+    TaskUtil.trigger(watermark, runner, taskContext, this.runtimeContext)
+    // do checkpoint
+    super.onWatermarkProgress(watermark)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
new file mode 100644
index 0000000..17c93bd
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
@@ -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.gearpump.streaming.refactor.dsl.window.impl
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.streaming.dsl.window.api.Trigger
+
+trait ReduceFnRunner {
+
+  def process(message: Message): Unit
+
+  def onTrigger(trigger: Trigger): Unit
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/WindowRunner.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/WindowRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/WindowRunner.scala
new file mode 100644
index 0000000..43d28ef
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/WindowRunner.scala
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gearpump.streaming.refactor.dsl.window.impl
+
+import java.time.Instant
+
+import com.gs.collections.api.block.predicate.Predicate
+import com.gs.collections.api.block.procedure.Procedure
+import com.gs.collections.impl.list.mutable.FastList
+import com.gs.collections.impl.map.sorted.mutable.TreeSortedMap
+import org.apache.gearpump.streaming.dsl.window.api.WindowFunction.Context
+import org.apache.gearpump.streaming.dsl.window.api.{Discarding, Windows}
+import org.apache.gearpump.streaming.dsl.window.impl.Window
+import org.apache.gearpump.streaming.refactor.dsl.plan.functions.FunctionRunner
+import org.apache.gearpump.streaming.refactor.state.RuntimeContext
+import org.apache.gearpump.streaming.source.Watermark
+import org.apache.gearpump.streaming.task.TaskUtil
+
+import scala.collection.mutable.ArrayBuffer
+
+case class TimestampedValue[T](value: T, timestamp: Instant)
+
+case class TriggeredOutputs[T](outputs: TraversableOnce[TimestampedValue[T]],
+    watermark: Instant)
+
+trait WindowRunner[IN, OUT] extends java.io.Serializable {
+
+  def process(timestampedValue: TimestampedValue[IN]): Unit
+
+  def trigger(time: Instant, runtimeContext: RuntimeContext): TriggeredOutputs[OUT]
+
+}
+
+case class AndThen[IN, MIDDLE, OUT](left: WindowRunner[IN, MIDDLE],
+    right: WindowRunner[MIDDLE, OUT]) extends WindowRunner[IN, OUT] {
+
+  override def process(timestampedValue: TimestampedValue[IN]): Unit = {
+    left.process(timestampedValue)
+  }
+
+  override def trigger(time: Instant, runtimeContext: RuntimeContext): TriggeredOutputs[OUT] = {
+    val lOutputs = left.trigger(time, runtimeContext)
+    lOutputs.outputs.foreach(right.process)
+    right.trigger(lOutputs.watermark, runtimeContext)
+  }
+}
+
+class DefaultWindowRunner[IN, OUT](
+    windows: Windows,
+    fnRunner: FunctionRunner[IN, OUT])
+  extends WindowRunner[IN, OUT] {
+
+  private val windowFn = windows.windowFn
+  private val windowInputs = new TreeSortedMap[Window, FastList[TimestampedValue[IN]]]
+  private var setup = false
+  private var watermark = Watermark.MIN
+
+  override def process(timestampedValue: TimestampedValue[IN]): Unit = {
+    val wins = windowFn(new Context[IN] {
+      override def element: IN = timestampedValue.value
+
+      override def timestamp: Instant = timestampedValue.timestamp
+    })
+    wins.foreach { win =>
+      if (windowFn.isNonMerging) {
+        if (!windowInputs.containsKey(win)) {
+          val inputs = new FastList[TimestampedValue[IN]]
+          windowInputs.put(win, inputs)
+        }
+        windowInputs.get(win).add(timestampedValue)
+      } else {
+        merge(windowInputs, win, timestampedValue)
+      }
+    }
+
+    def merge(
+        winIns: TreeSortedMap[Window, FastList[TimestampedValue[IN]]],
+        win: Window, tv: TimestampedValue[IN]): Unit = {
+      val intersected = winIns.keySet.select(new Predicate[Window] {
+        override def accept(each: Window): Boolean = {
+          win.intersects(each)
+        }
+      })
+      var mergedWin = win
+      val mergedInputs = FastList.newListWith(tv)
+      intersected.forEach(new Procedure[Window] {
+        override def value(each: Window): Unit = {
+          mergedWin = mergedWin.span(each)
+          mergedInputs.addAll(winIns.remove(each))
+        }
+      })
+      winIns.put(mergedWin, mergedInputs)
+    }
+  }
+
+  override def trigger(time: Instant, runtimeContext: RuntimeContext): TriggeredOutputs[OUT] = {
+    @annotation.tailrec
+    def onTrigger(
+        outputs: ArrayBuffer[TimestampedValue[OUT]],
+        wmk: Instant): TriggeredOutputs[OUT] = {
+      if (windowInputs.notEmpty()) {
+        val firstWin = windowInputs.firstKey
+        if (!time.isBefore(firstWin.endTime)) {
+          val inputs = windowInputs.remove(firstWin)
+          if (!setup) {
+            fnRunner.setup(runtimeContext)
+            setup = true
+          }
+          inputs.forEach(new Procedure[TimestampedValue[IN]] {
+            override def value(tv: TimestampedValue[IN]): Unit = {
+              fnRunner.process(tv.value).foreach {
+                out: OUT => outputs += TimestampedValue(out, tv.timestamp)
+              }
+            }
+          })
+          fnRunner.finish().foreach {
+            out: OUT =>
+              outputs += TimestampedValue(out, firstWin.endTime.minusMillis(1))
+          }
+          val newWmk = TaskUtil.max(wmk, firstWin.endTime)
+          if (windows.accumulationMode == Discarding) {
+            fnRunner.teardown(runtimeContext)
+            // discarding, setup need to be called for each window
+            setup = false
+          }
+          onTrigger(outputs, newWmk)
+        } else {
+          // minimum of end of last triggered window and start of first un-triggered window
+          TriggeredOutputs(outputs, TaskUtil.min(wmk, firstWin.startTime))
+        }
+      } else {
+        if (time == Watermark.MAX) {
+          TriggeredOutputs(outputs, Watermark.MAX)
+        } else {
+          TriggeredOutputs(outputs, wmk)
+        }
+      }
+    }
+
+    val triggeredOutputs = onTrigger(ArrayBuffer.empty[TimestampedValue[OUT]], watermark)
+    watermark = TaskUtil.max(watermark, triggeredOutputs.watermark)
+    TriggeredOutputs(triggeredOutputs.outputs, watermark)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
new file mode 100644
index 0000000..d0b84cb
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
@@ -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.gearpump.streaming.refactor.sink
+
+import akka.actor.ActorSystem
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.Processor
+import org.apache.gearpump.streaming.sink.DataSink
+
+object DataSinkProcessor {
+  def apply(
+      dataSink: DataSink,
+      parallelism: Int = 1,
+      description: String = "",
+      taskConf: UserConfig = UserConfig.empty)(implicit system: ActorSystem)
+    : Processor[DataSinkTask] = {
+    Processor[DataSinkTask](parallelism, description = description,
+      taskConf.withValue[DataSink](DataSinkTask.DATA_SINK, dataSink))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkTask.scala
new file mode 100644
index 0000000..24be828
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkTask.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.sink
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StatefulTask}
+import org.apache.gearpump.streaming.sink.DataSink
+import org.apache.gearpump.streaming.task.{TaskContext}
+
+object DataSinkTask {
+  val DATA_SINK = "data_sink"
+}
+
+class DataSinkTask private[sink](context: TaskContext, conf: UserConfig, sink: DataSink)
+  extends StatefulTask(context, conf) {
+
+  def this(context: TaskContext, conf: UserConfig) = {
+    this(context, conf, conf.getValue[DataSink](DataSinkTask.DATA_SINK)(context.system).get)
+  }
+
+  override def open(runtimeContext: RuntimeContext): Unit = {
+    LOG.info("opening data sink...")
+    sink.open(context)
+  }
+
+  override def invoke(message: Message): Unit = {
+    sink.write(message)
+  }
+
+  override def close(runtimeContext: RuntimeContext): Unit = {
+    LOG.info("closing data sink...")
+    sink.close()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceProcessor.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceProcessor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceProcessor.scala
new file mode 100644
index 0000000..de584d5
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceProcessor.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.source
+
+import akka.actor.ActorSystem
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.dsl.window.api.{WindowFunction, Windows}
+import org.apache.gearpump.streaming.dsl.window.impl.Window
+import org.apache.gearpump.streaming.refactor.dsl.plan.functions.DummyRunner
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{DefaultWindowRunner, WindowRunner}
+import org.apache.gearpump.streaming.source.DataSource
+import org.apache.gearpump.streaming.{Constants, Processor}
+
+object DataSourceProcessor {
+  def apply(
+      dataSource: DataSource,
+      parallelism: Int = 1,
+      description: String = "",
+      taskConf: UserConfig = UserConfig.empty)(implicit system: ActorSystem)
+    : Processor[DataSourceTask[Any, Any]] = {
+    Processor[DataSourceTask[Any, Any]](parallelism, description,
+      taskConf
+        .withValue[DataSource](Constants.GEARPUMP_STREAMING_SOURCE, dataSource)
+        .withValue[WindowRunner[Any, Any]](Constants.GEARPUMP_STREAMING_OPERATOR,
+        new DefaultWindowRunner[Any, Any](
+          Windows(PerElementWindowFunction, description = "perElementWindows"),
+          new DummyRunner[Any])))
+  }
+
+
+  case object PerElementWindowFunction extends WindowFunction {
+    override def apply[T](
+        context: WindowFunction.Context[T]): Array[Window] = {
+      Array(Window(context.timestamp, context.timestamp.plusMillis(1)))
+    }
+
+    override def isNonMerging: Boolean = true
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceTask.scala
new file mode 100644
index 0000000..ac6468d
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/source/DataSourceTask.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.source
+
+import java.time.Instant
+
+import org.apache.gearpump._
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.Constants._
+import org.apache.gearpump.streaming.refactor.dsl.task.TaskUtil
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{TimestampedValue, WindowRunner}
+import org.apache.gearpump.streaming.refactor.state.{RuntimeContext, StatefulTask}
+import org.apache.gearpump.streaming.source.{DataSource, DataSourceConfig, Watermark}
+import org.apache.gearpump.streaming.task.{Task, TaskContext}
+
+/**
+ * Default Task container for [[org.apache.gearpump.streaming.source.DataSource]] that
+ * reads from DataSource in batch
+ * See [[org.apache.gearpump.streaming.source.DataSourceProcessor]] for its usage
+ *
+ * DataSourceTask calls:
+ *  - `DataSource.open()` in `onStart` and pass in
+ *  [[org.apache.gearpump.streaming.task.TaskContext]]
+ * and application start time
+ *  - `DataSource.read()` in each `onNext`, which reads a batch of messages
+ *  - `DataSource.close()` in `onStop`
+ */
+class DataSourceTask[IN, OUT] private[source](
+    source: DataSource,
+    windowRunner: WindowRunner[IN, OUT],
+    context: TaskContext,
+    conf: UserConfig)
+  extends StatefulTask(context, conf) {
+
+  private var runtimeContext: RuntimeContext = null
+
+  def this(context: TaskContext, conf: UserConfig) = {
+    this(
+      conf.getValue[DataSource](GEARPUMP_STREAMING_SOURCE)(context.system).get,
+      conf.getValue[WindowRunner[IN, OUT]](GEARPUMP_STREAMING_OPERATOR)(context.system).get,
+      context, conf
+    )
+  }
+
+  private val batchSize = conf.getInt(DataSourceConfig.SOURCE_READ_BATCH_SIZE).getOrElse(1000)
+
+  override def open(runtimeContext: RuntimeContext): Unit = {
+    this.runtimeContext = runtimeContext
+    LOG.info(s"opening data source at ${runtimeContext.getStartTime.toEpochMilli}")
+    source.open(context, runtimeContext.getStartTime)
+    self ! Watermark(source.getWatermark)
+    super.open(runtimeContext)
+  }
+
+  override def invoke(m: Message): Unit = {
+    0.until(batchSize).foreach { _ =>
+      Option(source.read()).foreach(
+        msg => windowRunner.process(
+          TimestampedValue(msg.value.asInstanceOf[IN], msg.timestamp)))
+    }
+
+    self ! Watermark(source.getWatermark)
+  }
+
+  override def onWatermarkProgress(watermark: Instant): Unit = {
+    TaskUtil.trigger(watermark, windowRunner, context, this.runtimeContext)
+  }
+
+
+  override def close(stateContext: RuntimeContext): Unit = {
+    LOG.info("closing data source...")
+    source.close()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/fe410304/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
new file mode 100644
index 0000000..c387960
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state
+
+import java.time.Instant
+
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.api.StateInternals
+
+/**
+ *
+ */
+trait RuntimeContext {
+
+  def getStateInternals[KT](keyCoder: Coder[KT], key: KT): StateInternals
+
+  def getStartTime: Instant
+
+}



[7/7] incubator-gearpump git commit: [Gearpump 311] refactor state management

Posted by ma...@apache.org.
[Gearpump 311] refactor state management


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/db8abf99
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/db8abf99
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/db8abf99

Branch: refs/heads/state
Commit: db8abf99fe3dd7c8d00975cbc0832cfa7e3b240f
Parents: 7068699
Author: vinoyang <ya...@gmail.com>
Authored: Wed Jun 21 23:51:47 2017 +0800
Committer: vinoyang <ya...@gmail.com>
Committed: Sun Jul 23 10:41:01 2017 +0800

----------------------------------------------------------------------
 .../streaming/refactor/coder/Coder.java         |  2 --
 .../dsl/window/impl/ReduceFnRunner.scala        |  8 +----
 .../refactor/sink/DataSinkProcessor.scala       | 21 ++-----------
 .../refactor/state/RuntimeContext.scala         |  9 ------
 .../streaming/refactor/state/StateSpec.scala    | 32 +++++++++++++++++++
 .../streaming/refactor/state/StateTag.scala     | 33 ++++++++++++++++++++
 .../streaming/refactor/state/StatefulTask.scala | 12 -------
 .../streaming/refactor/state/api/State.scala    | 25 +++++++++++++++
 8 files changed, 93 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
index e1999ed..edbe9a1 100644
--- a/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/refactor/coder/Coder.java
@@ -19,7 +19,6 @@
 package org.apache.gearpump.streaming.refactor.coder;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Objects;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.CountingOutputStream;
 
@@ -130,5 +129,4 @@ public abstract class Coder<T> implements Serializable {
                     coder, Joiner.on("%n  ").join(reasons));
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
index e706f4f..3fb8034 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
@@ -20,16 +20,10 @@ package org.apache.gearpump.streaming.refactor.dsl.window.impl
 import org.apache.gearpump.Message
 import org.apache.gearpump.streaming.dsl.window.api.Trigger
 
-<<<<<<< HEAD:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/window/impl/ReduceFnRunner.scala
 trait ReduceFnRunner {
 
   def process(message: Message): Unit
 
   def onTrigger(trigger: Trigger): Unit
-=======
-trait State {
 
-  def clear: Unit
->>>>>>> e6ce91c... [Gearpump 311] refactor state management:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/State.scala
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
index 6665766..e79f271 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-<<<<<<< HEAD:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/sink/DataSinkProcessor.scala
 package org.apache.gearpump.streaming.refactor.sink
 
 import akka.actor.ActorSystem
@@ -30,24 +29,8 @@ object DataSinkProcessor {
       parallelism: Int = 1,
       description: String = "",
       taskConf: UserConfig = UserConfig.empty)(implicit system: ActorSystem)
-    : Processor[DataSinkTask] = {
+  : Processor[DataSinkTask] = {
     Processor[DataSinkTask](parallelism, description = description,
       taskConf.withValue[DataSink](DataSinkTask.DATA_SINK, dataSink))
   }
-=======
-package org.apache.gearpump.streaming.refactor.state
-
-import org.apache.gearpump.streaming.refactor.state.api.State
-
-trait StateTag[StateT <: State] extends Serializable {
-
-  def appendTo(sb: Appendable)
-
-  def getId: String
-
-  def getSpec: StateSpec[StateT]
-
-  def bind(binder: StateBinder): StateT
-
->>>>>>> e6ce91c... [Gearpump 311] refactor state management:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTag.scala
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
index 8832aee..f538400 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
@@ -23,16 +23,7 @@ import java.time.Instant
 import org.apache.gearpump.streaming.refactor.coder.Coder
 import org.apache.gearpump.streaming.refactor.state.api.StateInternals
 
-<<<<<<< HEAD:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/RuntimeContext.scala
-/**
- *
- */
 trait RuntimeContext {
-=======
-trait StateSpec[StateT <: State] extends Serializable {
-
-  def bind(id: String, binder: StateBinder): StateT
->>>>>>> e6ce91c... [Gearpump 311] refactor state management:streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpec.scala
 
   def getStateInternals[KT](keyCoder: Coder[KT], key: KT): StateInternals
 

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpec.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpec.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpec.scala
new file mode 100644
index 0000000..91cdbe5
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateSpec.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state
+
+import org.apache.gearpump.streaming.refactor.coder.Coder
+import org.apache.gearpump.streaming.refactor.state.api.State
+
+trait StateSpec[StateT <: State] extends Serializable {
+
+  def bind(id: String, binder: StateBinder): StateT
+
+  def offerCoders(coders: Array[Coder[StateT]]): Unit
+
+  def finishSpecifying: Unit
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTag.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTag.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTag.scala
new file mode 100644
index 0000000..9fa865d
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StateTag.scala
@@ -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.gearpump.streaming.refactor.state
+
+import org.apache.gearpump.streaming.refactor.state.api.State
+
+trait StateTag[StateT <: State] extends Serializable {
+
+  def appendTo(sb: Appendable)
+
+  def getId: String
+
+  def getSpec: StateSpec[StateT]
+
+  def bind(binder: StateBinder): StateT
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
index 0f94052..531ff66 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/StatefulTask.scala
@@ -31,16 +31,8 @@ import org.apache.gearpump.streaming.refactor.state.heap.HeapStateInternalsFacto
 import org.apache.gearpump.streaming.state.impl.{CheckpointManager, PersistentStateConfig}
 import org.apache.gearpump.streaming.task.{Task, TaskContext, UpdateCheckpointClock}
 import org.apache.gearpump.streaming.transaction.api.CheckpointStoreFactory
-import org.apache.gearpump.util.LogUtil
 import org.apache.gearpump.{Message, TimeStamp}
 
-<<<<<<< HEAD
-=======
-object StatefulTask {
-  val LOG = LogUtil.getLogger(getClass)
-}
-
->>>>>>> e6ce91c... [Gearpump 311] refactor state management
 abstract class StatefulTask(taskContext: TaskContext, conf: UserConfig)
   extends Task(taskContext, conf) {
 
@@ -60,11 +52,7 @@ abstract class StatefulTask(taskContext: TaskContext, conf: UserConfig)
   // core state data
   var encodedKeyStateMap: Map[String, Table[String, String, Array[Byte]]] = null
 
-<<<<<<< HEAD
   def open(runtimeContext: RuntimeContext): Unit = {}
-=======
-  def open: Unit = {}
->>>>>>> e6ce91c... [Gearpump 311] refactor state management
 
   def invoke(message: Message): Unit
 

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/db8abf99/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/State.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/State.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/State.scala
new file mode 100644
index 0000000..5c01977
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/state/api/State.scala
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.state.api
+
+trait State {
+
+  def clear: Unit
+
+}


[3/7] incubator-gearpump git commit: [GEARPUMP-311] refactor state management

Posted by ma...@apache.org.
[GEARPUMP-311] refactor state management


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/b9f10866
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/b9f10866
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/b9f10866

Branch: refs/heads/state
Commit: b9f108667a035a791ea016853cefc952922927ab
Parents: fe41030
Author: vinoyang <ya...@gmail.com>
Authored: Sat Jul 22 18:21:23 2017 +0800
Committer: vinoyang <ya...@gmail.com>
Committed: Sat Jul 22 18:21:23 2017 +0800

----------------------------------------------------------------------
 .../refactor/dsl/javaapi/JavaStream.scala       |  98 +++++
 .../streaming/refactor/dsl/plan/OP.scala        | 379 +++++++++++++++++++
 .../refactor/dsl/scalaapi/Stream.scala          | 307 +++++++++++++++
 3 files changed, 784 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/b9f10866/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStream.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStream.scala
new file mode 100644
index 0000000..0cb1185
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/javaapi/JavaStream.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gearpump.streaming.refactor.dsl.javaapi
+
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.dsl.api.functions.{FilterFunction, FoldFunction, MapFunction, ReduceFunction}
+import org.apache.gearpump.streaming.dsl.javaapi.functions.{GroupByFunction, FlatMapFunction => JFlatMapFunction}
+import org.apache.gearpump.streaming.refactor.dsl.javaapi.functions.{FlatMapWithStateFunction => JFlatMapWithStateFunction}
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.Stream
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
+import org.apache.gearpump.streaming.dsl.window.api.Windows
+import org.apache.gearpump.streaming.refactor.dsl.api.functions.MapWithStateFunction
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.functions.FlatMapWithStateFunction
+import org.apache.gearpump.streaming.task.Task
+
+/**
+ * Java DSL
+ */
+class JavaStream[T](val stream: Stream[T]) {
+
+  /** FlatMap on stream */
+  def flatMap[R](fn: JFlatMapFunction[T, R], description: String): JavaStream[R] = {
+    new JavaStream[R](stream.flatMap(FlatMapFunction(fn), "flatMap"))
+  }
+
+  def flatMapWithState[R](fn: JFlatMapWithStateFunction[T, R],
+      description: String): JavaStream[R] = {
+    new JavaStream[R](stream.flatMapWithState(FlatMapWithStateFunction(fn), "flatMapWithState"))
+  }
+
+  /** Map on stream */
+  def map[R](fn: MapFunction[T, R], description: String): JavaStream[R] = {
+    new JavaStream[R](stream.flatMap(FlatMapFunction(fn), description))
+  }
+
+  def mapWithState[R](fn: MapWithStateFunction[T, R], description: String): JavaStream[R] = {
+    new JavaStream[R](stream.flatMapWithState(FlatMapWithStateFunction(fn), description))
+  }
+
+  /** Only keep the messages that FilterFunction returns true.  */
+  def filter(fn: FilterFunction[T], description: String): JavaStream[T] = {
+    new JavaStream[T](stream.flatMap(FlatMapFunction(fn), description))
+  }
+
+  def fold[A](fn: FoldFunction[T, A], description: String): JavaStream[A] = {
+    new JavaStream[A](stream.fold(fn, description))
+  }
+
+  /** Does aggregation on the stream */
+  def reduce(fn: ReduceFunction[T], description: String): JavaStream[T] = {
+    new JavaStream[T](stream.reduce(fn, description))
+  }
+
+  def log(): Unit = {
+    stream.log()
+  }
+
+  /** Merges streams of same type together */
+  def merge(other: JavaStream[T], parallelism: Int, description: String): JavaStream[T] = {
+    new JavaStream[T](stream.merge(other.stream, parallelism, description))
+  }
+
+  /**
+   * Group by a stream and turns it to a list of sub-streams. Operations chained after
+   * groupBy applies to sub-streams.
+   */
+  def groupBy[GROUP](fn: GroupByFunction[T, GROUP],
+      parallelism: Int, description: String): JavaStream[T] = {
+    new JavaStream[T](stream.groupBy(fn.groupBy, parallelism, description))
+  }
+
+  def window(win: Windows): JavaStream[T] = {
+    new JavaStream[T](stream.window(win))
+  }
+
+  /** Add a low level Processor to process messages */
+  def process[R](
+      processor: Class[_ <: Task], parallelism: Int, conf: UserConfig, description: String)
+    : JavaStream[R] = {
+    new JavaStream[R](stream.process(processor, parallelism, conf, description))
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/b9f10866/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/OP.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/OP.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/OP.scala
new file mode 100644
index 0000000..9744ec6
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/plan/OP.scala
@@ -0,0 +1,379 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.dsl.plan.functions
+
+import akka.actor.ActorSystem
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.Processor.DefaultProcessor
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{AndThen => WindowRunnerAT}
+import org.apache.gearpump.streaming.{Constants, Processor}
+import org.apache.gearpump.streaming.refactor.dsl.task.{GroupByTask, TransformTask}
+import org.apache.gearpump.streaming.dsl.window.api.{GlobalWindows, Windows}
+import org.apache.gearpump.streaming.refactor.dsl.window.impl.{DefaultWindowRunner, WindowRunner}
+import org.apache.gearpump.streaming.refactor.source.DataSourceTask
+import org.apache.gearpump.streaming.sink.{DataSink, DataSinkProcessor}
+import org.apache.gearpump.streaming.source.DataSource
+import org.apache.gearpump.streaming.task.Task
+
+import scala.reflect.ClassTag
+
+object Op {
+
+  def concatenate(desc1: String, desc2: String): String = {
+    if (desc1 == null || desc1.isEmpty) desc2
+    else if (desc2 == null || desc2.isEmpty) desc1
+    else desc1 + "." + desc2
+  }
+
+  def concatenate(config1: UserConfig, config2: UserConfig): UserConfig = {
+    config1.withConfig(config2)
+  }
+
+  def withGlobalWindowsDummyRunner(op: Op, userConfig: UserConfig,
+      processor: Processor[_ <: Task])(implicit system: ActorSystem): Processor[_ <: Task] = {
+    if (userConfig.getValue(Constants.GEARPUMP_STREAMING_OPERATOR).isEmpty) {
+      op.chain(
+        WindowOp(GlobalWindows()).chain(TransformOp(new DummyRunner[Any]))
+      ).toProcessor
+    } else {
+      processor
+    }
+  }
+
+}
+
+/**
+ * This is a vertex on the logical plan.
+ */
+sealed trait Op {
+
+  def description: String
+
+  def userConfig: UserConfig
+
+  def chain(op: Op)(implicit system: ActorSystem): Op
+
+  def toProcessor(implicit system: ActorSystem): Processor[_ <: Task]
+}
+
+/**
+ * This represents a low level Processor.
+ */
+case class ProcessorOp[T <: Task](
+    processor: Class[T],
+    parallelism: Int,
+    userConfig: UserConfig,
+    description: String)
+  extends Op {
+
+  def this(
+      parallelism: Int = 1,
+      userConfig: UserConfig = UserConfig.empty,
+      description: String = "processor")(implicit classTag: ClassTag[T]) = {
+    this(classTag.runtimeClass.asInstanceOf[Class[T]], parallelism, userConfig, description)
+  }
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    throw new OpChainException(this, other)
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    DefaultProcessor(parallelism, description, userConfig, processor)
+  }
+}
+
+/**
+ * This represents a DataSource.
+ */
+case class DataSourceOp(
+    dataSource: DataSource,
+    parallelism: Int = 1,
+    description: String = "source",
+    userConfig: UserConfig = UserConfig.empty)
+  extends Op {
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    other match {
+      case op: WindowTransformOp[_, _] =>
+        DataSourceOp(
+          dataSource,
+          parallelism,
+          Op.concatenate(description, op.description),
+          Op.concatenate(userConfig.withValue(Constants.GEARPUMP_STREAMING_OPERATOR,
+            op.windowRunner),
+            op.userConfig))
+      case op: TransformOp[_, _] =>
+        chain(
+          WindowOp(GlobalWindows()).chain(op))
+      case op: WindowOp =>
+        chain(
+          op.chain(TransformOp(new DummyRunner[Any]())))
+      case op: TransformWindowTransformOp[_, _, _] =>
+        chain(
+          WindowOp(GlobalWindows()).chain(op.transformOp)
+            .chain(op.windowTransformOp))
+      case _ =>
+        throw new OpChainException(this, other)
+    }
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    Op.withGlobalWindowsDummyRunner(this, userConfig,
+      Processor[DataSourceTask[Any, Any]](parallelism, description,
+        userConfig.withValue(Constants.GEARPUMP_STREAMING_SOURCE, dataSource))
+    )
+  }
+}
+
+/**
+ * This represents a DataSink.
+ */
+case class DataSinkOp(
+    dataSink: DataSink,
+    parallelism: Int = 1,
+    description: String = "sink",
+    userConfig: UserConfig = UserConfig.empty)
+  extends Op {
+
+  override def chain(op: Op)(implicit system: ActorSystem): Op = {
+    throw new OpChainException(this, op)
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    DataSinkProcessor(dataSink, parallelism, description)
+  }
+}
+
+/**
+ * This represents operations that can be chained together
+ * (e.g. flatMap, map, filter, reduce) and further chained
+ * to another Op to be used
+ */
+case class TransformOp[IN, OUT](
+    fn: FunctionRunner[IN, OUT],
+    userConfig: UserConfig = UserConfig.empty) extends Op {
+
+  override def description: String = fn.description
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    other match {
+      case op: TransformOp[OUT, _] =>
+        // TODO: preserve type info
+        // f3(f2(f1(in)))
+        // => ChainableOp(f1).chain(ChainableOp(f2)).chain(ChainableOp(f3))
+        // => AndThen(AndThen(f1, f2), f3)
+        TransformOp(
+          AndThen(fn, op.fn),
+          Op.concatenate(userConfig, op.userConfig))
+      case op: WindowOp =>
+        TransformWindowTransformOp(this,
+          WindowTransformOp(new DefaultWindowRunner[OUT, OUT](
+            op.windows, new DummyRunner[OUT]
+          ), op.description, op.userConfig))
+      case op: TransformWindowTransformOp[OUT, _, _] =>
+        TransformWindowTransformOp(TransformOp(
+          AndThen(fn, op.transformOp.fn),
+          Op.concatenate(userConfig, op.transformOp.userConfig)
+        ), op.windowTransformOp)
+      case _ =>
+        throw new OpChainException(this, other)
+    }
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    WindowOp(GlobalWindows()).chain(this).toProcessor
+  }
+}
+
+/**
+ * This is an intermediate operation, produced by chaining WindowOp and TransformOp.
+ * Usually, it will be chained to a DataSourceOp, GroupByOp or MergeOp.
+ * Otherwise, it will be translated to a Processor of TransformTask.
+ */
+case class WindowTransformOp[IN, OUT](
+    windowRunner: WindowRunner[IN, OUT],
+    description: String,
+    userConfig: UserConfig) extends Op {
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    other match {
+      case op: WindowTransformOp[OUT, _] =>
+        WindowTransformOp(
+          WindowRunnerAT(windowRunner, op.windowRunner),
+          Op.concatenate(description, op.description),
+          Op.concatenate(userConfig, op.userConfig)
+        )
+      case _ =>
+        throw new OpChainException(this, other)
+    }
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    // TODO: this should be chained to DataSourceOp / GroupByOp / MergeOp
+    Processor[TransformTask[Any, Any]](1, description, userConfig.withValue(
+      Constants.GEARPUMP_STREAMING_OPERATOR, windowRunner))
+  }
+}
+
+/**
+ * This is an intermediate operation, produced by chaining TransformOp and WindowOp.
+ * It will later be chained to a WindowOp, which results in two WindowTransformOps.
+ * Finally, they will be chained to a single WindowTransformOp.
+ */
+case class TransformWindowTransformOp[IN, MIDDLE, OUT](
+    transformOp: TransformOp[IN, MIDDLE],
+    windowTransformOp: WindowTransformOp[MIDDLE, OUT]) extends Op {
+
+  override def description: String = {
+    throw new UnsupportedOperationException(s"description is not supported on $this")
+  }
+
+  override def userConfig: UserConfig = {
+    throw new UnsupportedOperationException(s"userConfig is not supported on $this")
+  }
+
+  override def chain(op: Op)(implicit system: ActorSystem): Op = {
+    throw new UnsupportedOperationException(s"chain is not supported on $this")
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    WindowOp(GlobalWindows()).chain(this).toProcessor
+  }
+}
+
+/**
+ * This represents a window aggregation, together with a following TransformOp
+ */
+case class WindowOp(
+    windows: Windows,
+    userConfig: UserConfig = UserConfig.empty) extends Op {
+
+  override def description: String = windows.description
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    other match {
+      case op: TransformOp[_, _] =>
+        WindowTransformOp(new DefaultWindowRunner(windows, op.fn),
+          Op.concatenate(description, op.description),
+          Op.concatenate(userConfig, op.userConfig))
+      case op: WindowOp =>
+        chain(TransformOp(new DummyRunner[Any])).chain(op.chain(TransformOp(new DummyRunner[Any])))
+      case op: TransformWindowTransformOp[_, _, _] =>
+        WindowTransformOp(new DefaultWindowRunner(windows, op.transformOp.fn),
+          Op.concatenate(description, op.transformOp.description),
+          Op.concatenate(userConfig, op.transformOp.userConfig)).chain(op.windowTransformOp)
+      case _ =>
+        throw new OpChainException(this, other)
+    }
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    chain(TransformOp(new DummyRunner[Any])).toProcessor
+  }
+
+}
+
+/**
+ * This represents a Processor with groupBy and window aggregation
+ */
+case class GroupByOp[IN, GROUP] private(
+    groupBy: IN => GROUP,
+    parallelism: Int = 1,
+    description: String = "groupBy",
+    override val userConfig: UserConfig = UserConfig.empty)
+  extends Op {
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    other match {
+      case op: WindowTransformOp[_, _] =>
+        GroupByOp(
+          groupBy,
+          parallelism,
+          Op.concatenate(description, op.description),
+          Op.concatenate(
+            userConfig
+              .withValue(Constants.GEARPUMP_STREAMING_OPERATOR, op.windowRunner),
+            userConfig))
+      case op: WindowOp =>
+        chain(op.chain(TransformOp(new DummyRunner[Any]())))
+      case _ =>
+        throw new OpChainException(this, other)
+    }
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    Op.withGlobalWindowsDummyRunner(this, userConfig,
+      Processor[GroupByTask[IN, GROUP, Any]](parallelism, description,
+        userConfig.withValue(Constants.GEARPUMP_STREAMING_GROUPBY_FUNCTION, groupBy)))
+  }
+}
+
+/**
+ * This represents a Processor transforming merged streams
+ */
+case class MergeOp(
+    parallelism: Int = 1,
+    description: String = "merge",
+    userConfig: UserConfig = UserConfig.empty)
+  extends Op {
+
+  override def chain(other: Op)(implicit system: ActorSystem): Op = {
+    other match {
+      case op: WindowTransformOp[_, _] =>
+        MergeOp(
+          parallelism,
+          description,
+          Op.concatenate(userConfig.withValue(Constants.GEARPUMP_STREAMING_OPERATOR,
+            op.windowRunner),
+            op.userConfig))
+      case op: WindowOp =>
+        chain(op.chain(TransformOp(new DummyRunner[Any]())))
+      case _ =>
+        throw new OpChainException(this, other)
+    }
+  }
+
+  override def toProcessor(implicit system: ActorSystem): Processor[_ <: Task] = {
+    Op.withGlobalWindowsDummyRunner(this, userConfig,
+      Processor[TransformTask[Any, Any]](parallelism, description, userConfig))
+  }
+
+}
+
+/**
+ * This is an edge on the logical plan.
+ */
+trait OpEdge
+
+/**
+ * The upstream OP and downstream OP doesn't require network data shuffle.
+ * e.g. TransformOp
+ */
+case object Direct extends OpEdge
+
+/**
+ * The upstream OP and downstream OP DOES require network data shuffle.
+ * e.g. GroupByOp
+ */
+case object Shuffle extends OpEdge
+
+/**
+ * Runtime exception thrown on chaining.
+ */
+class OpChainException(op1: Op, op2: Op) extends RuntimeException(s"$op1 can't be chained by $op2")

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/b9f10866/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/Stream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/Stream.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/Stream.scala
new file mode 100644
index 0000000..c0499c9
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/refactor/dsl/scalaapi/Stream.scala
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gearpump.streaming.refactor.dsl.scalaapi
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.dsl.api.functions.{FilterFunction, FoldFunction, MapFunction, ReduceFunction}
+import org.apache.gearpump.streaming.refactor.dsl.plan.functions._
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
+import org.apache.gearpump.streaming.dsl.window.api._
+import org.apache.gearpump.streaming.refactor.dsl.api.functions.MapWithStateFunction
+import org.apache.gearpump.streaming.refactor.dsl.scalaapi.functions.FlatMapWithStateFunction
+import org.apache.gearpump.streaming.sink.DataSink
+import org.apache.gearpump.streaming.task.{Task, TaskContext}
+import org.apache.gearpump.util.Graph
+import org.slf4j.{Logger, LoggerFactory}
+
+import scala.language.implicitConversions
+
+class Stream[T](
+    private val graph: Graph[Op, OpEdge], private val thisNode: Op,
+    private val edge: Option[OpEdge] = None,
+    private val windows: Windows = GlobalWindows()) {
+
+  /**
+   * Returns a new stream by applying a flatMap function to each element
+   * and flatten the results.
+   *
+   * @param fn flatMap function
+   * @param description The description message for this operation
+   * @return A new stream with type [R]
+   */
+  def flatMap[R](fn: T => TraversableOnce[R], description: String = "flatMap"): Stream[R] = {
+    this.flatMap(FlatMapFunction(fn), description)
+  }
+
+  /**
+   * Returns a new stream by applying a flatMap function to each element
+   * and flatten the results.
+   *
+   * @param fn flatMap function
+   * @param description The description message for this operation
+   * @return A new stream with type [R]
+   */
+  def flatMap[R](fn: FlatMapFunction[T, R], description: String): Stream[R] = {
+    transform(new FlatMapper[T, R](fn, description))
+  }
+
+  def flatMapWithState[R](fn: FlatMapWithStateFunction[T, R], description: String): Stream[R] = {
+    transform(new FlatMapper[T, R](fn, description))
+  }
+
+  /**
+   * Returns a new stream by applying a map function to each element.
+   *
+   * @param fn map function
+   * @return A new stream with type [R]
+   */
+  def map[R](fn: T => R, description: String = "map"): Stream[R] = {
+    this.map(MapFunction(fn), description)
+  }
+
+  /**
+   * Returns a new stream by applying a map function to each element.
+   *
+   * @param fn map function
+   * @return A new stream with type [R]
+   */
+  def map[R](fn: MapFunction[T, R], description: String): Stream[R] = {
+    this.flatMap(FlatMapFunction(fn), description)
+  }
+
+  def mapWithState[R](fn: MapWithStateFunction[T, R], description: String): Stream[R] = {
+    this.flatMapWithState(FlatMapWithStateFunction(fn), description);
+  }
+
+  /**
+   * Returns a new Stream keeping the elements that satisfy the filter function.
+   *
+   * @param fn filter function
+   * @return a new stream after filter
+   */
+  def filter(fn: T => Boolean, description: String = "filter"): Stream[T] = {
+    this.filter(FilterFunction(fn), description)
+  }
+
+  /**
+   * Returns a new Stream keeping the elements that satisfy the filter function.
+   *
+   * @param fn filter function
+   * @return a new stream after filter
+   */
+  def filter(fn: FilterFunction[T], description: String): Stream[T] = {
+    this.flatMap(FlatMapFunction(fn), description)
+  }
+
+  /**
+   * Returns a new stream by applying a fold function over all the elements
+   *
+   * @param fn fold function
+   * @return a new stream after fold
+   */
+  def fold[A](fn: FoldFunction[T, A], description: String): Stream[A] = {
+    transform(new FoldRunner(fn, description))
+  }
+
+  /**
+   * Returns a new stream by applying a reduce function over all the elements.
+   *
+   * @param fn reduce function
+   * @param description description message for this operator
+   * @return a new stream after reduce
+   */
+  def reduce(fn: (T, T) => T, description: String = "reduce"): Stream[T] = {
+    reduce(ReduceFunction(fn), description)
+  }
+
+  /**
+   * Returns a new stream by applying a reduce function over all the elements.
+   *
+   * @param fn reduce function
+   * @param description description message for this operator
+   * @return a new stream after reduce
+   */
+  def reduce(fn: ReduceFunction[T], description: String): Stream[T] = {
+    fold(fn, description).map(_.get)
+  }
+
+  private def transform[R](fn: FunctionRunner[T, R]): Stream[R] = {
+    val op = TransformOp(fn)
+    graph.addVertex(op)
+    graph.addEdge(thisNode, edge.getOrElse(Direct), op)
+    new Stream(graph, op, None, windows)
+  }
+
+  /**
+   * Log to task log file
+   */
+  def log(): Unit = {
+    this.map(msg => {
+      LoggerFactory.getLogger("dsl").info(msg.toString)
+      msg
+    }, "log")
+  }
+
+  /**
+   * Merges data from two stream into one
+   *
+   * @param other the other stream
+   * @return  the merged stream
+   */
+  def merge(other: Stream[T], parallelism: Int = 1, description: String = "merge"): Stream[T] = {
+    val mergeOp = MergeOp(parallelism, description, UserConfig.empty)
+    graph.addVertex(mergeOp)
+    graph.addEdge(thisNode, edge.getOrElse(Direct), mergeOp)
+    graph.addEdge(other.thisNode, other.edge.getOrElse(Shuffle), mergeOp)
+    val winOp = Stream.addWindowOp(graph, mergeOp, windows)
+    new Stream[T](graph, winOp, None, windows)
+  }
+
+  /**
+   * Group by function (T => Group)
+   *
+   * For example, we have T type, People(name: String, gender: String, age: Int)
+   * groupBy[People](_.gender) will group the people by gender.
+   *
+   * You can append other combinators after groupBy
+   *
+   * For example,
+   * {{{
+   * Stream[People].groupBy(_.gender).flatMap(..).filter(..).reduce(..)
+   * }}}
+   *
+   * @param fn  Group by function
+   * @param parallelism  Parallelism level
+   * @param description  The description
+   * @return the grouped stream
+   */
+  def groupBy[GROUP](fn: T => GROUP, parallelism: Int = 1,
+      description: String = "groupBy"): Stream[T] = {
+    val gbOp = GroupByOp(fn, parallelism, description)
+    graph.addVertex(gbOp)
+    graph.addEdge(thisNode, edge.getOrElse(Shuffle), gbOp)
+    val winOp = Stream.addWindowOp(graph, gbOp, windows)
+    new Stream(graph, winOp, None, windows)
+  }
+
+  /**
+   * Window function
+   *
+   * @param windows window definition
+   * @return the windowed [[Stream]]
+   */
+  def window(windows: Windows): Stream[T] = {
+    val winOp = Stream.addWindowOp(graph, thisNode, windows)
+    new Stream(graph, winOp, None, windows)
+  }
+
+  /**
+   * Connects with a low level Processor(TaskDescription)
+   *
+   * @param processor  a user defined processor
+   * @param parallelism  parallelism level
+   * @return  new stream after processing with type [R]
+   */
+  def process[R](
+      processor: Class[_ <: Task], parallelism: Int, conf: UserConfig = UserConfig.empty,
+      description: String = "process"): Stream[R] = {
+    val processorOp = ProcessorOp(processor, parallelism, conf, description)
+    graph.addVertex(processorOp)
+    graph.addEdge(thisNode, edge.getOrElse(Shuffle), processorOp)
+    new Stream[R](graph, processorOp, Some(Shuffle), windows)
+  }
+
+
+}
+
+class KVStream[K, V](stream: Stream[Tuple2[K, V]]) {
+  /**
+   * GroupBy key
+   *
+   * Applies to Stream[Tuple2[K,V]]
+   *
+   * @param parallelism  the parallelism for this operation
+   * @return  the new KV stream
+   */
+  def groupByKey(parallelism: Int = 1): Stream[Tuple2[K, V]] = {
+    stream.groupBy(Stream.getTupleKey[K, V], parallelism, "groupByKey")
+  }
+
+  /**
+   * Sum the value of the tuples
+   *
+   * Apply to Stream[Tuple2[K,V]], V must be of type Number
+   *
+   * For input (key, value1), (key, value2), will generate (key, value1 + value2)
+   * @param numeric  the numeric operations
+   * @return  the sum stream
+   */
+  def sum(implicit numeric: Numeric[V]): Stream[(K, V)] = {
+    stream.reduce(Stream.sumByKey[K, V](numeric), "sum")
+  }
+}
+
+object Stream {
+
+  def apply[T](graph: Graph[Op, OpEdge], node: Op, edge: Option[OpEdge],
+      windows: Windows): Stream[T] = {
+    new Stream[T](graph, node, edge, windows)
+  }
+
+  def getTupleKey[K, V](tuple: Tuple2[K, V]): K = tuple._1
+
+  def sumByKey[K, V](numeric: Numeric[V]): (Tuple2[K, V], Tuple2[K, V]) => Tuple2[K, V]
+  = (tuple1, tuple2) => Tuple2(tuple1._1, numeric.plus(tuple1._2, tuple2._2))
+
+  def addWindowOp(graph: Graph[Op, OpEdge], op: Op, win: Windows): Op = {
+    val winOp = WindowOp(win)
+    graph.addVertex(winOp)
+    graph.addEdge(op, Direct, winOp)
+    winOp
+  }
+
+  implicit def streamToKVStream[K, V](stream: Stream[Tuple2[K, V]]): KVStream[K, V] = {
+    new KVStream(stream)
+  }
+
+  implicit class Sink[T](stream: Stream[T]) extends java.io.Serializable {
+    def sink(dataSink: DataSink, parallelism: Int = 1,
+        conf: UserConfig = UserConfig.empty, description: String = "sink"): Stream[T] = {
+      implicit val sink = DataSinkOp(dataSink, parallelism, description, conf)
+      stream.graph.addVertex(sink)
+      stream.graph.addEdge(stream.thisNode, Shuffle, sink)
+      new Stream[T](stream.graph, sink, None, stream.windows)
+    }
+  }
+}
+
+class LoggerSink[T] extends DataSink {
+  var logger: Logger = _
+
+  override def open(context: TaskContext): Unit = {
+    this.logger = context.logger
+  }
+
+  override def write(message: Message): Unit = {
+    logger.info("logging message " + message.value)
+  }
+
+  override def close(): Unit = Unit
+}
\ No newline at end of file