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