[tinkerpop] branch tp4 updated: lots of cleanup and reorg. Path now implements TSequence (smart, duplicate keys allowed). Thought through and tweaked all the methods of the core structure API/. TTuple

2019-04-30 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new bd79bcc  lots of cleanup and reorg. Path now implements TSequence 
(smart, duplicate keys allowed). Thought through and tweaked all the methods of 
the core structure API/. TTuple and TSequence are making lots of sense -- 
basically, map and list respectively.
bd79bcc is described below

commit bd79bcc4c1d5db3f1bcf3e40e7f1d27892c928e4
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 30 04:05:56 2019 -0600

lots of cleanup and reorg. Path now implements TSequence (smart, duplicate 
keys allowed). Thought through and tweaked all the methods of the core 
structure API/. TTuple and TSequence are making lots of sense -- basically, map 
and list respectively.
---
 .../language/gremlin/AbstractTraversal.java|  2 +-
 .../tinkerpop/language/gremlin/Traversal.java  |  4 +-
 .../language/gremlin/TraversalSource.java  |  6 +--
 .../language/gremlin/core/CoreTraversal.java   |  4 +-
 .../tinkerpop/machine/bytecode/Bytecode.java   | 11 +++--
 .../tinkerpop/machine/bytecode/Instruction.java|  6 +--
 .../machine/function/filter/HasKeyFilter.java  |  8 +---
 .../machine/function/flatmap/DbFlatMap.java| 17 ---
 .../machine/function/flatmap/EntriesFlatMap.java   | 19 
 .../machine/function/reduce/GroupCountReduce.java  |  4 +-
 .../strategy/decoration/ExplainStrategy.java   |  1 +
 .../structure/{util/J2Tuple.java => TPair.java}| 52 ++
 .../apache/tinkerpop/machine/structure/TTuple.java | 13 +++---
 .../tinkerpop/machine/structure/graph/TEdge.java   |  6 ++-
 .../tinkerpop/machine/structure/graph/TVertex.java | 16 +++
 .../{JSequence.java => CompositeSequence.java} | 22 +
 .../structure/util/{T2Tuple.java => JPair.java}| 27 +--
 .../machine/structure/util/JSequence.java  |  5 +++
 .../tinkerpop/machine/structure/util/JTuple.java   | 21 ++---
 .../machine/traverser/path/BasicPath.java  | 28 +++-
 .../machine/traverser/path/EmptyPath.java  | 10 ++---
 .../tinkerpop/machine/traverser/path/Path.java | 21 ++---
 .../tinkerpop/machine/bytecode/BytecodeTest.java   |  4 +-
 .../machine/bytecode/InstructionTest.java  | 10 ++---
 .../machine/bytecode/compiler/CompilationTest.java |  6 +--
 .../blueprints/data/BlueprintsVertex.java  | 18 +++-
 .../provider/BlueprintsVerticesStrategy.java   |  2 +-
 .../machine/structure/jdbc/JDBCDatabase.java   | 17 +++
 .../tinkerpop/machine/structure/jdbc/JDBCRow.java  | 27 +++
 .../jdbc/bytecode/compiler/JDBCCompiler.java   | 15 +++
 .../jdbc/function/flatmap/SqlFlatMap.java  |  5 +++
 .../structure/jdbc/strategy/JDBCQueryStrategy.java |  2 +-
 .../structure/jdbc/strategy/JDBCStrategy.java  |  2 +-
 33 files changed, 191 insertions(+), 220 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
index 51f7e08..793a469 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
@@ -54,7 +54,7 @@ public abstract class AbstractTraversal implements 
Traversal {
 protected final  Traversal addInstruction(final String op, 
final Object... args) {
 if (this.executed)
 throw new IllegalStateException("The traversal has already been 
submitted and can no longer be mutated");
-this.bytecode.addInstruction(this.currentCoefficient, op, args);
+this.bytecode.addInstruction(this.currentCoefficient, null, op, args);
 this.currentCoefficient.unity();
 return (Traversal) this;
 }
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 5015a9e..b974502 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -21,7 +21,7 @@ package org.apache.tinkerpop.language.gremlin;
 import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.apache.tinkerpop.machine.structure.TTuple;
 import org.apache.tinkerpop.machine.structure.rdbms.TDatabase;
-import org.apache.tinkerpop.machine.structure.util.T2Tuple;
+import org.apache.tinkerpop.machine.structure.TPair;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.path.Pat

[tinkerpop] branch tp4 updated: minor nothing. done for the day.

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 0ae83d2  minor nothing. done for the day.
0ae83d2 is described below

commit 0ae83d2e1fb6ece61202b5ffa2e9fbeff47ec9e5
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 17:02:50 2019 -0600

minor nothing. done for the day.
---
 .../org/apache/tinkerpop/machine/function/flatmap/InjectFlatMap.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/flatmap/InjectFlatMap.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/flatmap/InjectFlatMap.java
index 1635b2b..8dbf75b 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/flatmap/InjectFlatMap.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/flatmap/InjectFlatMap.java
@@ -42,7 +42,7 @@ public final class InjectFlatMap extends 
AbstractFunction implements
 }
 
 @Override
-public Iterator apply(Traverser traverser) {
+public Iterator apply(final Traverser traverser) {
 return new ArrayIterator<>(this.objects);
 }
 



[tinkerpop] branch tp4 updated: added hashCode() and toString() to FlatMapInitial.

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 57dfb64  added hashCode() and toString() to FlatMapInitial.
57dfb64 is described below

commit 57dfb646ffbc466f964aaf04bcab9f42b1350c61
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 16:41:52 2019 -0600

added hashCode() and toString() to FlatMapInitial.
---
 .../tinkerpop/machine/function/initial/FlatMapInitial.java| 11 +++
 1 file changed, 11 insertions(+)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
index 2a2ca21..66afefb 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
@@ -22,6 +22,7 @@ import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.AbstractFunction;
 import org.apache.tinkerpop.machine.function.InitialFunction;
 import org.apache.tinkerpop.machine.traverser.species.EmptyTraverser;
+import org.apache.tinkerpop.machine.util.StringFactory;
 
 import java.util.Iterator;
 
@@ -41,4 +42,14 @@ public final class FlatMapInitial extends 
AbstractFunction implements I
 public Iterator get() {
 return this.function.apply(EmptyTraverser.instance());
 }
+
+@Override
+public int hashCode() {
+return super.hashCode() ^ this.function.hashCode();
+}
+
+@Override
+public String toString() {
+return StringFactory.makeFunctionString(this, this.function);
+}
 }



[tinkerpop] branch tp4 updated: added rdf/. Super easy. FlatMapInitial is used to wrap FlatMap functions that can serve as initials so we don't have to duplicate code.

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4ac1cb8  added rdf/. Super easy. FlatMapInitial is used to wrap 
FlatMap functions that can serve as initials so we don't have to duplicate code.
4ac1cb8 is described below

commit 4ac1cb896126f54eeb334667d4bfbd0ff3d25b62
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 16:31:43 2019 -0600

added rdf/. Super easy. FlatMapInitial is used to wrap FlatMap functions 
that can serve as initials so we don't have to duplicate code.
---
 .../bytecode/compiler/BytecodeCompiler.java|  6 ++---
 .../machine/function/initial/FlatMapInitial.java   |  9 +++
 .../tinkerpop/machine/structure/rdf/TQuad.java | 27 
 .../machine/structure/rdf/TStatement.java  | 28 +
 .../tinkerpop/machine/structure/rdf/TStore.java| 28 +
 .../tinkerpop/machine/structure/rdf/TTriple.java   | 29 ++
 6 files changed, 118 insertions(+), 9 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
index 07b49ed..eca364f 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
@@ -19,7 +19,6 @@
 package org.apache.tinkerpop.machine.bytecode.compiler;
 
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
-import org.apache.tinkerpop.machine.bytecode.BytecodeUtil;
 import org.apache.tinkerpop.machine.bytecode.Instruction;
 import org.apache.tinkerpop.machine.function.CFunction;
 import org.apache.tinkerpop.machine.function.initial.FlatMapInitial;
@@ -37,8 +36,9 @@ public interface BytecodeCompiler {
 final List> functions = new ArrayList<>();
 for (final Instruction instruction : bytecode.getInstructions()) {
 final CFunction function = this.compile(instruction);
-functions.add(functions.isEmpty() && 
bytecode.getParent().isEmpty() && function instanceof Initializing ?
-new FlatMapInitial<>(function.coefficient(), 
function.label(), (Initializing) function, 
BytecodeUtil.getTraverserFactory(bytecode).get()) : function);
+functions.add(functions.isEmpty() && 
bytecode.getParent().isEmpty() && function instanceof Initializing ? // if this 
is the first function, then wrap it in an InitialFunction
+new FlatMapInitial<>(function.coefficient(), 
function.label(), (Initializing) function) :
+function);
 }
 return functions;
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
index 8b06567..2a2ca21 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/initial/FlatMapInitial.java
@@ -21,7 +21,7 @@ package org.apache.tinkerpop.machine.function.initial;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.AbstractFunction;
 import org.apache.tinkerpop.machine.function.InitialFunction;
-import org.apache.tinkerpop.machine.traverser.TraverserFactory;
+import org.apache.tinkerpop.machine.traverser.species.EmptyTraverser;
 
 import java.util.Iterator;
 
@@ -31,17 +31,14 @@ import java.util.Iterator;
 public final class FlatMapInitial extends AbstractFunction implements 
InitialFunction {
 
 private final Initializing function;
-private final TraverserFactory traverserFactory;
 
-
-public FlatMapInitial(final Coefficient coefficient, final String 
label, final Initializing function, final TraverserFactory 
traverserFactory) {
+public FlatMapInitial(final Coefficient coefficient, final String 
label, final Initializing function) {
 super(coefficient, label);
 this.function = function;
-this.traverserFactory = traverserFactory;
 }
 
 @Override
 public Iterator get() {
-return this.function.apply(traverserFactory.create(this, null));
+return this.function.apply(EmptyTraverser.instance());
 }
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/structure/rdf/TQuad.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/structure/rdf/TQuad.java
new file mode 10

[tinkerpop] branch tp4 updated: dah. commited with System.out.printlns(). I'm pooped. Need to rest.

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 36770ac  dah. commited with System.out.printlns(). I'm pooped. Need to 
rest.
36770ac is described below

commit 36770acdaf1d8becd67e75f8fb5528043d16345c
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 14:13:10 2019 -0600

dah. commited with System.out.printlns(). I'm pooped. Need to rest.
---
 .../java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java | 1 -
 .../tinkerpop/machine/structure/jdbc/strategy/JDBCQueryStrategy.java| 2 --
 2 files changed, 3 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
index 5b46c9e..5b7f078 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
@@ -79,7 +79,6 @@ public final class BasicPath implements Path {
 
 @Override
 public Object get(final Pop pop, final String label) {
-   System.out.println(this.labels + ":::" + this.objects);
 if (Pop.last == pop) {
 for (int i = this.labels.size() - 1; i >= 0; i--) {
 if (label.equals(this.labels.get(i)))
diff --git 
a/java/machine/structure/jdbc/src/main/java/org/apache/tinkerpop/machine/structure/jdbc/strategy/JDBCQueryStrategy.java
 
b/java/machine/structure/jdbc/src/main/java/org/apache/tinkerpop/machine/structure/jdbc/strategy/JDBCQueryStrategy.java
index 3adb915..203bcdb 100644
--- 
a/java/machine/structure/jdbc/src/main/java/org/apache/tinkerpop/machine/structure/jdbc/strategy/JDBCQueryStrategy.java
+++ 
b/java/machine/structure/jdbc/src/main/java/org/apache/tinkerpop/machine/structure/jdbc/strategy/JDBCQueryStrategy.java
@@ -34,7 +34,6 @@ public final class JDBCQueryStrategy extends 
AbstractStrategy void apply(final Bytecode bytecode) {
 if (bytecode.getParent().isEmpty() && 
BytecodeUtil.startsWith(bytecode, Symbols.DB, Symbols.VALUES, Symbols.DB, 
Symbols.VALUES, Symbols.HAS_KEY_VALUE, Symbols.PATH)) {
-System.out.println(bytecode);
 final JDBCDatabase db = (JDBCDatabase) 
bytecode.getInstructions().get(0).args()[0];
 bytecode.getInstructions().remove(0); // DB
 final String table1 = (String) 
bytecode.getInstructions().get(0).args()[0];
@@ -49,7 +48,6 @@ public final class JDBCQueryStrategy extends 
AbstractStrategy inst = bytecode.getInstructions().remove(0); 
// HAS_KEY_VALUE
 bytecode.getInstructions().remove(0); // PATH
 bytecode.addInstruction(0, inst.coefficient(), "jdbc:sql", 
db.getConnection(), as1, as2, query);
-System.out.println(bytecode);
 }
 }
 }



[tinkerpop] branch tp4 updated: Created a JDBCQueryStrategy that looks for 'join' semantics in the bytecode and folds it into a custom jdbs:sql instruction that does a SELECT call.

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 7142dc1  Created a JDBCQueryStrategy that looks for 'join' semantics 
in the bytecode and folds it into a custom jdbs:sql instruction that does a 
SELECT call.
7142dc1 is described below

commit 7142dc16d8fc81ad8bd4090096b42e5b9b1744f4
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 13:56:21 2019 -0600

Created a JDBCQueryStrategy that looks for 'join' semantics in the bytecode 
and folds it into a custom jdbs:sql instruction that does a SELECT call.
---
 .../tinkerpop/machine/bytecode/Bytecode.java   |   5 +
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  12 +++
 .../bytecode/compiler/BytecodeCompiler.java|   3 +-
 .../machine/bytecode/compiler/Compilation.java |   4 +-
 .../bytecode/compiler/SourceCompilation.java   |   4 +-
 .../machine/function/barrier/OrderBarrier.java |   2 +-
 .../machine/function/branch/RepeatBranch.java  |   4 +-
 .../machine/function/initial/FlatMapInitial.java   |   9 +-
 .../processor/{ => util}/EmptyProcessor.java   |   5 +-
 .../processor/{ => util}/FilterProcessor.java  |   2 +-
 .../processor/{ => util}/LoopsProcessor.java   |   2 +-
 .../processor/{ => util}/SimpleProcessor.java  |   4 +-
 .../tinkerpop/machine/structure/TSymbol.java   |  26 -
 .../machine/structure/graph/IdSymbol.java  |  52 --
 .../machine/structure/graph/LabelSymbol.java   |  52 --
 .../tinkerpop/machine/structure/graph/TGraph.java  |   4 +-
 .../machine/structure/graph/TVertices.java |  27 --
 .../structure/{ => util}/EmptyStructure.java   |   5 +-
 .../tinkerpop/machine/structure/util/JTuple.java   |   5 +
 .../structure/{graph/G.java => util/TSymbol.java}  |  22 +++--
 .../machine/traverser/path/BasicPath.java  |   1 +
 .../machine/structure/jdbc/JDBCDatabase.java   |   6 +-
 .../machine/structure/jdbc/JDBCStructure.java  |  11 ++-
 .../jdbc/bytecode/compiler/JDBCCompiler.java   |  69 ++
 .../jdbc/function/flatmap/SqlFlatMap.java  | 105 +
 .../structure/jdbc/strategy/JDBCQueryStrategy.java |  55 +++
 .../structure/jdbc/strategy/JDBCStrategy.java  |   7 ++
 .../tinkerpop/machine/structure/jdbc/JDBCTest.java |   8 +-
 28 files changed, 324 insertions(+), 187 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
index acc..7519652 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
@@ -64,6 +64,11 @@ public final class Bytecode implements Cloneable, 
Serializable {
 this.instructions.add(new Instruction<>(coefficient, op, args));
 }
 
+public void addInstruction(final int index, final Coefficient 
coefficient, final String op, final Object... args) {
+BytecodeUtil.linkBytecodeChildren(this, args);
+this.instructions.add(index, new Instruction<>(coefficient, op, args));
+}
+
 public List> getInstructions() {
 return this.instructions;
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index c4ea488..afb3072 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -187,4 +187,16 @@ public final class BytecodeUtil {
 }
 return Optional.of(COP_TraverserFactory.instance());
 }
+
+public static  boolean startsWith(final Bytecode bytecode, String... 
ops) {
+int i = 0;
+for (final Instruction instruction : bytecode.getInstructions()) {
+if (!instruction.op().equals(ops[i]))
+return false;
+i++;
+if (i >= ops.length)
+return true;
+}
+return false;
+}
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
index 16cf912..07b49ed 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeCompiler.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeComp

[tinkerpop] branch tp4 updated: There are no more InitialFunctions. If a FlatMapFunction implements Initializing then that means it can generate an output that isn't dependent on input. This way, we c

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 1494567  There are no more InitialFunctions. If a FlatMapFunction 
implements Initializing then that means it can generate an output that isn't 
dependent on input. This way, we can have V() and mid-traversal V() without 
having two steps. Likewise for db() and inject(). Created PathObjectMap which 
is for path('x').by()... it will emit a single object, not a Path. Lots of 
other cleanup and organization.
1494567 is described below

commit 1494567d9371d885ac9af7e435efff9621a87ca6
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 11:04:37 2019 -0600

There are no more InitialFunctions. If a FlatMapFunction implements 
Initializing then that means it can generate an output that isn't dependent on 
input. This way, we can have V() and mid-traversal V() without having two 
steps. Likewise for db() and inject(). Created PathObjectMap which is for 
path('x').by()... it will emit a single object, not a Path. Lots of other 
cleanup and organization.
---
 .../language/gremlin/TraversalSource.java  |  2 +-
 .../tinkerpop/language/gremlin/TraversalUtil.java  |  9 ++-
 .../language/gremlin/core/CoreTraversal.java   | 11 +--
 .../apache/tinkerpop/language/gremlin/core/__.java |  6 +-
 .../machine/bytecode/compiler/Argument.java|  3 -
 .../bytecode/compiler/BytecodeCompiler.java|  6 +-
 .../machine/bytecode/compiler/CoreCompiler.java| 16 ++--
 .../machine/bytecode/compiler/FunctionType.java|  2 +-
 .../machine/bytecode/compiler/MethodArgument.java  | 93 --
 .../DbInitial.java => flatmap/DbFlatMap.java}  | 26 +++---
 .../InjectFlatMap.java}| 24 --
 .../machine/function/initial/FlatMapInitial.java}  | 18 ++---
 .../initial/Initializing.java} |  9 +--
 .../tinkerpop/machine/function/map/PathMap.java| 32 +---
 .../machine/function/map/PathObjectMap.java| 65 +++
 .../strategy/decoration/ExplainStrategy.java   |  2 +-
 .../apache/tinkerpop/machine/structure/TTuple.java |  2 +
 .../tinkerpop/machine/structure/util/J2Tuple.java  |  5 ++
 .../tinkerpop/machine/structure/util/JTuple.java   |  5 ++
 .../machine/traverser/path/BasicPath.java  |  6 --
 .../machine/traverser/path/EmptyPath.java  |  5 --
 .../tinkerpop/machine/traverser/path/Path.java |  7 +-
 .../tinkerpop/machine/processor/pipes/Pipes.java   |  6 +-
 .../bytecode/compiler/BlueprintsCompiler.java  |  4 +-
 .../blueprints/data/BlueprintsVertex.java  |  5 ++
 .../{initial => flatmap}/VerticesFlatMap.java  | 16 ++--
 .../provider/BlueprintsVerticesStrategy.java   |  2 +-
 .../machine/structure/jdbc/JDBCDatabase.java   |  6 ++
 .../tinkerpop/machine/structure/jdbc/JDBCRow.java  | 10 +++
 .../tinkerpop/machine/structure/jdbc/JDBCTest.java |  2 +-
 30 files changed, 209 insertions(+), 196 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index bcbf648..1b91203 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
@@ -97,7 +97,7 @@ public class TraversalSource implements Cloneable {
 this.prepareSourceCode();
 final Bytecode bytecode = this.bytecode.clone();
 final Coefficient coefficient = this.coefficient.clone();
-bytecode.addInstruction(coefficient, Symbols.INITIAL, objects);
+bytecode.addInstruction(coefficient, Symbols.INJECT, objects);
 return new CoreTraversal<>(this.machine, bytecode, coefficient); // 
TODO
 }
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
index 25157fd..3f98959 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
@@ -61,10 +61,11 @@ public final class TraversalUtil {
 return args;
 }
 
-public static Object[] addObjects(final Object[] original, final Object... 
updates) {
-final Object[] objects = new Object[original.length + updates.length];
-System.arraycopy(original, 0, objects, 0, original.length);
-System.arraycopy(updates, 0, objects, original.length, updates.length);
+public static Object[] addObjects(final Object head, final Object[] 
original, final Objec

[tinkerpop] branch tp4 updated: Generalized path() to work with no, one, or many labels. Have INNER JOIN working with jdbc/. Path now implements TTuple and is thus, subject to has(), value(), entries(

2019-04-29 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 706ad64  Generalized path() to work with no, one, or many labels. Have 
INNER JOIN working with jdbc/. Path now implements TTuple and is thus, subject 
to has(), value(), entries(), etc.
706ad64 is described below

commit 706ad64b78ba1144276294805f325d7cd44ea024
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 29 09:14:50 2019 -0600

Generalized path() to work with no, one, or many labels. Have INNER JOIN 
working with jdbc/. Path now implements TTuple and is thus, subject to has(), 
value(), entries(), etc.
---
 .../tinkerpop/language/gremlin/Traversal.java  | 12 -
 .../language/gremlin/TraversalSource.java  |  2 +-
 .../language/gremlin/core/CoreTraversal.java   | 30 ++-
 .../apache/tinkerpop/language/gremlin/core/__.java | 13 +
 .../tinkerpop/machine/bytecode/Bytecode.java   |  1 +
 .../machine/bytecode/compiler/CoreCompiler.java|  5 ++
 .../machine/function/filter/HasKeyFilter.java  |  8 ++-
 .../DbInitial.java => flatmap/EntriesFlatMap.java} | 33 ++--
 .../machine/function/initial/DbInitial.java| 18 +--
 .../machine/function/reduce/GroupCountReduce.java  |  4 +-
 .../tinkerpop/machine/structure/TSequence.java |  4 ++
 .../apache/tinkerpop/machine/structure/TTuple.java |  8 ++-
 .../structure/graph/{LabelSymbol.java => G.java}   |  9 +++-
 .../machine/structure/graph/IdSymbol.java  | 27 +-
 .../machine/structure/graph/LabelSymbol.java   | 29 ++-
 .../tinkerpop/machine/structure/graph/TGraph.java  |  2 +-
 .../structure/{table => rdbms}/TDatabase.java  |  5 +-
 .../machine/structure/{table => rdbms}/TRow.java   |  2 +-
 .../machine/structure/{table => rdbms}/TTable.java |  2 +-
 .../tinkerpop/machine/structure/util/J2Tuple.java  | 18 +--
 .../{graph/TGraph.java => util/JSequence.java} | 26 --
 .../tinkerpop/machine/structure/util/JTuple.java   | 26 +-
 .../machine/traverser/path/BasicPath.java  | 35 +
 .../machine/traverser/path/EmptyPath.java  | 30 +++
 .../tinkerpop/machine/traverser/path/Path.java |  4 +-
 .../tinkerpop/machine/processor/pipes/Pipes.java   |  6 +--
 .../blueprints/data/BlueprintsVertex.java  | 14 +-
 .../machine/structure/jdbc/JDBCDatabase.java   | 58 +++---
 .../tinkerpop/machine/structure/jdbc/JDBCRow.java  | 45 +++--
 .../machine/structure/jdbc/JDBCStructure.java  |  4 +-
 .../machine/structure/jdbc/JDBCTable.java  | 25 +-
 .../structure/jdbc/strategy/JDBCStrategy.java  | 13 +++--
 .../tinkerpop/machine/structure/jdbc/JDBCTest.java | 17 +--
 33 files changed, 460 insertions(+), 75 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index c281220..5015a9e 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -20,6 +20,8 @@ package org.apache.tinkerpop.language.gremlin;
 
 import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.apache.tinkerpop.machine.structure.TTuple;
+import org.apache.tinkerpop.machine.structure.rdbms.TDatabase;
+import org.apache.tinkerpop.machine.structure.util.T2Tuple;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.path.Path;
 
@@ -53,10 +55,14 @@ public interface Traversal extends Iterator {
 
 public Traversal count();
 
+public Traversal db();
+
 public Traversal emit();
 
 public Traversal emit(final Traversal emitTraversal); // 
TODO: why not 
 
+public  Traversal> entries();
+
 public Traversal explain();
 
 public Traversal filter(final Traversal filterTraversal);
@@ -95,7 +101,11 @@ public interface Traversal extends Iterator {
 
 public Traversal order();
 
-public Traversal path(final String... labels);
+public Traversal path();
+
+public  Traversal path(final String label);
+
+public Traversal path(final String label, final String... 
labels);
 
 public Traversal repeat(final Traversal repeatTraversal);
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index 82f3ddc..bcbf648 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin

[tinkerpop] branch tp4 updated: fixed a bug in Pipes.subscribe().

2019-04-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 576f880  fixed a bug in Pipes.subscribe().
576f880 is described below

commit 576f880e5918b0912fa8b9bdf531d24e116be3ba
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 23 06:54:50 2019 -0600

fixed a bug in Pipes.subscribe().
---
 .../src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java | 2 +-
 .../main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java   | 1 -
 2 files changed, 1 insertion(+), 2 deletions(-)

diff --git 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
index 20007c4..0bb39fe 100644
--- 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
+++ 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
@@ -89,7 +89,7 @@ public class Beam implements Processor {
 @Override
 public void stop() {
 try {
-if (null != this.pipelineResult)
+if (this.isRunning())
 this.pipelineResult.cancel();
 } catch (final IOException e) {
 throw new RuntimeException(e.getMessage(), e);
diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
index 317659f..cc5a4cf 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
@@ -115,7 +115,6 @@ public final class Pipes implements Processor {
 if (this.isRunning())
 throw Processor.Exceptions.processorIsCurrentlyRunning(this);
 
-this.alive.set(Boolean.TRUE);
 new Thread(() -> {
 final Iterator> iterator = this.iterator(starts);
 while (iterator.hasNext()) {



[tinkerpop] branch tp4 updated: some reorg on RxJava.

2019-04-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new d9a9863  some reorg on RxJava.
d9a9863 is described below

commit d9a986302a487359ebb42387bfb7a3a851ffcbca
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 23 05:02:31 2019 -0600

some reorg on RxJava.
---
 .../tinkerpop/machine/processor/pipes/Pipes.java   |  1 -
 .../machine/processor/rxjava/AbstractRxJava.java   | 23 +-
 .../machine/processor/rxjava/ParallelRxJava.java   |  5 +++--
 .../machine/processor/rxjava/SerialRxJava.java |  5 +++--
 4 files changed, 15 insertions(+), 19 deletions(-)

diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
index 9f3c7c8..317659f 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
@@ -100,7 +100,6 @@ public final class Pipes implements Processor {
 
 @Override
 public Iterator> iterator(final Iterator> 
starts) {
-
 if (this.isRunning())
 throw Processor.Exceptions.processorIsCurrentlyRunning(this);
 
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
index d16a4e1..9d88651 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
@@ -23,6 +23,7 @@ import 
org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.TraverserSet;
+import org.apache.tinkerpop.machine.util.IteratorUtils;
 
 import java.util.Iterator;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -63,15 +64,8 @@ public abstract class AbstractRxJava implements 
Processor {
 
 @Override
 public Iterator> iterator(final Iterator> 
starts) {
-if (this.isRunning())
-throw Processor.Exceptions.processorIsCurrentlyRunning(this);
-
-this.running.set(Boolean.TRUE);
-this.starts.clear();
-this.ends.clear();
-starts.forEachRemaining(this.starts::add);
-this.prepareFlow(this.ends::add);
-return new Iterator<>() {
+this.prepareFlow(starts, this.ends::add);
+return IteratorUtils.onLast(new Iterator<>() {
 @Override
 public boolean hasNext() {
 waitForCompletionOrResult();
@@ -83,11 +77,15 @@ public abstract class AbstractRxJava implements 
Processor {
 waitForCompletionOrResult();
 return ends.remove();
 }
-};
+}, this::stop);
 }
 
 @Override
 public void subscribe(final Iterator> starts, final 
Consumer> consumer) {
+this.prepareFlow(starts, consumer::accept);
+}
+
+protected void prepareFlow(final Iterator> starts, final 
io.reactivex.functions.Consumer> consumer) {
 if (this.isRunning())
 throw Processor.Exceptions.processorIsCurrentlyRunning(this);
 
@@ -95,13 +93,10 @@ public abstract class AbstractRxJava implements 
Processor {
 this.starts.clear();
 this.ends.clear();
 starts.forEachRemaining(this.starts::add);
-this.prepareFlow(consumer::accept);
 }
 
-protected abstract void prepareFlow(final 
io.reactivex.functions.Consumer> consumer);
-
 private void waitForCompletionOrResult() {
-while (this.ends.isEmpty() && this.isRunning()) {
+while (this.ends.isEmpty() && this.running.get()) {
 // wait until either the flow is complete or there is a traverser 
result
 }
 }
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index b0d92b1..3aac6a2 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -34,13 +34,13 @@ import 
org.apache.tinkerpop.machine.function.InitialFunction;
 import org

[tinkerpop] branch tp4 updated: Processor no longer implement Iterator. It now supports both a push-based and a pull-based execution via subscribe(Consumer) and iterator(), respectively. I haven't don

2019-04-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 2cf7cdf  Processor no longer implement Iterator. It now supports both 
a push-based and a pull-based execution via subscribe(Consumer) and iterator(), 
respectively. I haven't done this properly for Beam as I need to rework 
TraverserServer. Also, need to think about how to use Compilation within a 
push-based system so that local/nested traversals don't block on 
iterator.next().
2cf7cdf is described below

commit 2cf7cdfed05234a22b29a124c6617fb45bf5e7de
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 23 04:38:15 2019 -0600

Processor no longer implement Iterator. It now supports both a push-based 
and a pull-based execution via subscribe(Consumer) and iterator(), 
respectively. I haven't done this properly for Beam as I need to rework 
TraverserServer. Also, need to think about how to use Compilation within a 
push-based system so that local/nested traversals don't block on 
iterator.next().
---
 .../machine/bytecode/compiler/Compilation.java | 39 +++--
 .../machine/processor/EmptyProcessor.java  | 16 --
 .../machine/processor/FilterProcessor.java |  7 ++-
 .../machine/processor/LoopsProcessor.java  |  5 +-
 .../tinkerpop/machine/processor/Processor.java | 45 +--
 .../machine/processor/SimpleProcessor.java | 44 +++
 .../tinkerpop/machine/species/BasicMachine.java|  3 +-
 .../tinkerpop/machine/species/LocalMachine.java|  5 +-
 .../tinkerpop/machine/util/IteratorUtils.java  | 35 +++-
 .../tinkerpop/machine/processor/beam/Beam.java | 43 ++
 .../machine/processor/pipes/BranchStep.java|  4 +-
 .../tinkerpop/machine/processor/pipes/Pipes.java   | 45 +++
 .../machine/processor/pipes/RepeatStep.java| 22 +---
 .../machine/processor/rxjava/AbstractRxJava.java   | 65 --
 .../machine/processor/rxjava/ParallelRxJava.java   | 31 +--
 .../machine/processor/rxjava/SerialRxJava.java | 15 +++--
 16 files changed, 305 insertions(+), 119 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
index 32a04e4..f7105d9 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
@@ -28,6 +28,7 @@ import org.apache.tinkerpop.machine.structure.EmptyStructure;
 import org.apache.tinkerpop.machine.structure.StructureFactory;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.TraverserFactory;
+import org.apache.tinkerpop.machine.util.IteratorUtils;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -75,53 +76,51 @@ public final class Compilation implements 
Serializable, Cloneable {
 }
 
 public Processor getProcessor() {
-if (null == this.processor)
-this.processor = this.processorFactory.mint(this);
+this.prepareProcessor();
 return this.processor;
 }
 
 private void prepareProcessor() {
 if (null == this.processor)
 this.processor = this.processorFactory.mint(this);
-else
-this.processor.reset();
 }
 
-private Traverser prepareTraverser(final Traverser traverser) {
+private Iterator> prepareTraverser(final Traverser 
traverser) {
 final Traverser clone = traverser.clone();
 clone.coefficient().unity();
-return clone;
+return IteratorUtils.of(clone);
 }
 
 public Traverser mapTraverser(final Traverser traverser) {
 this.prepareProcessor();
-this.processor.addStart(this.prepareTraverser(traverser));
-if (!this.processor.hasNext())
+final Iterator> iterator = 
this.processor.iterator(this.prepareTraverser(traverser));
+if (!iterator.hasNext())
 throw new RuntimeException("The nested traversal is not a map 
function: " + this);
-return this.processor.next();
+final Traverser result = iterator.next();
+this.processor.stop();
+return result;
 }
 
 public Traverser mapObject(final S object) {
 this.prepareProcessor();
-
this.processor.addStart(this.traverserFactory.create(this.functions.get(0), 
object));
-return this.processor.next();
+final Iterator> iterator = 
this.processor.iterator(this.prepareTraverser(this.traverserFactory.create(this.functions.get(0),
 object)));
+final Traverser res

[tinkerpop] branch tp4 updated: I have a working implementation of recurssive-based repeat() in RxJava. This was an intense slog. Learned alot. I have written some pretty nasty nested branching test c

2019-04-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 6b39526  I have a working implementation of recurssive-based repeat() 
in RxJava. This was an intense slog. Learned alot. I have written some pretty 
nasty nested branching test cases and the model is holding up.
6b39526 is described below

commit 6b39526f4bd0109d6789af5c1d300bc18463c6d3
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 18 11:12:23 2019 -0600

I have a working implementation of recurssive-based repeat() in RxJava. 
This was an intense slog. Learned alot. I have written some pretty nasty nested 
branching test cases and the model is holding up.
---
 .../apache/tinkerpop/machine/SimpleTestSuite.java  |  14 +++
 .../machine/processor/rxjava/RepeatHead.java   | 135 +
 .../machine/processor/rxjava/RepeatTail.java   | 119 ++
 .../machine/processor/rxjava/SerialRxJava.java |  41 ---
 4 files changed, 290 insertions(+), 19 deletions(-)

diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
index 862ff71..7b1b6e8 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
@@ -24,6 +24,8 @@ import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.junit.jupiter.api.Test;
 
 import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import static org.apache.tinkerpop.language.gremlin.P.gt;
 import static org.apache.tinkerpop.language.gremlin.P.lt;
@@ -225,6 +227,18 @@ public class SimpleTestSuite extends 
AbstractTestSuite {
 g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(2));
 }
 
+// @Test (Parallel RxJava finds this too compute intensive)
+void g_injectX1X_repeatXunionXincr__incrX_unionXincr__incrXX_timesX3X() {
+verify(Stream.generate(() -> 
7L).limit(64).collect(Collectors.toList()),
+g.inject(1L).repeat(__.union(incr(), 
incr()).union(incr(),incr())).times(3));
+}
+
+// @Test (Parallel RxJava finds this too compute intensive)
+void 
g_injectX1X_repeatXunionXincr__unionXincr__incrXX_unionXincr__incrXX_timesX3X() 
{
+verify(Stream.generate(() -> 
7L).limit(216).collect(Collectors.toList()),
+g.inject(1L).repeat(__.union(incr(), 
__.union(incr(),incr())).union(incr(),incr())).times(3));
+}
+
 @Test
 void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX3X() {
 verify(List.of(4L, 5L, 5L, 6L, 5L, 6L, 6L, 7L),
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RepeatHead.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RepeatHead.java
new file mode 100644
index 000..ef94b8b
--- /dev/null
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RepeatHead.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tinkerpop.machine.processor.rxjava;
+
+import io.reactivex.Flowable;
+import io.reactivex.FlowableTransformer;
+import org.apache.tinkerpop.machine.function.branch.RepeatBranch;
+import org.apache.tinkerpop.machine.traverser.Traverser;
+import org.apache.tinkerpop.machine.traverser.TraverserSet;
+import org.reactivestreams.Processor;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+final class RepeatHead implements Processor, 
Traverser>, FlowableTransformer, Traverser> {
+
+private final TraverserSet starts = new TraverserSet<>();
+private final RepeatBranch repeatBranch;
+private Publisher> source;
+pr

[tinkerpop] branch tp4 updated: Figured out how to compile a Flow once and reuse it over and over again. This is great for nested traversals where a single traverser is inserted and result is returned

2019-04-11 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 11ceb5b  Figured out how to compile a Flow once and reuse it over and 
over again. This is great for nested traversals where a single traverser is 
inserted and result is returned and this happens over and over again for each 
incoming traverser. By 'caching' the Flow, we save on compilation costs.
11ceb5b is described below

commit 11ceb5baa725384893aa05d5c71bbaadc9ee604d
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 11 08:09:16 2019 -0600

Figured out how to compile a Flow once and reuse it over and over again. 
This is great for nested traversals where a single traverser is inserted and 
result is returned and this happens over and over again for each incoming 
traverser. By 'caching' the Flow, we save on compilation costs.
---
 .../apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java   | 3 ++-
 .../apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java   | 5 -
 .../org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java | 6 +-
 3 files changed, 11 insertions(+), 3 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
index c9b35c3..6c478f4 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
@@ -60,9 +60,10 @@ public abstract class AbstractRxJava implements 
Processor {
 
 @Override
 public void reset() {
+if (null != this.disposable)
+this.disposable.dispose();
 this.starts.clear();
 this.ends.clear();
-this.disposable = null;
 this.executed = false;
 }
 
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index d278cbd..c1cda6f 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -50,6 +50,7 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 
 private final ExecutorService threadPool;
 private final String bytecodeId;
+private final ParallelFlowable> flowable;
 
 ParallelRxJava(final Compilation compilation, final 
ExecutorService threadPool) {
 super(compilation);
@@ -57,13 +58,15 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 this.bytecodeId = compilation.getBytecode().getParent().isEmpty() ?
 (String) 
BytecodeUtil.getSourceInstructions(compilation.getBytecode(), 
RxJavaProcessor.RX_ROOT_BYTECODE_ID).get(0).args()[0] :
 null;
+// compile once and use many times
+this.flowable = 
this.compile(ParallelFlowable.from(Flowable.fromIterable(this.starts)).runOn(Schedulers.from(this.threadPool)),
 this.compilation);
 }
 
 @Override
 protected void prepareFlow() {
 if (!this.executed) {
 this.executed = true;
-this.disposable = 
this.compile(ParallelFlowable.from(Flowable.fromIterable(this.starts)).runOn(Schedulers.from(this.threadPool)),
 this.compilation).
+this.disposable = this.flowable.
 doOnNext(this.ends::add).
 sequential().
 doFinally(() -> {
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
index e68b2ad..5da1091 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
@@ -45,15 +45,19 @@ import java.util.Map;
  */
 public final class SerialRxJava extends AbstractRxJava {
 
+private final Flowable> flowable;
+
 SerialRxJava(final Compilation compilation) {
 super(compilation);
+// compile once and reuse many times
+this.flowable = 
SerialRxJava.compile(Flowable.fromIterable(this.starts), this.compilation);
 }
 
 @Override
 protected void prepareFlow() {
 if (!this.executed) {
 this.executed = true;
-this.disposable = 
SerialRxJav

[tinkerpop] branch tp4 updated: did some local testing to determine whether or not SerialRxJava was returning results in real-time. It wasn't. Learned about subscribeOn(thread) method to ensure that t

2019-04-10 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new b90f19d  did some local testing to determine whether or not 
SerialRxJava was returning results in real-time. It wasn't. Learned about 
subscribeOn(thread) method to ensure that the subscriber of the flowable is not 
in the same thread as the executor. Works.
b90f19d is described below

commit b90f19dddf8f467e835219930e620910feda8a26
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 10 17:48:04 2019 -0600

did some local testing to determine whether or not SerialRxJava was 
returning results in real-time. It wasn't. Learned about subscribeOn(thread) 
method to ensure that the subscriber of the flowable is not in the same thread 
as the executor. Works.
---
 .../org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java  | 2 +-
 .../org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java| 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index 5a6c26a..d278cbd 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -72,7 +72,7 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 
RxJavaProcessor.THREAD_POOLS.remove(this.bytecodeId);
 }
 }).
-subscribe(); // don't block the execution so results can 
be streamed back in real-time
+subscribeOn(Schedulers.newThread()).subscribe(); // don't 
block the execution so results can be streamed back in real-time
 
 }
 this.waitForCompletionOrResult();
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
index c6d2f41..e68b2ad 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
@@ -20,6 +20,7 @@ package org.apache.tinkerpop.machine.processor.rxjava;
 
 import io.reactivex.Flowable;
 import io.reactivex.processors.PublishProcessor;
+import io.reactivex.schedulers.Schedulers;
 import org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.function.BarrierFunction;
 import org.apache.tinkerpop.machine.function.BranchFunction;
@@ -54,7 +55,7 @@ public final class SerialRxJava extends 
AbstractRxJava {
 this.executed = true;
 this.disposable = 
SerialRxJava.compile(Flowable.fromIterable(this.starts), this.compilation).
 doOnNext(this.ends::add).
-subscribe(); // don't block the execution so results can 
be streamed back in real-time
+subscribeOn(Schedulers.newThread()).subscribe(); // don't 
block the execution so results can be streamed back in real-time
 }
 this.waitForCompletionOrResult();
 }



[tinkerpop] branch tp4 updated: consistent terminology between various packages. minor generalization in RxJava. Bout to dive into a new repeat() implementation for RxJava. Wish me luck.

2019-04-10 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4df7443  consistent terminology between various packages. minor 
generalization in RxJava. Bout to dive into a new repeat() implementation for 
RxJava. Wish me luck.
4df7443 is described below

commit 4df7443b704d0685bc02c9b7f6afc4a63c651c49
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 10 08:23:09 2019 -0600

consistent terminology between various packages. minor generalization in 
RxJava. Bout to dive into a new repeat() implementation for RxJava. Wish me 
luck.
---
 .../apache/tinkerpop/language/gremlin/AbstractTraversal.java | 12 ++--
 .../tinkerpop/machine/processor/rxjava/AbstractRxJava.java   |  8 ++--
 .../tinkerpop/machine/processor/rxjava/ParallelRxJava.java   |  8 ++--
 .../tinkerpop/machine/processor/rxjava/SerialRxJava.java |  4 +---
 4 files changed, 15 insertions(+), 17 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
index f65279d..51f7e08 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
@@ -37,7 +37,7 @@ public abstract class AbstractTraversal implements 
Traversal {
 protected final Bytecode bytecode;
 protected Coefficient currentCoefficient;
 private Iterator> traversers = null;
-private boolean locked = false;
+private boolean executed = false;
 
 // iteration helpers
 private long lastCount = 0L;
@@ -52,16 +52,16 @@ public abstract class AbstractTraversal implements 
Traversal {
 ///
 
 protected final  Traversal addInstruction(final String op, 
final Object... args) {
-if (this.locked)
-throw new IllegalStateException("The traversal has already been 
compiled and can no longer be mutated");
+if (this.executed)
+throw new IllegalStateException("The traversal has already been 
submitted and can no longer be mutated");
 this.bytecode.addInstruction(this.currentCoefficient, op, args);
 this.currentCoefficient.unity();
 return (Traversal) this;
 }
 
-private final void prepareTraversal() {
-if (!this.locked) {
-this.locked = true;
+private void prepareTraversal() {
+if (!this.executed) {
+this.executed = true;
 this.traversers = this.machine.submit(this.bytecode);
 }
 }
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
index 22a6e9e..c9b35c3 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
@@ -24,8 +24,6 @@ import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.TraverserSet;
 
-import java.util.concurrent.atomic.AtomicBoolean;
-
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -69,4 +67,10 @@ public abstract class AbstractRxJava implements 
Processor {
 }
 
 protected abstract void prepareFlow();
+
+void waitForCompletionOrResult() {
+while (!this.disposable.isDisposed() && this.ends.isEmpty()) {
+// wait until either the flow is complete or there is a traverser 
result
+}
+}
 }
\ No newline at end of file
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index 8828e5e..5a6c26a 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -63,9 +63,7 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 protected void prepareFlow() {
 if (!this.executed) {
 this.executed = true;
-this.disposable = this.compile(
-ParallelFlowable.from(Flowable.fromIterable(this.starts)).
-runOn(Schedulers.from(this.threadPool)), 
this.compilation).
+this.disposable = 
thi

[tinkerpop] branch tp4 updated: learned about Disposable. Got rid of the AtomicBoolean in AbstractRxJava. Both serial and parallel processors support real-time streaming of results.

2019-04-09 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new ec18e21  learned about Disposable. Got rid of the AtomicBoolean in 
AbstractRxJava. Both serial and parallel processors support real-time streaming 
of results.
ec18e21 is described below

commit ec18e214bee503fc3fe94a1557c14f63df395bd7
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 9 13:20:28 2019 -0600

learned about Disposable. Got rid of the AtomicBoolean in AbstractRxJava. 
Both serial and parallel processors support real-time streaming of results.
---
 .../tinkerpop/machine/processor/rxjava/AbstractRxJava.java   | 5 +++--
 .../tinkerpop/machine/processor/rxjava/ParallelRxJava.java   | 9 +
 .../apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java  | 8 +++-
 3 files changed, 11 insertions(+), 11 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
index 4da0657..22a6e9e 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/AbstractRxJava.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.machine.processor.rxjava;
 
+import io.reactivex.disposables.Disposable;
 import org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.traverser.Traverser;
@@ -33,7 +34,7 @@ public abstract class AbstractRxJava implements 
Processor {
 static final int MAX_REPETITIONS = 8; // TODO: this needs to be a dynamic 
configuration
 
 boolean executed = false;
-final AtomicBoolean alive = new AtomicBoolean(Boolean.FALSE);
+Disposable disposable;
 final TraverserSet starts = new TraverserSet<>();
 final TraverserSet ends = new TraverserSet<>();
 final Compilation compilation;
@@ -63,8 +64,8 @@ public abstract class AbstractRxJava implements 
Processor {
 public void reset() {
 this.starts.clear();
 this.ends.clear();
+this.disposable = null;
 this.executed = false;
-this.alive.set(Boolean.FALSE);
 }
 
 protected abstract void prepareFlow();
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index daa8bd4..8828e5e 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -63,22 +63,23 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 protected void prepareFlow() {
 if (!this.executed) {
 this.executed = true;
-this.alive.set(Boolean.TRUE);
-this.compile(
+this.disposable = this.compile(
 ParallelFlowable.from(Flowable.fromIterable(this.starts)).
 runOn(Schedulers.from(this.threadPool)), 
this.compilation).
 doOnNext(this.ends::add).
 sequential().
 doFinally(() -> {
-this.alive.set(Boolean.FALSE);
 if (null != this.bytecodeId) { // only the parent 
compilation should close the thread pool
 this.threadPool.shutdown();
 
RxJavaProcessor.THREAD_POOLS.remove(this.bytecodeId);
 }
 }).
-blockingSubscribe(); // thread this so results can be 
received before computation completes
+subscribe(); // don't block the execution so results can 
be streamed back in real-time
 
 }
+while (!this.disposable.isDisposed() && this.ends.isEmpty()) {
+// only return if there is a result ready from the flow (or the 
flow is dead)
+}
 }
 
 // EXECUTION PLAN COMPILER
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
index 183f043..87d2c97 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/Seri

[tinkerpop] branch tp4 updated: More work on ParallelRxJava thread pools. I also added SimpleRemoteParallelTest to check to make sure MachieServer + multi-threaded traversal execution play well togeth

2019-04-09 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 53d5623  More work on ParallelRxJava thread pools. I also added 
SimpleRemoteParallelTest to check to make sure MachieServer + multi-threaded 
traversal execution play well together. They do.
53d5623 is described below

commit 53d5623e34436a6debfc2adfde58a5a385b0724d
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 9 13:02:40 2019 -0600

More work on ParallelRxJava thread pools. I also added 
SimpleRemoteParallelTest to check to make sure MachieServer + multi-threaded 
traversal execution play well together. They do.
---
 .../tinkerpop/machine/AbstractTestSuite.java   | 15 +++---
 .../machine/processor/beam/SimpleRemoteTest.java   | 15 --
 .../machine/processor/pipes/SimpleRemoteTest.java  | 15 --
 .../machine/processor/rxjava/ParallelRxJava.java   | 11 +--
 .../machine/processor/rxjava/RxJavaProcessor.java  | 15 +++---
 .../processor/rxjava/strategy/RxJavaStrategy.java  |  8 
 .../machine/processor/rxjava/RxJavaBenchmark.java  |  4 ++--
 .../processor/rxjava/SimpleLocalParallelTest.java  |  2 +-
 ...rialTest.java => SimpleRemoteParallelTest.java} | 23 +-
 .../processor/rxjava/SimpleRemoteSerialTest.java   | 15 --
 10 files changed, 77 insertions(+), 46 deletions(-)

diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
index 8524292..822636e 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
@@ -40,16 +40,17 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 public abstract class AbstractTestSuite {
 
 protected Machine machine;
-protected TraversalSource g;
+TraversalSource g;
 
-public AbstractTestSuite(final Machine machine, final Bytecode source) {
+AbstractTestSuite(final Machine machine, final Bytecode source) {
 this.machine = machine;
 this.g = Gremlin.traversal(machine);
-BytecodeUtil.mergeSourceInstructions(source, 
TraversalUtil.getBytecode(g));
+BytecodeUtil.mergeSourceInstructions(source, 
TraversalUtil.getBytecode(this.g));
 }
 
 @AfterAll
 public void shutdown() {
+this.g.close();
 this.machine.close();
 }
 
@@ -69,4 +70,12 @@ public abstract class AbstractTestSuite {
 assertFalse(traversal.hasNext());
 }
 
+public static void sleep(final int time) {
+try {
+Thread.sleep(time);
+} catch (final InterruptedException e) {
+// do nothing
+}
+}
+
 }
diff --git 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
index 5b3448a..72fac52 100644
--- 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
+++ 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
@@ -18,12 +18,14 @@
  */
 package org.apache.tinkerpop.machine.processor.beam;
 
+import org.apache.tinkerpop.machine.AbstractTestSuite;
 import org.apache.tinkerpop.machine.SimpleTestSuite;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.compiler.CoreCompiler;
 import org.apache.tinkerpop.machine.species.remote.MachineServer;
 import org.apache.tinkerpop.machine.species.remote.RemoteMachine;
 import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
 
 import java.util.Map;
 
@@ -46,14 +48,15 @@ public class SimpleRemoteTest extends SimpleTestSuite {
 super(RemoteMachine.open(, "localhost", ), BYTECODE);
 }
 
+@AfterEach
+void delayShutdown() {
+AbstractTestSuite.sleep(100);
+}
+
 @AfterAll
-static void stopServer() {
+void stopServer() {
 SERVER.close();
-try {
-Thread.sleep(10);
-} catch (final InterruptedException e) {
-
-}
+AbstractTestSuite.sleep(100);
 }
 
 }
\ No newline at end of file
diff --git 
a/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
 
b/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
index 5e0b32b..f895ada 100644
--- 
a/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
+++ 
b

[tinkerpop] branch tp4 updated: thread pool hell. I have it so RxJavaStrategy analyzes nested compilations to see if they are 'simple' or not. if they are simple, then a serial flow is used to execute

2019-04-09 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new ef0fd1f  thread pool hell. I have it so RxJavaStrategy analyzes nested 
compilations to see if they are 'simple' or not. if they are simple, then a 
serial flow is used to execute it. This way, we don't exhaust the threadpool 
with overly threaded nests of compilations. RxJavaStrategy is the first complex 
strategy. It uses the new Bytecode hiearchy to assoicate a thread pool with the 
root bytecode.
ef0fd1f is described below

commit ef0fd1f3a6f87939ebd063f14f078cf47750f3ac
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 9 11:51:42 2019 -0600

thread pool hell. I have it so RxJavaStrategy analyzes nested compilations 
to see if they are 'simple' or not. if they are simple, then a serial flow is 
used to execute it. This way, we don't exhaust the threadpool with overly 
threaded nests of compilations. RxJavaStrategy is the first complex strategy. 
It uses the new Bytecode hiearchy to assoicate a thread pool with the root 
bytecode.
---
 .../machine/processor/rxjava/ParallelRxJava.java   |  2 +-
 .../machine/processor/rxjava/RxJavaProcessor.java  |  4 ++--
 .../machine/processor/rxjava/SerialRxJava.java |  2 +-
 .../processor/rxjava/strategy/RxJavaStrategy.java  | 27 --
 .../machine/processor/rxjava/RxJavaBenchmark.java  |  6 ++---
 .../processor/rxjava/SimpleLocalParallelTest.java  |  2 +-
 6 files changed, 28 insertions(+), 15 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index 5b81368..9c9476d 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -64,8 +64,8 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 runOn(Schedulers.from(this.threadPool)), 
this.compilation).
 doOnNext(this.ends::add).
 sequential().
-doOnComplete(() -> this.alive.set(Boolean.FALSE)).
 doFinally(() -> {
+this.alive.set(Boolean.FALSE);
 if 
(this.compilation.getBytecode().getParent().isEmpty()) // only the parent 
compilation should close the thread pool
 this.threadPool.shutdown();
 }).
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJavaProcessor.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJavaProcessor.java
index be5d850..e9792eb 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJavaProcessor.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJavaProcessor.java
@@ -56,8 +56,8 @@ public final class RxJavaProcessor implements 
ProcessorFactory {
 public  Processor mint(final Compilation 
compilation) {
 final int threads = (int) 
this.configuration.getOrDefault(RxJavaProcessor.RXJAVA_THREADS, 0);
 final String id = (String) 
BytecodeUtil.getSourceInstructions(BytecodeUtil.getRootBytecode(compilation.getBytecode()),
 RX_BYCODE_ID).get(0).args()[0];
-final ExecutorService threadPool = 
RxJavaProcessor.THREAD_POOLS.compute(id, (key, value) -> null == value && 
threads > 0 ? Executors.newCachedThreadPool() : value);
-// System.out.println(id + "--" + threadPool + "---" + THREAD_POOLS);
+final ExecutorService threadPool = threads > 0 ? 
RxJavaProcessor.THREAD_POOLS.computeIfAbsent(id, key -> 
Executors.newFixedThreadPool(threads)) : null;
+// System.out.println(id + "::" + threads + "--" + threadPool);
 return null == threadPool ?
 new SerialRxJava<>(compilation) :
 new ParallelRxJava<>(compilation, threadPool);
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
index ace99b4..183f043 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
@@ -55,7 +55,7 @@ public final class SerialRxJava extends 
AbstractRxJava 

[tinkerpop] branch tp4 updated: nested Bytecode now knows about its parent bytecode. It is also possible to walk the Bytecode tree up to the root. This is used to share information between Compilation

2019-04-09 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new ae6ac40  nested Bytecode now knows about its parent bytecode. It is 
also possible to walk the Bytecode tree up to the root. This is used to share 
information between Compilations such as thread pool info in ParallelRxJava. 
Noticing a weird relationship between ProcessorFactory and Compilation. Need to 
think things through. Anywho, for now ParallelRxJava traverasls share a common 
thread pool for their execution. Need to get smart and not thread simple nests 
so we don't spawn so ma [...]
ae6ac40 is described below

commit ae6ac400ba1eb511ab52995dad430dda12365d99
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 9 09:03:34 2019 -0600

nested Bytecode now knows about its parent bytecode. It is also possible to 
walk the Bytecode tree up to the root. This is used to share information 
between Compilations such as thread pool info in ParallelRxJava. Noticing a 
weird relationship between ProcessorFactory and Compilation. Need to think 
things through. Anywho, for now ParallelRxJava traverasls share a common thread 
pool for their execution. Need to get smart and not thread simple nests so we 
don't spawn so many threads.
---
 .../tinkerpop/language/gremlin/TraversalUtil.java  |  5 +-
 .../language/gremlin/common/CommonTraversal.java   |  6 +--
 .../language/gremlin/core/CoreTraversal.java   |  6 +--
 .../tinkerpop/machine/bytecode/Bytecode.java   | 33 +++-
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   | 24 +
 .../tinkerpop/machine/bytecode/Instruction.java| 16 ++
 .../machine/bytecode/compiler/Compilation.java |  8 +++
 .../tinkerpop/machine/util/StringFactory.java  |  5 ++
 .../tinkerpop/machine/bytecode/BytecodeTest.java   | 60 ++
 .../machine/processor/rxjava/ParallelRxJava.java   | 13 ++---
 .../machine/processor/rxjava/RxJavaProcessor.java  | 22 ++--
 .../processor/rxjava/strategy/RxJavaStrategy.java  | 20 ++--
 .../machine/processor/rxjava/RxJavaBenchmark.java  |  2 +-
 .../processor/rxjava/SimpleLocalParallelTest.java  |  2 +-
 14 files changed, 183 insertions(+), 39 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
index efdf824..3587e4c 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
@@ -45,9 +45,8 @@ public final class TraversalUtil {
 }
 
 public static  Traversal insertRepeatInstruction(final 
AbstractTraversal traversal, final char type, final Object argument) {
-final Instruction lastInstruction = 
traversal.bytecode.lastInstruction();
-if (lastInstruction.op().equals(Symbols.REPEAT))
-lastInstruction.addArgs(type, argument);
+if (traversal.bytecode.lastInstruction().op().equals(Symbols.REPEAT))
+traversal.bytecode.addArgs(type, argument);
 else
 traversal.addInstruction(Symbols.REPEAT, type, argument);
 return traversal;
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index 7d44169..95aeb51 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
@@ -61,13 +61,13 @@ public class CommonTraversal extends 
AbstractTraversal {
 
 @Override
 public Traversal by(final String byString) {
-this.bytecode.lastInstruction().addArg(byString);
+this.bytecode.addArgs(byString);
 return this;
 }
 
 @Override
 public Traversal by(final Traversal byTraversal) {
-
this.bytecode.lastInstruction().addArg(TraversalUtil.getBytecode(byTraversal));
+this.bytecode.addArgs(TraversalUtil.getBytecode(byTraversal));
 return this;
 }
 
@@ -78,7 +78,7 @@ public class CommonTraversal extends 
AbstractTraversal {
 
 @Override
 public Traversal by(final Traversal byTraversal, final 
Order order) {
-
this.bytecode.lastInstruction().addArgs(TraversalUtil.getBytecode(byTraversal), 
order.name());
+this.bytecode.addArgs(TraversalUtil.getBytecode(byTraversal), 
order.name());
 return this;
 }
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop

[tinkerpop] branch tp4 updated: cleaned up Traverser species equals() and hashCode() some more. Scary to have faults in those.

2019-04-09 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new a633015  cleaned up Traverser species equals() and hashCode() some 
more. Scary to have faults in those.
a633015 is described below

commit a633015216f1646b0efa876003dd3f07b8e9e834
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 9 06:58:46 2019 -0600

cleaned up Traverser species equals() and hashCode() some more. Scary to 
have faults in those.
---
 .../apache/tinkerpop/machine/traverser/species/COP_Traverser.java  | 6 --
 .../apache/tinkerpop/machine/traverser/species/COR_Traverser.java  | 7 ++-
 .../tinkerpop/machine/traverser/species/ProjectedTraverser.java| 3 ++-
 3 files changed, 12 insertions(+), 4 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COP_Traverser.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COP_Traverser.java
index 24ad297..f2bda7a 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COP_Traverser.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COP_Traverser.java
@@ -57,12 +57,14 @@ public class COP_Traverser extends CO_Traverser 
{
 
 @Override
 public boolean equals(final Object other) {
-return other instanceof COP_Traverser && ((COP_Traverser) 
other).object.equals(this.object);
+return other instanceof COP_Traverser &&
+((COP_Traverser) other).object.equals(this.object) &&
+((COP_Traverser) other).path.equals(this.path);
 }
 
 @Override
 public int hashCode() {
-return this.object.hashCode(); // TODO: include path
+return this.object.hashCode() ^ this.path.hashCode();
 }
 
 @Override
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
index ba4184a..d5ed29c 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
@@ -29,7 +29,7 @@ public class COR_Traverser extends CO_Traverser {
 
 private short loops = 0;
 
-public COR_Traverser(final Coefficient coefficient, final S object) {
+COR_Traverser(final Coefficient coefficient, final S object) {
 super(coefficient, object);
 }
 
@@ -62,4 +62,9 @@ public class COR_Traverser extends CO_Traverser {
 this.object.equals(((COR_Traverser) other).object) &&
 this.loops == ((COR_Traverser) other).loops;
 }
+
+@Override
+public int hashCode() {
+return super.hashCode() ^ this.loops;
+}
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/ProjectedTraverser.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/ProjectedTraverser.java
index 935a516..fe6c7a4 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/ProjectedTraverser.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/ProjectedTraverser.java
@@ -83,7 +83,8 @@ public class ProjectedTraverser implements 
Traverser {
 
 @Override
 public boolean equals(final Object object) {
-return object instanceof ProjectedTraverser && 
this.baseTraverser.equals(((ProjectedTraverser) object).baseTraverser);
+return object instanceof ProjectedTraverser &&
+this.baseTraverser.equals(((ProjectedTraverser) 
object).baseTraverser);
 }
 
 @Override



[tinkerpop] branch tp4 updated: found a bug in COR_Traverser that was showing up in looping in Pipes. Didn't do equals() correctly and TraverserSet traversers were merging inappropriatly.

2019-04-09 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new e252855  found a bug in COR_Traverser that was showing up in looping 
in Pipes. Didn't do equals() correctly and TraverserSet traversers were merging 
inappropriatly.
e252855 is described below

commit e252855c136126b19a09aaacb592c61c535d909b
Author: Marko A. Rodriguez 
AuthorDate: Tue Apr 9 06:53:02 2019 -0600

found a bug in COR_Traverser that was showing up in looping in Pipes. 
Didn't do equals() correctly and TraverserSet traversers were merging 
inappropriatly.
---
 .../apache/tinkerpop/machine/traverser/species/COR_Traverser.java  | 7 +++
 .../main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java| 2 +-
 .../org/apache/tinkerpop/machine/processor/pipes/BranchStep.java   | 4 ++--
 3 files changed, 10 insertions(+), 3 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
index b9ccc67..ba4184a 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/species/COR_Traverser.java
@@ -55,4 +55,11 @@ public class COR_Traverser extends CO_Traverser {
 clone.coefficient.multiply(function.coefficient());
 return clone;
 }
+
+@Override
+public boolean equals(final Object other) {
+return other instanceof COR_Traverser &&
+this.object.equals(((COR_Traverser) other).object) &&
+this.loops == ((COR_Traverser) other).loops;
+}
 }
diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
index 31177e4..862ff71 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
@@ -225,7 +225,7 @@ public class SimpleTestSuite extends 
AbstractTestSuite {
 g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(2));
 }
 
-//@Test
+@Test
 void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX3X() {
 verify(List.of(4L, 5L, 5L, 6L, 5L, 6L, 6L, 7L),
 g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(3));
diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
index 0ef8280..667af99 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
@@ -71,8 +71,8 @@ final class BranchStep extends AbstractStep {
 }
 }
 if (!found) {
-for (final Compilation defaultBranches : 
this.defaultBranches) {
-((MultiIterator>) 
this.nextTraversers).addIterator(defaultBranches.addTraverser(traverser.clone()));
+for (final Compilation defaultBranch : 
this.defaultBranches) {
+((MultiIterator>) 
this.nextTraversers).addIterator(defaultBranch.addTraverser(traverser.clone()));
 }
 }
 }



[tinkerpop] branch tp4 updated: added a Thread.sleep() in the test suite to make sure the servers are shutdown fully between tests. Learned about share() for branching in Flowable. Added it to SerialR

2019-04-08 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 3b452af  added a Thread.sleep() in the test suite to make sure the 
servers are shutdown fully between tests. Learned about share() for branching 
in Flowable. Added it to SerialRxJava.
3b452af is described below

commit 3b452af20116bdfb61252d5e8ea75bf3b40c35ca
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 8 20:07:31 2019 -0600

added a Thread.sleep() in the test suite to make sure the servers are 
shutdown fully between tests. Learned about share() for branching in Flowable. 
Added it to SerialRxJava.
---
 .../apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java| 5 +
 .../apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java   | 5 +
 .../org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java  | 4 ++--
 .../tinkerpop/machine/processor/rxjava/SimpleRemoteSerialTest.java   | 5 +
 4 files changed, 17 insertions(+), 2 deletions(-)

diff --git 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
index b65460f..5b3448a 100644
--- 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
+++ 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/SimpleRemoteTest.java
@@ -49,6 +49,11 @@ public class SimpleRemoteTest extends SimpleTestSuite {
 @AfterAll
 static void stopServer() {
 SERVER.close();
+try {
+Thread.sleep(10);
+} catch (final InterruptedException e) {
+
+}
 }
 
 }
\ No newline at end of file
diff --git 
a/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
 
b/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
index 63c79af..5e0b32b 100644
--- 
a/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
+++ 
b/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/SimpleRemoteTest.java
@@ -44,6 +44,11 @@ class SimpleRemoteTest extends SimpleTestSuite {
 @AfterAll
 static void stopServer() {
 SERVER.close();
+try {
+Thread.sleep(10);
+} catch (final InterruptedException e) {
+
+}
 }
 
 }
\ No newline at end of file
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
index 5331ee3..ace99b4 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/SerialRxJava.java
@@ -90,7 +90,7 @@ public final class SerialRxJava extends 
AbstractRxJava {
 final BarrierFunction barrierFunction = 
(BarrierFunction) function;
 return flow.reduce(barrierFunction.getInitialValue(), new 
Barrier<>(barrierFunction)).toFlowable().flatMapIterable(new 
BarrierFlow<>(barrierFunction, traverserFactory));
 } else if (function instanceof BranchFunction) {
-final Flowable selectorFlow = flow.map(new 
BranchFlow<>((BranchFunction) function));
+final Flowable selectorFlow = flow.map(new 
BranchFlow<>((BranchFunction) function)).share();
 final List>> branchFlows = new 
ArrayList<>();
 int branchCounter = 0;
 for (final Map.Entry, List>> branches : ((BranchFunction) function).getBranches().entrySet()) {
@@ -110,7 +110,7 @@ public final class SerialRxJava extends 
AbstractRxJava {
 Flowable selectorFlow;
 for (int i = 0; i < MAX_REPETITIONS; i++) {
 if (repeatBranch.hasStartPredicates()) {
-selectorFlow = flow.flatMapIterable(new 
RepeatStart<>(repeatBranch));
+selectorFlow = flow.flatMapIterable(new 
RepeatStart<>(repeatBranch)).share();
 outputs.add(selectorFlow.filter(list -> 
list.get(0).equals(0)).map(list -> (Traverser) list.get(1)));
 flow = compile(selectorFlow.filter(list -> 
list.get(0).equals(1)).map(list -> (Traverser) list.get(1)), 
repeatBranch.getRepeat());
 } else
diff --git 
a/java/machine/processor/rxjava/src/test/java/org/apache/tinkerpop/machine/processor/rxjava/SimpleRemoteSerialTest.java
 
b/java/machine/process

[tinkerpop] branch tp4 updated: was able to remove a sequential conversion in ParallelRxJava repeat().

2019-04-08 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 5b64d7b  was able to remove a sequential conversion in ParallelRxJava 
repeat().
5b64d7b is described below

commit 5b64d7b715218b422402392bff7b6249b25f5b54
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 8 16:27:47 2019 -0600

was able to remove a sequential conversion in ParallelRxJava repeat().
---
 .../tinkerpop/machine/processor/rxjava/ParallelRxJava.java   | 12 
 1 file changed, 8 insertions(+), 4 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
index 75bb398..2ee8b35 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ParallelRxJava.java
@@ -90,7 +90,8 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 else if (function instanceof FilterFunction) {
 return (ParallelFlowable) flow.filter(new 
FilterFlow<>((FilterFunction) function));
 } else if (function instanceof FlatMapFunction) {
-return flow.sequential().flatMapIterable(new 
FlatMapFlow<>((FlatMapFunction) 
function)).parallel().runOn(Schedulers.from(this.threadPool));
+final FlatMapFlow flatMapFlow = new 
FlatMapFlow<>((FlatMapFunction) function);
+return flow.flatMap(t -> 
Flowable.fromIterable(flatMapFlow.apply(t)));
 } else if (function instanceof InitialFunction) {
 return Flowable.fromIterable(() -> 
IteratorUtils.map(((InitialFunction) function).get(), s -> 
traverserFactory.create(function, 
s))).parallel().runOn(Schedulers.from(this.threadPool));
 } else if (function instanceof ReduceFunction) {
@@ -120,13 +121,16 @@ public final class ParallelRxJava extends 
AbstractRxJava {
 ParallelFlowable selectorFlow;
 for (int i = 0; i < MAX_REPETITIONS; i++) {
 if (repeatBranch.hasStartPredicates()) {
-selectorFlow = flow.sequential().flatMapIterable(new 
RepeatStart<>(repeatBranch)).parallel().runOn(Schedulers.from(this.threadPool));
+final RepeatStart repeatStart = new 
RepeatStart<>(repeatBranch);
+selectorFlow = flow.flatMap(t -> 
Flowable.fromIterable(repeatStart.apply(t)));
 outputs.add(selectorFlow.filter(list -> 
list.get(0).equals(0)).map(list -> (Traverser) list.get(1)).sequential());
 flow = this.compile(selectorFlow.filter(list -> 
list.get(0).equals(1)).map(list -> (Traverser) list.get(1)), 
(Compilation) repeatBranch.getRepeat());
 } else
 flow = this.compile(flow, (Compilation) 
repeatBranch.getRepeat());
-selectorFlow = flow.sequential().flatMapIterable(new 
RepeatEnd<>(repeatBranch)).parallel().runOn(Schedulers.from(this.threadPool));
-outputs.add(selectorFlow.sequential().filter(list -> 
list.get(0).equals(0)).map(list -> (Traverser) list.get(1)));
+///
+final RepeatEnd repeatEnd = new 
RepeatEnd<>(repeatBranch);
+selectorFlow = flow.flatMap(t -> 
Flowable.fromIterable(repeatEnd.apply(t)));
+outputs.add(selectorFlow.filter(list -> 
list.get(0).equals(0)).map(list -> (Traverser) list.get(1)).sequential());
 flow = selectorFlow.filter(list -> 
list.get(0).equals(1)).map(list -> (Traverser) list.get(1));
 }
 return (ParallelFlowable) 
PublishProcessor.merge(outputs).parallel().runOn(Schedulers.from(this.threadPool));



[tinkerpop] branch tp4 updated: added a benchmark test case in rxJava to compare Parallel, serial, and Pipes.

2019-04-08 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new d35228c  added a benchmark test case in rxJava to compare Parallel, 
serial, and Pipes.
d35228c is described below

commit d35228caec1fae2932ce615d2df278cbfea5ef61
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 8 10:52:12 2019 -0600

added a benchmark test case in rxJava to compare Parallel, serial, and 
Pipes.
---
 .../language/gremlin/AbstractTraversal.java| 12 
 .../tinkerpop/language/gremlin/Traversal.java  |  2 +
 .../tinkerpop/machine/function/FilterFunction.java |  2 +
 .../machine/function/FlatMapFunction.java  |  2 +
 .../tinkerpop/machine/function/MapFunction.java|  2 +
 .../machine/function/filter/IdentityFilter.java|  7 +-
 .../machine/function/flatmap/UnfoldFlatMap.java|  8 ++-
 .../machine/function/map/ConstantMap.java  |  6 ++
 .../tinkerpop/machine/function/map/IncrMap.java|  7 +-
 .../tinkerpop/machine/function/map/LoopsMap.java   |  7 +-
 .../machine/processor/beam/SimpleRemoteTest.java   |  7 +-
 java/machine/processor/rxjava/pom.xml  |  6 ++
 .../machine/processor/rxjava/FilterFlow.java   |  8 +--
 .../machine/processor/rxjava/FlatMapFlow.java  |  8 +--
 .../machine/processor/rxjava/MapFlow.java  |  8 +--
 .../machine/processor/rxjava/RxJavaBenchmark.java  | 78 ++
 16 files changed, 149 insertions(+), 21 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
index 3d2811c..f65279d 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
@@ -26,6 +26,7 @@ import org.apache.tinkerpop.machine.traverser.Traverser;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.NoSuchElementException;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -102,6 +103,17 @@ public abstract class AbstractTraversal 
implements Traversal {
 }
 
 @Override
+public void iterate() {
+try {
+while (true) {
+this.nextTraverser();
+}
+} catch (final NoSuchElementException e) {
+// do nothing
+}
+}
+
+@Override
 public String toString() {
 return this.bytecode.toString();
 }
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 98b5d9a..f0fa839 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -119,4 +119,6 @@ public interface Traversal extends Iterator {
 
 public Traverser nextTraverser();
 
+public void iterate();
+
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FilterFunction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FilterFunction.java
index a8bcb7b..cd0653e 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FilterFunction.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FilterFunction.java
@@ -26,4 +26,6 @@ import java.util.function.Predicate;
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public interface FilterFunction extends Predicate>, 
CFunction {
+
+public FilterFunction clone();
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FlatMapFunction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FlatMapFunction.java
index 573af55..4a2aa8e 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FlatMapFunction.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/FlatMapFunction.java
@@ -27,4 +27,6 @@ import java.util.function.Function;
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public interface FlatMapFunction extends Function, 
Iterator>, CFunction {
+
+public FlatMapFunction clone();
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/MapFunction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/MapFunction.java
index e18738a..6d4d254 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/m

[tinkerpop] branch tp4 updated: minor tweaks to servers to try and make sure address binding doesn't fail in test suite.

2019-04-08 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new d77e0b7  minor tweaks to servers to try and make sure address binding 
doesn't fail in test suite.
d77e0b7 is described below

commit d77e0b765a99e8ce498aefa79adfcb24091a0ab6
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 8 10:09:30 2019 -0600

minor tweaks to servers to try and make sure address binding doesn't fail 
in test suite.
---
 .../apache/tinkerpop/machine/species/remote/MachineServer.java| 8 +++-
 .../apache/tinkerpop/machine/species/remote/TraverserServer.java  | 6 ++
 2 files changed, 5 insertions(+), 9 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index e041d83..8931f40 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -41,7 +41,7 @@ public final class MachineServer implements AutoCloseable {
 
 private final int machineServerPort;
 private ServerSocket machineServerSocket;
-private AtomicBoolean serverAlive = new AtomicBoolean(Boolean.FALSE);
+private AtomicBoolean serverAlive = new AtomicBoolean(Boolean.TRUE);
 private final Machine machine = LocalMachine.open();
 
 public MachineServer(final int machineServerPort) {
@@ -51,7 +51,6 @@ public final class MachineServer implements AutoCloseable {
 
 private void run() {
 try {
-this.serverAlive.set(Boolean.TRUE);
 this.machineServerSocket = new 
ServerSocket(this.machineServerPort);
 while (this.serverAlive.get()) {
 final Socket clientSocket = this.machineServerSocket.accept();
@@ -66,10 +65,9 @@ public final class MachineServer implements AutoCloseable {
 public void close() {
 if (this.serverAlive.get()) {
 try {
-if (null != this.machineServerSocket)
-this.machineServerSocket.close();
-this.machine.close();
 this.serverAlive.set(Boolean.FALSE);
+this.machineServerSocket.close();
+this.machine.close();
 } catch (final IOException e) {
 throw new RuntimeException(e.getMessage(), e);
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
index 4e61fff..5153f92 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
@@ -38,7 +38,7 @@ public final class TraverserServer implements 
AutoCloseable, Iterator traverserSet = new TraverserSet<>();
 private final int serverPort;
 private ServerSocket serverSocket;
-private AtomicBoolean serverAlive = new AtomicBoolean(Boolean.FALSE);
+private AtomicBoolean serverAlive = new AtomicBoolean(Boolean.TRUE);
 
 public TraverserServer(final int serverPort) {
 this.serverPort = serverPort;
@@ -47,7 +47,6 @@ public final class TraverserServer implements 
AutoCloseable, Iterator implements 
AutoCloseable, Iterator

[tinkerpop] branch tp4 updated: got ParallelFlowable really nice in rxJava. Had to do some thread safe work on TraverserSet. Having a weird socket issue where tests are closing and starting too fast b

2019-04-08 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 51f8eb9  got ParallelFlowable really nice in rxJava. Had to do some 
thread safe work on TraverserSet. Having a weird socket issue where tests are 
closing and starting too fast before socket can be fully closed. Don't know why 
this creeped up.
51f8eb9 is described below

commit 51f8eb9fdd906eba9336b7eb4eb4389dcd69e44a
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 8 09:54:06 2019 -0600

got ParallelFlowable really nice in rxJava. Had to do some thread safe work 
on TraverserSet. Having a weird socket issue where tests are closing and 
starting too fast before socket can be fully closed. Don't know why this 
creeped up.
---
 .../machine/function/branch/RepeatBranch.java  |  4 +-
 .../machine/species/remote/MachineServer.java  |  8 ++--
 .../machine/species/remote/TraverserServer.java| 16 ++--
 .../tinkerpop/machine/traverser/TraverserSet.java  | 32 +---
 .../apache/tinkerpop/machine/SimpleTestSuite.java  |  2 +-
 .../machine/processor/rxjava/AbstractRxJava.java   |  3 +-
 .../machine/processor/rxjava/FlatMapFlow.java  |  6 +--
 .../machine/processor/rxjava/ParallelRxJava.java   | 44 +++---
 .../machine/processor/rxjava/RepeatEnd.java| 32 +---
 .../machine/processor/rxjava/RepeatStart.java  | 30 ---
 .../machine/processor/rxjava/SerialRxJava.java | 11 --
 11 files changed, 109 insertions(+), 79 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
index eddb43b..378eb6d 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
@@ -112,8 +112,8 @@ public final class RepeatBranch extends 
AbstractFunction {
 public RepeatBranch clone() {
 final RepeatBranch clone = (RepeatBranch) super.clone();
 clone.repeatCompilation = this.repeatCompilation.clone();
-clone.emitCompilation = this.emitCompilation.clone();
-clone.untilCompilation = this.untilCompilation.clone();
+clone.emitCompilation = null == this.emitCompilation ? null : 
this.emitCompilation.clone();
+clone.untilCompilation = null == this.untilCompilation ? null : 
this.untilCompilation.clone();
 return clone;
 }
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index 8931f40..e041d83 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -41,7 +41,7 @@ public final class MachineServer implements AutoCloseable {
 
 private final int machineServerPort;
 private ServerSocket machineServerSocket;
-private AtomicBoolean serverAlive = new AtomicBoolean(Boolean.TRUE);
+private AtomicBoolean serverAlive = new AtomicBoolean(Boolean.FALSE);
 private final Machine machine = LocalMachine.open();
 
 public MachineServer(final int machineServerPort) {
@@ -51,6 +51,7 @@ public final class MachineServer implements AutoCloseable {
 
 private void run() {
 try {
+this.serverAlive.set(Boolean.TRUE);
 this.machineServerSocket = new 
ServerSocket(this.machineServerPort);
 while (this.serverAlive.get()) {
 final Socket clientSocket = this.machineServerSocket.accept();
@@ -65,9 +66,10 @@ public final class MachineServer implements AutoCloseable {
 public void close() {
 if (this.serverAlive.get()) {
 try {
-this.serverAlive.set(Boolean.FALSE);
-this.machineServerSocket.close();
+if (null != this.machineServerSocket)
+this.machineServerSocket.close();
 this.machine.close();
+this.serverAlive.set(Boolean.FALSE);
 } catch (final IOException e) {
 throw new RuntimeException(e.getMessage(), e);
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
index d647f96..4e61fff 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote

[tinkerpop] branch tp4 updated: first working version of parallel RxJava. This implementation provides the pattern for dealing with threading. Functions that maintain compilations (nested traversals),

2019-04-08 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 467c543  first working version of parallel RxJava. This implementation 
provides the pattern for dealing with threading. Functions that maintain 
compilations (nested traversals), need to be ThreadLocal so that the 
compilation object isn't interacted with multiple threads. I thought I was 
going to have to ThreadLocal throughout machine-core, but thankfully, no. It 
makes sense -- if your processors 'map operator' is threaded, then the TP map 
function it wraps needs to ensure thread- [...]
467c543 is described below

commit 467c5439a19c23c2aa9d88bab973fd421556d16c
Author: Marko A. Rodriguez 
AuthorDate: Mon Apr 8 07:46:24 2019 -0600

first working version of parallel RxJava. This implementation provides the 
pattern for dealing with threading. Functions that maintain compilations 
(nested traversals), need to be ThreadLocal so that the compilation object 
isn't interacted with multiple threads. I thought I was going to have to 
ThreadLocal throughout machine-core, but thankfully, no. It makes sense -- if 
your processors 'map operator' is threaded, then the TP map function it wraps 
needs to ensure thread-safe use of th [...]
---
 .../machine/function/branch/BranchBranch.java  |   2 +-
 .../tinkerpop/machine/processor/pipes/Pipes.java   |  22 ++--
 .../machine/processor/rxjava/AbstractRxJava.java   |  70 +++
 .../machine/processor/rxjava/BranchFlow.java   |  14 ++-
 .../machine/processor/rxjava/FilterFlow.java   |   6 +-
 .../machine/processor/rxjava/MapFlow.java  |   6 +-
 .../machine/processor/rxjava/ParallelRxJava.java   | 134 +
 .../machine/processor/rxjava/RepeatEnd.java|  64 ++
 .../machine/processor/rxjava/RepeatStart.java  |  63 ++
 .../machine/processor/rxjava/RxJavaProcessor.java  |  19 ++-
 .../rxjava/{RxJava.java => SerialRxJava.java}  | 125 +++
 .../processor/rxjava/strategy/RxJavaStrategy.java  |   4 +-
 ...LocalTest.java => SimpleLocalParallelTest.java} |   8 +-
 ...leLocalTest.java => SimpleLocalSerialTest.java} |   4 +-
 ...RemoteTest.java => SimpleRemoteSerialTest.java} |   4 +-
 15 files changed, 408 insertions(+), 137 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
index 165a87d..b548b8c 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
@@ -62,7 +62,7 @@ public final class BranchBranch extends 
AbstractFunction implements
 for (final Compilation compilation : entry.getValue()) {
 compilations.add(compilation.clone());
 }
-clone.branches.put(entry.getKey().clone(), compilations);
+clone.branches.put(null == entry.getKey() ? null : 
entry.getKey().clone(), compilations);
 }
 return clone;
 }
diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
index 9ef86e1..e370c58 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/Pipes.java
@@ -28,8 +28,8 @@ import org.apache.tinkerpop.machine.function.InitialFunction;
 import org.apache.tinkerpop.machine.function.MapFunction;
 import org.apache.tinkerpop.machine.function.ReduceFunction;
 import org.apache.tinkerpop.machine.function.branch.RepeatBranch;
-import org.apache.tinkerpop.machine.processor.pipes.util.InMemoryReducer;
 import org.apache.tinkerpop.machine.processor.Processor;
+import org.apache.tinkerpop.machine.processor.pipes.util.InMemoryReducer;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
 import java.util.ArrayList;
@@ -45,7 +45,7 @@ public final class Pipes implements Processor {
 private SourceStep startStep;
 
 public Pipes(final Compilation compilation) {
-Step previousStep = EmptyStep.instance();
+Step previousStep = EmptyStep.instance();
 for (final CFunction function : compilation.getFunctions()) {
 final Step nextStep;
 if (this.steps.isEmpty() && !(function instanceof 
InitialFunction)) {
@@ -55,22 +55,22 @@ public final class Pipes implements Processor {
 }
 
 if (function instanceof RepeatBranch)
-  

[tinkerpop] branch tp4 updated: figured out the hanging bug with RxJava. It doesn't like lots of merges and the repitition-based repeat model can hang. Dropped down the repititions for now, but will n

2019-04-04 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 3ff208d  figured out the hanging bug with RxJava. It doesn't like lots 
of merges and the repitition-based repeat model can hang. Dropped down the 
repititions for now, but will need to figure out how to do real looping in 
RxJava. Exposed more test cases.
3ff208d is described below

commit 3ff208d72afa9660c856b8e0449f20e453967ac4
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 4 09:09:07 2019 -0600

figured out the hanging bug with RxJava. It doesn't like lots of merges and 
the repitition-based repeat model can hang. Dropped down the repititions for 
now, but will need to figure out how to do real looping in RxJava. Exposed more 
test cases.
---
 .../org/apache/tinkerpop/machine/SimpleTestSuite.java   |  4 ++--
 .../tinkerpop/machine/processor/pipes/PipesTest.java| 17 -
 .../tinkerpop/machine/processor/rxjava/RxJava.java  | 11 ++-
 3 files changed, 8 insertions(+), 24 deletions(-)

diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
index 4b88855..b53577b 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
@@ -213,13 +213,13 @@ public class SimpleTestSuite extends 
AbstractTestSuite {
 g.inject(1L, 2L, 
3L).repeat(__.incr().flatMap(choose(is(lt(8L)), incr(.times(3));
 }
 
-//@Test
+@Test
 void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX1X() {
 verify(List.of(2L, 3L),
 g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(1));
 }
 
-//@Test
+@Test
 void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX2X() {
 verify(List.of(3L, 4L, 4L, 5L),
 g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(2));
diff --git 
a/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/PipesTest.java
 
b/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/PipesTest.java
index 1f0c37c..8d94250 100644
--- 
a/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/PipesTest.java
+++ 
b/java/machine/processor/pipes/src/test/java/org/apache/tinkerpop/machine/processor/pipes/PipesTest.java
@@ -24,7 +24,6 @@ import org.apache.tinkerpop.language.gremlin.TraversalSource;
 import org.apache.tinkerpop.language.gremlin.TraversalUtil;
 import org.apache.tinkerpop.language.gremlin.common.__;
 import org.apache.tinkerpop.machine.Machine;
-import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.apache.tinkerpop.machine.coefficient.LongCoefficient;
 import org.apache.tinkerpop.machine.species.LocalMachine;
 import org.apache.tinkerpop.machine.strategy.optimization.IdentityStrategy;
@@ -53,20 +52,4 @@ class PipesTest {
 System.out.println("\n--\n");
 }
 
-@Test
-void testOrder() {
-final Machine machine = LocalMachine.open();
-final TraversalSource g = Gremlin.traversal(machine)
-.withCoefficient(LongCoefficient.class)
-.withProcessor(PipesProcessor.class)
-.withStrategy(IdentityStrategy.class);
-
-Traversal traversal = g.inject(7L, 3L, 5L, 20L, 1L, 
2L).incr().order().by(Order.desc);
-System.out.println(TraversalUtil.getBytecode(traversal));
-System.out.println(traversal);
-System.out.println(TraversalUtil.getBytecode(traversal));
-System.out.println(traversal.toList());
-System.out.println("\n--\n");
-}
-
 }
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
index b7eac76..214004f 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
@@ -46,7 +46,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
  */
 public final class RxJava implements Processor {
 
-private static final int MAX_REPETITIONS = 15; // TODO: this needs to be a 
dynamic configuration
+private static final int MAX_REPETITIONS = 8; // TODO: this needs to be a 
dynamic configuration
 
 private final AtomicBoolean alive = new AtomicBoolean(Boolean.TRUE);
 private boolean executed = false;
@@ -127,12 +127,13 @@ public final class RxJava implements 
Process

[tinkerpop] branch tp4 updated: more test case. added flatMap() step. working through numerous nestings to try and expose why a repeat(union()) nest is not working for Pipes and RxJava. Other more com

2019-04-04 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 7595d5f  more test case. added flatMap() step. working through 
numerous nestings to try and expose why a repeat(union()) nest is not working 
for Pipes and RxJava. Other more complex nestings work fine. Still searching.
7595d5f is described below

commit 7595d5ff311154bfec22716ad2b4600bb660e7a9
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 4 08:23:51 2019 -0600

more test case. added flatMap() step. working through numerous nestings to 
try and expose why a repeat(union()) nest is not working for Pipes and RxJava. 
Other more complex nestings work fine. Still searching.
---
 .../tinkerpop/language/gremlin/Traversal.java  |  2 +
 .../language/gremlin/common/CommonTraversal.java   |  5 +++
 .../tinkerpop/language/gremlin/common/__.java  | 18 +++-
 .../language/gremlin/core/CoreTraversal.java   |  6 +++
 .../machine/bytecode/compiler/Argument.java|  3 ++
 .../bytecode/compiler/BytecodeArgument.java|  8 
 .../machine/bytecode/compiler/Compilation.java |  2 +
 .../bytecode/compiler/ConstantArgument.java|  8 
 .../machine/bytecode/compiler/MethodArgument.java  |  7 
 .../machine/function/flatmap/FlatMapFlatMap.java   | 10 ++---
 .../apache/tinkerpop/machine/SimpleTestSuite.java  | 48 +++---
 11 files changed, 96 insertions(+), 21 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index f917ee1..98b5d9a 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -61,6 +61,8 @@ public interface Traversal extends Iterator {
 
 public Traversal filter(final Traversal filterTraversal);
 
+public  Traversal flatMap(final Traversal 
flatMapTraversal);
+
 public Traversal> groupCount();
 
 public  Traversal> hasKey(final P predicate);
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index 0d7ff05..7d44169 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
@@ -129,6 +129,11 @@ public class CommonTraversal extends 
AbstractTraversal {
 }
 
 @Override
+public  Traversal flatMap(final Traversal 
flatMapTraversal) {
+return this.addInstruction(Symbols.FLATMAP, 
TraversalUtil.getBytecode(flatMapTraversal));
+}
+
+@Override
 public Traversal> groupCount() {
 return this.addInstruction(Symbols.GROUP_COUNT);
 }
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
index f678025..0bd98df 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
@@ -38,10 +38,22 @@ public class __ {
 return __.start().c(coefficient);
 }
 
+public static  Traversal choose(final Traversal 
predicate, final Traversal trueTraversal, final Traversal 
falseTraversal) {
+return __.start().choose(predicate, trueTraversal, 
falseTraversal);
+}
+
+public static  Traversal choose(final Traversal 
predicate, final Traversal trueTraversal) {
+return __.start().choose(predicate, trueTraversal);
+}
+
 public static  Traversal constant(final S constant) {
 return __.start().constant(constant);
 }
 
+public static  Traversal flatMap(final Traversal traversal) {
+return __.start().flatMap(traversal);
+}
+
 public static  Traversal identity() {
 return __.start().identity();
 }
@@ -66,6 +78,10 @@ public class __ {
 return __.start().loops();
 }
 
+public static  Traversal map(final Traversal 
traversal) {
+return __.start().map(traversal);
+}
+
 public static  Traversal sum() {
 return __.start().sum();
 }
@@ -74,7 +90,7 @@ public class __ {
 return __.start().count();
 }
 
-public static  Traversal union(final Traversal... traversals) {
+public static  Traversal union(final Traversal... traversals) {
 return __.start().union(traversals);
 }
 }
diff --git 
a/java/language/gremlin/src/main/java/o

[tinkerpop] branch tp4 updated: fixed a bug in RepeatStep in Pipes. Added nested repeat(union()) test case to SimpleTestSuite and only Beam passes. Commented out for now.

2019-04-04 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new abf9fd6  fixed a bug in RepeatStep in Pipes. Added nested 
repeat(union()) test case to SimpleTestSuite and only Beam passes. Commented 
out for now.
abf9fd6 is described below

commit abf9fd628aba7e6387727c2d793fd66be407e9bd
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 4 07:25:25 2019 -0600

fixed a bug in RepeatStep in Pipes. Added nested repeat(union()) test case 
to SimpleTestSuite and only Beam passes. Commented out for now.
---
 .../apache/tinkerpop/machine/SimpleTestSuite.java  | 28 +
 .../machine/processor/pipes/BranchStep.java|  6 +--
 .../machine/processor/pipes/RepeatStep.java| 47 --
 3 files changed, 48 insertions(+), 33 deletions(-)

diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
index 3404d4d..49bb764 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
@@ -199,16 +199,28 @@ public class SimpleTestSuite extends 
AbstractTestSuite {
 
 // NESTED REPEAT TESTING
 
-// @Test
-void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX3X() { // r.u.e
-verify(List.of(3L, 4L, 4L, 5L, 4L, 5L, 5L, 6L),
-g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(3));
+//@Test
+void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX1X() {
+verify(List.of(2L, 3L),
+g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(1));
+}
+
+//@Test
+void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX2X() {
+verify(List.of(3L, 4L, 4L, 5L),
+g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(2));
+}
 
-// 1
-// 1 2
-// 2 3 3 4
-// 3 4 4 5 4 5 5 6
+//@Test
+void g_injectX1X_repeatXunionXincr__incr_incrXX_timesX3X() {
+verify(List.of(4L, 5L, 5L, 6L, 5L, 6L, 6L, 7L),
+g.inject(1L).repeat(union(incr(), 
__.incr().incr())).times(3));
 }
+// 1
+// 1 2
+// 2 3 3 4
+// 3 4 4 5 4 5 5 6
+
 
 @Test
 void g_injectX7_3_5_20_1_2_5X_incr_order_byXdescX() {
diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
index 32ed3eb..0ef8280 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/BranchStep.java
@@ -57,7 +57,7 @@ final class BranchStep extends AbstractStep {
 return this.nextTraversers.next();
 }
 
-private final void stageOutput() {
+private void stageOutput() {
 while (!this.nextTraversers.hasNext() && this.previousStep.hasNext()) {
 boolean found = false;
 this.nextTraversers = new MultiIterator<>();
@@ -66,13 +66,13 @@ final class BranchStep extends AbstractStep {
 if (entry.getKey().filterTraverser(traverser)) {
 found = true;
 for (final Compilation branch : entry.getValue()) 
{
-((MultiIterator>) 
this.nextTraversers).addIterator(branch.addTraverser(traverser));
+((MultiIterator>) 
this.nextTraversers).addIterator(branch.addTraverser(traverser.clone()));
 }
 }
 }
 if (!found) {
 for (final Compilation defaultBranches : 
this.defaultBranches) {
-((MultiIterator>) 
this.nextTraversers).addIterator(defaultBranches.addTraverser(traverser));
+((MultiIterator>) 
this.nextTraversers).addIterator(defaultBranches.addTraverser(traverser.clone()));
 }
 }
 }
diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/RepeatStep.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/RepeatStep.java
index 830ad45..8dc25f3 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/RepeatStep.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/RepeatStep.java
@@ -63,33 +63,36 @@ final class RepeatStep extends AbstractStep {
 return this.outputTraversers.remove();
 }
 
-pri

[tinkerpop] branch tp4 updated: renamed ReduceFlow in RxJava to Reducer as its not a Flow.

2019-04-04 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4302a2f  renamed ReduceFlow in RxJava to Reducer as its not a Flow.
4302a2f is described below

commit 4302a2f6894520acd5615ca62a21e882186e48c7
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 4 06:40:14 2019 -0600

renamed ReduceFlow in RxJava to Reducer as its not a Flow.
---
 .../machine/processor/rxjava/{ReduceFlow.java => Reducer.java}| 4 ++--
 .../java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java| 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ReduceFlow.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/Reducer.java
similarity index 89%
rename from 
java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ReduceFlow.java
rename to 
java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/Reducer.java
index 145d22d..6aab534 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/ReduceFlow.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/Reducer.java
@@ -25,11 +25,11 @@ import org.apache.tinkerpop.machine.traverser.Traverser;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-final class ReduceFlow implements BiFunction, 
Traverser, Traverser> {
+final class Reducer implements BiFunction, 
Traverser, Traverser> {
 
 private final ReduceFunction function;
 
-ReduceFlow(final ReduceFunction function) {
+Reducer(final ReduceFunction function) {
 this.function = function;
 }
 
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
index e4a050e..b7eac76 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
@@ -120,7 +120,7 @@ public final class RxJava implements Processor {
 return Flowable.fromIterable(() -> 
IteratorUtils.map(((InitialFunction) function).get(), s -> 
traverserFactory.create(function, s)));
 } else if (function instanceof ReduceFunction) {
 final ReduceFunction reduceFunction = (ReduceFunction) function;
-return flow.reduce(traverserFactory.create(reduceFunction, 
reduceFunction.getInitialValue()), new 
ReduceFlow<>(reduceFunction)).toFlowable();
+return flow.reduce(traverserFactory.create(reduceFunction, 
reduceFunction.getInitialValue()), new Reducer<>(reduceFunction)).toFlowable();
 } else if (function instanceof BarrierFunction) {
 final BarrierFunction barrierFunction = 
(BarrierFunction) function;
 return flow.reduce(barrierFunction.getInitialValue(), new 
Barrier<>(barrierFunction)).toFlowable().flatMapIterable(new 
BarrierFlow<>(barrierFunction, traverserFactory));



[tinkerpop] branch tp4 updated: wrapped the TopologyUtil execution plan compilers from Beam and RxJava into the respective processors. Able to finalize and private lots of classes and methods. Added s

2019-04-04 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 1c09e4e  wrapped the TopologyUtil execution plan compilers from Beam 
and RxJava into the respective processors. Able to finalize and private lots of 
classes and methods. Added support for Barrier operations to RxJava. Added an 
order() test case to the SimpleTestSuite.
1c09e4e is described below

commit 1c09e4ec87936d03f3100dd80a91c3860a3617b1
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 4 06:37:09 2019 -0600

wrapped the TopologyUtil execution plan compilers from Beam and RxJava into 
the respective processors. Able to finalize and private lots of classes and 
methods. Added support for Barrier operations to RxJava. Added an order() test 
case to the SimpleTestSuite.
---
 .../tinkerpop/machine/AbstractTestSuite.java   |   7 +
 .../apache/tinkerpop/machine/SimpleTestSuite.java  |   7 +
 java/machine/processor/beam/pom.xml|   6 -
 .../machine/processor/beam/AbstractFn.java |   2 +-
 .../machine/processor/beam/BarrierFn.java  |   4 +-
 .../tinkerpop/machine/processor/beam/Beam.java | 119 +++-
 .../tinkerpop/machine/processor/beam/BranchFn.java |   4 +-
 .../tinkerpop/machine/processor/beam/FilterFn.java |   4 +-
 .../machine/processor/beam/FlatMapFn.java  |   4 +-
 .../tinkerpop/machine/processor/beam/Fn.java   |   2 +-
 .../machine/processor/beam/InitialFn.java  |   4 +-
 .../tinkerpop/machine/processor/beam/MapFn.java|   4 +-
 .../tinkerpop/machine/processor/beam/OutputFn.java |   2 +-
 .../tinkerpop/machine/processor/beam/ReduceFn.java |   6 +-
 .../machine/processor/beam/RepeatDeadEndFn.java|   6 +-
 .../machine/processor/beam/RepeatEndFn.java|   8 +-
 .../machine/processor/beam/RepeatStartFn.java  |  10 +-
 .../processor/beam/sideeffect/InMemoryBarrier.java |   2 +-
 .../machine/processor/beam/util/TopologyUtil.java  | 154 
 java/machine/processor/rxjava/pom.xml  |   6 -
 .../rxjava/{MapFlow.java => Barrier.java}  |  15 +-
 .../rxjava/{FlatMapFlow.java => BarrierFlow.java}  |  22 ++-
 .../machine/processor/rxjava/BranchFlow.java   |   4 +-
 .../machine/processor/rxjava/FilterFlow.java   |   4 +-
 .../machine/processor/rxjava/FlatMapFlow.java  |   4 +-
 .../machine/processor/rxjava/MapFlow.java  |   4 +-
 .../machine/processor/rxjava/ReduceFlow.java   |   5 +-
 .../tinkerpop/machine/processor/rxjava/RxJava.java | 129 -
 .../processor/rxjava/util/TopologyUtil.java| 155 -
 29 files changed, 320 insertions(+), 383 deletions(-)

diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
index fc07dc6..8524292 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/AbstractTestSuite.java
@@ -62,4 +62,11 @@ public abstract class AbstractTestSuite {
 assertFalse(traversal.hasNext());
 }
 
+static  void verifyOrder(final List expected, final 
Traversal traversal) {
+final List actual = traversal.toList();
+assertEquals(expected.size(), actual.size(), "Result set size differ: 
" + expected + " -- " + actual);
+assertEquals(expected, actual, "Result sets are not order equal: " + 
expected + " -- " + actual);
+assertFalse(traversal.hasNext());
+}
+
 }
diff --git 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
index 2cf143b..3404d4d 100644
--- 
a/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
+++ 
b/java/machine/machine-test/src/main/java/org/apache/tinkerpop/machine/SimpleTestSuite.java
@@ -21,6 +21,7 @@ package org.apache.tinkerpop.machine;
 import org.apache.tinkerpop.language.gremlin.P;
 import org.apache.tinkerpop.language.gremlin.common.__;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
+import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.junit.jupiter.api.Test;
 
 import java.util.List;
@@ -209,4 +210,10 @@ public class SimpleTestSuite extends 
AbstractTestSuite {
 // 3 4 4 5 4 5 5 6
 }
 
+@Test
+void g_injectX7_3_5_20_1_2_5X_incr_order_byXdescX() {
+verifyOrder(List.of(21L, 8L, 6L, 6L, 4L, 3L, 2L),
+g.inject(7L, 3L, 5L, 20L, 1L, 2L, 
5L).incr().order().by(Order.desc));
+}
+
 }
diff --git a/java/machine/processor/beam/pom.xml 
b/java/machine

[tinkerpop] branch tp4 updated: added machine-test package. we have SimpleTestSuite. The processors then have SimpleLocalTest and SimpleRemoteTest. This is too move forward with testing until a langua

2019-04-04 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4253b46  added machine-test package. we have SimpleTestSuite. The 
processors then have SimpleLocalTest and SimpleRemoteTest. This is too move 
forward with testing until a language agnostic test suite is developed.
4253b46 is described below

commit 4253b46a00a70d399b28d659cbd48dd750c09668
Author: Marko A. Rodriguez 
AuthorDate: Thu Apr 4 05:46:08 2019 -0600

added machine-test package. we have SimpleTestSuite. The processors then 
have SimpleLocalTest and SimpleRemoteTest. This is too move forward with 
testing until a language agnostic test suite is developed.
---
 .../language/gremlin/TraversalSource.java  |   2 +-
 .../tinkerpop/language/gremlin/TraversalUtil.java  |   4 +
 .../tinkerpop/language/gremlin/common/__.java  |   9 +
 java/machine/machine-core/pom.xml  |   7 -
 .../{processor/rxjava => machine-test}/pom.xml |  25 ++-
 .../tinkerpop/machine/AbstractTestSuite.java   |  65 +++
 .../apache/tinkerpop/machine/SimpleTestSuite.java  | 212 +
 java/machine/pom.xml   |   1 +
 java/machine/processor/beam/pom.xml|   2 +-
 .../tinkerpop/machine/processor/beam/BranchFn.java |  14 +-
 .../machine/processor/beam/RepeatEndFn.java|   9 +-
 .../machine/processor/beam/RepeatStartFn.java  |   3 +-
 .../machine/processor/beam/util/TopologyUtil.java  |   8 +
 .../tinkerpop/machine/processor/beam/BeamTest.java | 156 ---
 .../machine/processor/beam/SimpleLocalTest.java|  41 
 .../machine/processor/beam/SimpleRemoteTest.java   |  49 +
 java/machine/processor/pipes/pom.xml   |   4 +-
 .../machine/processor/pipes/RepeatStep.java|  10 +-
 .../machine/processor/pipes/PipesTest.java | 126 +---
 .../machine/processor/pipes/SimpleLocalTest.java   |  41 
 .../machine/processor/pipes/SimpleRemoteTest.java  |  49 +
 java/machine/processor/rxjava/pom.xml  |   2 +-
 .../processor/rxjava/util/TopologyUtil.java|   9 +-
 .../machine/processor/rxjava/RxJavaTest.java   | 145 --
 .../machine/processor/rxjava/SimpleLocalTest.java  |  41 
 .../machine/processor/rxjava/SimpleRemoteTest.java |  49 +
 26 files changed, 610 insertions(+), 473 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index 37a57f5..eb17866 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
@@ -41,7 +41,7 @@ import java.util.Map;
 public class TraversalSource implements Cloneable {
 
 private final Machine machine;
-private Bytecode bytecode;
+protected Bytecode bytecode;
 private Coefficient coefficient = (Coefficient) 
LongCoefficient.create();
 private boolean registered = false;
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
index cc1c2d4..efdf824 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
@@ -36,6 +36,10 @@ public final class TraversalUtil {
 return ((AbstractTraversal) traversal).bytecode;
 }
 
+public static  Bytecode getBytecode(final TraversalSource 
traversalSource) {
+return traversalSource.bytecode;
+}
+
 public static Object tryToGetBytecode(final Object object) {
 return object instanceof AbstractTraversal ? ((AbstractTraversal) 
object).bytecode : object;
 }
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
index 3361a5c..f678025 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.language.gremlin.common;
 
+import org.apache.tinkerpop.language.gremlin.P;
 import org.apache.tinkerpop.language.gremlin.Traversal;
 
 /**
@@ -53,6 +54,10 @@ public class __ {
 return __.start().is(object);
 }
 
+public static  Traversal is(final P predicate) {
+return __.start().is(predicate);
+}
+
 public static  Traver

[tinkerpop] branch tp4 updated: minor clean up. done for the day. parallel() will have to wait. have it about 75% working.

2019-04-03 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new a3e2023  minor clean up. done for the day. parallel() will have to 
wait. have it about 75% working.
a3e2023 is described below

commit a3e2023279bbea1e059d74e18fa6376ea99c9c26
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 3 15:22:34 2019 -0600

minor clean up. done for the day. parallel() will have to wait. have it 
about 75% working.
---
 .../main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
index cd81390..b665b41 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/RxJava.java
@@ -69,7 +69,7 @@ public final class RxJava implements Processor {
 private void prepareFlow() {
 if (!this.executed) {
 this.executed = true;
-TopologyUtil.compile(Flowable.fromIterable(this.starts), 
compilation).
+TopologyUtil.compile(Flowable.fromIterable(this.starts), 
this.compilation).
 doOnNext(this.ends::add).
 doOnComplete(() -> this.alive.set(Boolean.FALSE)).
 subscribe();



[tinkerpop] branch tp4 updated: was able to use TopologyUtil.compile() instead of Flowable.publish() to chain flows in rxJava.

2019-04-03 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new b46ffb1  was able to use TopologyUtil.compile() instead of 
Flowable.publish() to chain flows in rxJava.
b46ffb1 is described below

commit b46ffb10a8b2a7c0f4107900c822023159826faf
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 3 15:07:56 2019 -0600

was able to use TopologyUtil.compile() instead of Flowable.publish() to 
chain flows in rxJava.
---
 .../tinkerpop/machine/processor/rxjava/util/TopologyUtil.java| 9 -
 1 file changed, 4 insertions(+), 5 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
index a9bee79..71b95bd 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
@@ -77,11 +77,10 @@ public final class TopologyUtil {
 for (int i = 0; i < branches.getValue().size(); i++) {
 final Compilation branch = 
branches.getValue().get(i);
 final int id = i;
-branchFlows.add(
-selectorFlow.
+branchFlows.add(compile(selectorFlow.
 filter(list -> list.get(0).equals(null == 
branches.getKey() ? -1 : id)).
-map(list -> (Traverser) list.get(1)).
-publish(f -> compile(f, branch)));
+map(list -> (Traverser) list.get(1)),
+branch));
 }
 }
 Flowable> sink = (Flowable) flow.filter(t -> 
false); // branches are the only outputs
@@ -117,7 +116,7 @@ public final class TopologyUtil {
 return list;
 });
 outputs.add(selectorFlow.filter(list -> 
list.get(0).equals(0)).map(list -> (Traverser) list.get(1)));
-flow = selectorFlow.filter(list -> 
list.get(0).equals(1)).map(list -> (Traverser) list.get(1)).publish(f -> 
compile(f, repeatBranch.getRepeat()));
+flow = compile(selectorFlow.filter(list -> 
list.get(0).equals(1)).map(list -> (Traverser) list.get(1)), 
repeatBranch.getRepeat());
 selectorFlow = flow.flatMapIterable(t -> {
 final List list = new ArrayList<>();
 if (repeatBranch.hasEndPredicates()) {



[tinkerpop] branch tp4 updated: fixed a bug where repeat loops weren't being updated in RxJava.

2019-04-03 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new b01672b  fixed a bug where repeat loops weren't being updated in 
RxJava.
b01672b is described below

commit b01672b067195df27f17755ebb56d0d16e00e8d5
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 3 15:04:14 2019 -0600

fixed a bug where repeat loops weren't being updated in RxJava.
---
 .../tinkerpop/machine/processor/rxjava/util/TopologyUtil.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
index 4146623..a9bee79 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
@@ -126,16 +126,16 @@ public final class TopologyUtil {
 list.add(List.of(0, 
t.repeatDone(repeatBranch)));
 } else if (4 == repeatBranch.getEmitLocation() && 
repeatBranch.getEmit().filterTraverser(t)) {
 list.add(List.of(0, 
t.repeatDone(repeatBranch)));
-list.add(List.of(1, t));
+list.add(List.of(1, 
t.repeatLoop(repeatBranch)));
 } else
-list.add(List.of(1, t));
+list.add(List.of(1, 
t.repeatLoop(repeatBranch)));
 } else if (3 == repeatBranch.getEmitLocation()) {
 if (repeatBranch.getEmit().filterTraverser(t))
 list.add(List.of(0, 
t.repeatDone(repeatBranch)));
 if (4 == repeatBranch.getUntilLocation() && 
repeatBranch.getUntil().filterTraverser(t))
 list.add(List.of(0, 
t.repeatDone(repeatBranch)));
 else
-list.add(List.of(1, t));
+list.add(List.of(1, 
t.repeatLoop(repeatBranch)));
 }
 } else
 list.add(List.of(1, t.repeatLoop(repeatBranch)));



[tinkerpop] branch tp4 updated: got repeat() working.

2019-04-03 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new c7ee5a0  got repeat() working.
c7ee5a0 is described below

commit c7ee5a0c93f8f54509a2c82971ae6c108f2caec8
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 3 13:36:10 2019 -0600

got repeat() working.
---
 .../processor/rxjava/util/TopologyUtil.java| 91 +++---
 .../machine/processor/rxjava/RxJavaTest.java   | 83 +++-
 2 files changed, 143 insertions(+), 31 deletions(-)

diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
index b6b0e4a..4146623 100644
--- 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
+++ 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/util/TopologyUtil.java
@@ -27,6 +27,7 @@ import org.apache.tinkerpop.machine.function.FlatMapFunction;
 import org.apache.tinkerpop.machine.function.InitialFunction;
 import org.apache.tinkerpop.machine.function.MapFunction;
 import org.apache.tinkerpop.machine.function.ReduceFunction;
+import org.apache.tinkerpop.machine.function.branch.RepeatBranch;
 import org.apache.tinkerpop.machine.processor.rxjava.BranchFlow;
 import org.apache.tinkerpop.machine.processor.rxjava.FilterFlow;
 import org.apache.tinkerpop.machine.processor.rxjava.FlatMapFlow;
@@ -46,6 +47,8 @@ import java.util.Map;
  */
 public final class TopologyUtil {
 
+private static final int MAX_ITERATIONS = 20;
+
 public static  Flowable> compile(final 
Flowable> source, final Compilation compilation) {
 final TraverserFactory traverserFactory = 
compilation.getTraverserFactory();
 Flowable> sink = (Flowable) source;
@@ -55,34 +58,6 @@ public final class TopologyUtil {
 return sink;
 }
 
-/*
- private final void stageInput() {
-if (this.hasStartPredicates) {
-final Traverser traverser = this.inputTraversers.isEmpty() ? 
this.previousStep.next() : this.inputTraversers.remove();
-if (1 == this.untilLocation) {
-if (this.untilCompilation.filterTraverser(traverser)) {
-this.outputTraversers.add(traverser);
-} else if (2 == this.emitLocation && 
this.emitCompilation.filterTraverser(traverser)) {
-
this.outputTraversers.add(traverser.repeatDone(this.repeatBranch));
-this.repeat.addTraverser(traverser);
-} else
-this.repeat.addTraverser(traverser);
-} else if (1 == this.emitLocation) {
-if (this.emitCompilation.filterTraverser(traverser))
-
this.outputTraversers.add(traverser.repeatDone(this.repeatBranch));
-if (2 == this.untilLocation && 
this.untilCompilation.filterTraverser(traverser))
-
this.outputTraversers.add(traverser.repeatDone(this.repeatBranch));
-else
-this.repeat.addTraverser(traverser);
-}
-} else {
-this.repeat.addTraverser(this.inputTraversers.isEmpty() ? 
this.previousStep.next() : this.inputTraversers.remove());
-}
-}
-
-
- */
-
 private static  Flowable> 
extend(Flowable> flow, final CFunction function, final 
TraverserFactory traverserFactory) {
 if (function instanceof MapFunction)
 return flow.map(new MapFlow<>((MapFunction) function));
@@ -114,6 +89,66 @@ public final class TopologyUtil {
 sink = sink.mergeWith(branchFlow);
 }
 return sink;
+} else if (function instanceof RepeatBranch) {
+final RepeatBranch repeatBranch = (RepeatBranch) 
function;
+final List>> outputs = new ArrayList<>();
+for (int i = 0; i < MAX_ITERATIONS; i++) {
+Flowable selectorFlow = flow.flatMapIterable(t -> {
+final List list = new ArrayList<>();
+if (repeatBranch.hasStartPredicates()) {
+if (1 == repeatBranch.getUntilLocation()) {
+if (repeatBranch.getUntil().filterTraverser(t)) {
+list.add(List.of(0, 
t.repeatDone(repeatBranch)));
+} else if (2 == repeatBranch.getEmitLocation() && 
repeatBranch.getEmit().filterTraverser(t)) {
+list.add(List.of(1, t));
+list.add(List.of(0, 
t.repeatDone(repeatBranch)));
+

[tinkerpop] branch tp4 updated: first stub at RxJavaProcessor. everything works except for repeat(). I'm all confused and lost. Need to think.

2019-04-03 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 414211a  first stub at RxJavaProcessor. everything works except for 
repeat(). I'm all confused and lost. Need to think.
414211a is described below

commit 414211abde8d2f5f257b67f50ff4b192655db104
Author: Marko A. Rodriguez 
AuthorDate: Wed Apr 3 12:49:50 2019 -0600

first stub at RxJavaProcessor. everything works except for repeat(). I'm 
all confused and lost. Need to think.
---
 .../machine/function/branch/BranchBranch.java  |   3 +-
 java/machine/processor/pom.xml |   1 +
 java/machine/processor/rxjava/pom.xml  |  71 
 .../machine/processor/rxjava/BranchFlow.java   |  51 +
 .../machine/processor/rxjava/FilterFlow.java   |  40 +++
 .../machine/processor/rxjava/FlatMapFlow.java  |  40 +++
 .../machine/processor/rxjava/MapFlow.java  |  40 +++
 .../machine/processor/rxjava/ReduceFlow.java   |  41 +++
 .../tinkerpop/machine/processor/rxjava/RxJava.java |  84 +++
 .../machine/processor/rxjava/RxJavaProcessor.java  |  42 
 .../processor/rxjava/strategy/RxJavaStrategy.java  |  38 +++
 .../processor/rxjava/util/TopologyUtil.java| 120 +
 .../machine/processor/rxjava/RxJavaTest.java   |  68 
 13 files changed, 638 insertions(+), 1 deletion(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
index 13f182c..165a87d 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
@@ -27,6 +27,7 @@ import org.apache.tinkerpop.machine.function.BranchFunction;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -68,7 +69,7 @@ public final class BranchBranch extends 
AbstractFunction implements
 
 public static  BranchBranch compile(final Instruction 
instruction) {
 final Object[] args = instruction.args();
-final Map, List>> branches = 
new HashMap<>();
+final Map, List>> branches = 
new LinkedHashMap<>();
 for (int i = 0; i < args.length; i = i + 2) {
 final Compilation predicate = 
Symbols.DEFAULT.equals(args[i]) ? null : Compilation.compile(args[i]);
 final Compilation branch = Compilation.compile(args[i + 
1]);
diff --git a/java/machine/processor/pom.xml b/java/machine/processor/pom.xml
index f23513c..6a34820 100644
--- a/java/machine/processor/pom.xml
+++ b/java/machine/processor/pom.xml
@@ -27,5 +27,6 @@ limitations under the License.
 
 pipes
 beam
+rxjava
 
 
\ No newline at end of file
diff --git a/java/machine/processor/rxjava/pom.xml 
b/java/machine/processor/rxjava/pom.xml
new file mode 100644
index 000..d41d44f
--- /dev/null
+++ b/java/machine/processor/rxjava/pom.xml
@@ -0,0 +1,71 @@
+
+http://maven.apache.org/POM/4.0.0;
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
+4.0.0
+
+processor
+org.apache.tinkerpop
+4.0.0-SNAPSHOT
+
+Apache TinkerPop :: Machine :: RxJava
+rxjava
+
+
+org.apache.tinkerpop
+machine-core
+${project.version}
+
+
+io.reactivex.rxjava2
+rxjava
+2.2.8
+
+
+
+org.apache.tinkerpop
+gremlin
+${project.version}
+test
+
+
+org.apache.tinkerpop
+blueprints
+${project.version}
+test
+
+
+
+${basedir}/target
+${project.artifactId}-${project.version}
+
+
+${basedir}/src/test/resources
+
+
+
+
+
+org.apache.maven.plugins
+maven-surefire-plugin
+
+
+org.apache.maven.plugins
+maven-failsafe-plugin
+
+
+
+
\ No newline at end of file
diff --git 
a/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/BranchFlow.java
 
b/java/machine/processor/rxjava/src/main/java/org/apache/tinkerpop/machine/processor/rxjava/BranchFlow.java
new file mode 100644
index 000..5

[tinkerpop] branch tp4 updated: cleanup on Beam and order().by() instruction.

2019-03-28 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 22b7903  cleanup on Beam and order().by() instruction.
22b7903 is described below

commit 22b7903eb4f79e6bd172f33ef0de117c8f958566
Author: Marko A. Rodriguez 
AuthorDate: Thu Mar 28 11:06:43 2019 -0600

cleanup on Beam and order().by() instruction.
---
 .../tinkerpop/machine/function/barrier/OrderBarrier.java |  5 +
 .../org/apache/tinkerpop/machine/util/MultiComparator.java   | 12 
 .../apache/tinkerpop/machine/processor/beam/BarrierFn.java   |  8 
 .../tinkerpop/machine/processor/beam/util/TopologyUtil.java  |  8 
 4 files changed, 25 insertions(+), 8 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/OrderBarrier.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/OrderBarrier.java
index e7d8b66..ed81393 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/OrderBarrier.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/OrderBarrier.java
@@ -84,6 +84,11 @@ public final class OrderBarrier extends 
AbstractFunction implements
 return traverserSet;
 }
 
+@Override
+public int hashCode() {
+return super.hashCode() ^ this.comparator.hashCode() ^ 
this.compilationCircle.hashCode();
+}
+
 public static  OrderBarrier compile(final Instruction 
instruction) {
 final List> compilations = new ArrayList<>();
 final List> comparators = new ArrayList<>();
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/MultiComparator.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/MultiComparator.java
index 02f7a20..4135fae 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/MultiComparator.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/MultiComparator.java
@@ -61,6 +61,18 @@ public final class MultiComparator implements 
Comparator, Serializable {
 }
 }
 
+@Override
+public int hashCode() {
+return this.comparators.hashCode();
+}
+
+@Override
+public boolean equals(final Object object) {
+return object instanceof MultiComparator &&
+this.comparators.equals(((MultiComparator) 
object).comparators) &&
+this.isShuffle == ((MultiComparator) object).isShuffle;
+}
+
 public boolean isShuffle() {
 return this.isShuffle;
 }
diff --git 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/BarrierFn.java
 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/BarrierFn.java
index d736c60..742db4f 100644
--- 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/BarrierFn.java
+++ 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/BarrierFn.java
@@ -80,7 +80,7 @@ public class BarrierFn extends 
Combine.CombineFn, In
 return (Coder) new TraverserSetCoder();
 }
 
-public static class BarrierIterateFn extends DoFn> {
+public static class BarrierIterateFn extends DoFn> {
 
 private final BarrierFunction barrierFunction;
 private final TraverserFactory traverserFactory;
@@ -92,10 +92,10 @@ public class BarrierFn extends 
Combine.CombineFn, In
 }
 
 @ProcessElement
-public void processElement(final @Element B barrier, final 
OutputReceiver> output) {
+public void processElement(final @Element B barrier, final 
OutputReceiver> output) {
 final B local = 
this.barrierFunction.merge(this.barrierFunction.getInitialValue(), barrier);
-final Iterator> iterator = 
this.barrierFunction.returnsTraversers() ?
-(Iterator>) 
this.barrierFunction.createIterator(local) :
+final Iterator> iterator = 
this.barrierFunction.returnsTraversers() ?
+(Iterator>) 
this.barrierFunction.createIterator(local) :
 
IteratorUtils.map(this.barrierFunction.createIterator(local),
 e -> 
this.traverserFactory.create(this.barrierFunction, e));
 while (iterator.hasNext())
diff --git 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/TopologyUtil.java
 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/TopologyUtil.java
index aa67878..4f27ae7 100644
--- 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/mach

[tinkerpop] branch tp4 updated: Generalized order() to support arbitrary number of asc and desc by()s. Borrowed many ideas from TP3, but greatly simplified the implementation as there is no GraphCompu

2019-03-28 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new e1f13b3  Generalized order() to support arbitrary number of asc and 
desc by()s. Borrowed many ideas from TP3, but greatly simplified the 
implementation as there is no GraphComputer concepts to worry about.
e1f13b3 is described below

commit e1f13b3956c4ac0a060a55308e1f73dd5369cf2d
Author: Marko A. Rodriguez 
AuthorDate: Thu Mar 28 09:55:22 2019 -0600

Generalized order() to support arbitrary number of asc and desc by()s. 
Borrowed many ideas from TP3, but greatly simplified the implementation as 
there is no GraphComputer concepts to worry about.
---
 .../tinkerpop/language/gremlin/Traversal.java  |   7 +-
 .../language/gremlin/common/CommonTraversal.java   |  16 +++-
 .../tinkerpop/language/gremlin/common/__.java  |   4 +
 .../language/gremlin/core/CoreTraversal.java   |  17 +++-
 .../bytecode/compiler/CompilationCircle.java   |  17 +++-
 .../tinkerpop/machine/bytecode/compiler/Order.java |  85 +
 .../machine/function/barrier/OrderBarrier.java |  48 --
 .../tinkerpop/machine/function/map/PathMap.java|   4 +-
 .../traverser/species/ProjectedTraverser.java  | 103 +
 .../tinkerpop/machine/util/MultiComparator.java|  74 +++
 .../tinkerpop/machine/processor/beam/BeamTest.java |   3 +-
 .../machine/processor/pipes/PipesTest.java |   3 +-
 12 files changed, 363 insertions(+), 18 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index b8d478c..f917ee1 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.language.gremlin;
 
+import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.apache.tinkerpop.machine.structure.data.TMap;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.path.Path;
@@ -34,9 +35,13 @@ public interface Traversal extends Iterator {
 
 public Traversal barrier();
 
+public Traversal by(final String byString);
+
 public Traversal by(final Traversal byTraversal);
 
-public Traversal by(final String byString);
+public Traversal by(final Order order);
+
+public Traversal by(final Traversal byTraversal, final 
Order order);
 
 public Traversal c(final C coefficient);
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index d6adf91..0d7ff05 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
@@ -25,6 +25,7 @@ import org.apache.tinkerpop.language.gremlin.TraversalUtil;
 import org.apache.tinkerpop.machine.Machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.compiler.CommonCompiler.Symbols;
+import org.apache.tinkerpop.machine.bytecode.compiler.Order;
 import org.apache.tinkerpop.machine.bytecode.compiler.Pred;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.coefficient.LongCoefficient;
@@ -59,14 +60,25 @@ public class CommonTraversal extends 
AbstractTraversal {
 }
 
 @Override
+public Traversal by(final String byString) {
+this.bytecode.lastInstruction().addArg(byString);
+return this;
+}
+
+@Override
 public Traversal by(final Traversal byTraversal) {
 
this.bytecode.lastInstruction().addArg(TraversalUtil.getBytecode(byTraversal));
 return this;
 }
 
 @Override
-public Traversal by(final String byString) {
-this.bytecode.lastInstruction().addArg(byString);
+public Traversal by(final Order order) {
+return this.by(__.identity(), order);
+}
+
+@Override
+public Traversal by(final Traversal byTraversal, final 
Order order) {
+
this.bytecode.lastInstruction().addArgs(TraversalUtil.getBytecode(byTraversal), 
order.name());
 return this;
 }
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/__.java
index 8d7aef4..3361a5c 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common

[tinkerpop] branch tp4 updated: Got order() working in both Pipes and Beam. It was a bit crazy in beam as it requires a reduce and unfold. Fortunately, the BarrierFunction in machine-core was sufficie

2019-03-28 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new a948102  Got order() working in both Pipes and Beam. It was a bit 
crazy in beam as it requires a reduce and unfold. Fortunately, the 
BarrierFunction in machine-core was sufficiently spec'd for the job. This gives 
me confidence that all barriers will come naturally from here.
a948102 is described below

commit a9481022e162656716f03516fcdf56a77722c456
Author: Marko A. Rodriguez 
AuthorDate: Thu Mar 28 08:40:31 2019 -0600

Got order() working in both Pipes and Beam. It was a bit crazy in beam as 
it requires a reduce and unfold. Fortunately, the BarrierFunction in 
machine-core was sufficiently spec'd for the job. This gives me confidence that 
all barriers will come naturally from here.
---
 .../tinkerpop/language/gremlin/Traversal.java  |   2 +
 .../language/gremlin/TraversalSource.java  |   4 +-
 .../language/gremlin/common/CommonTraversal.java   |   5 +
 .../language/gremlin/core/CoreTraversal.java   |   5 +
 .../machine/bytecode/compiler/CommonCompiler.java  |   5 +
 .../machine/function/BarrierFunction.java  |   2 +
 .../machine/function/barrier/JoinBarrier.java  |   6 ++
 .../{StallBarrier.java => OrderBarrier.java}   |  32 ---
 .../machine/function/barrier/StallBarrier.java |   7 +-
 .../machine/function/flatmap/UnfoldFlatMap.java|   2 +-
 .../machine/processor/beam/BarrierFn.java  | 105 +
 .../machine/processor/beam/io/BarrierCoder.java|  63 +
 .../processor/beam/io/TraverserSetCoder.java   |  63 +
 .../processor/beam/sideeffect/InMemoryBarrier.java |  59 
 .../processor/beam/sideeffect/InMemoryReducer.java |   1 -
 .../machine/processor/beam/util/TopologyUtil.java  |   7 ++
 .../tinkerpop/machine/processor/beam/BeamTest.java |  17 
 .../machine/processor/pipes/BarrierStep.java   |   2 +-
 .../machine/processor/pipes/PipesTest.java |  16 
 19 files changed, 383 insertions(+), 20 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index a28e626..b8d478c 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -86,6 +86,8 @@ public interface Traversal extends Iterator {
 
 public  Traversal map(final Traversal mapTraversal);
 
+public Traversal order();
+
 public Traversal path(final String... labels);
 
 public Traversal repeat(final Traversal repeatTraversal);
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index e8c0bd1..37a57f5 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
@@ -56,7 +56,7 @@ public class TraversalSource implements Cloneable {
 public TraversalSource withCoefficient(final Class> coefficient) {
 final TraversalSource clone = this.clone();
 clone.bytecode.addUniqueSourceInstruction(Symbols.WITH_COEFFICIENT, 
coefficient);
-clone.coefficient = BytecodeUtil.getCoefficient(clone.bytecode).get();
+clone.coefficient = BytecodeUtil.getCoefficient(clone.bytecode).get(); 
// previously line guarantees existence
 return clone;
 }
 
@@ -104,7 +104,7 @@ public class TraversalSource implements Cloneable {
 
 //
 
-private final void prepareSourceCode() {
+private void prepareSourceCode() {
 if (!this.registered) {
 this.registered = true;
 this.bytecode = this.machine.register(this.bytecode);
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index 738d1be..d6adf91 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
@@ -192,6 +192,11 @@ public class CommonTraversal extends 
AbstractTraversal {
 }
 
 @Override
+public Traversal order() {
+return this.addInstruction(Symbols.ORDER);
+}
+
+@Override
 public Traversal path(final String... labels) {
 return this.addInstruction(Symbols.PATH, 
TraversalUtil.addO

[tinkerpop] branch tp4 updated: decided to use TP3 .equals() method for functions developed by @dkuppitz for steps.

2019-03-28 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4c09b58  decided to use TP3 .equals() method for functions developed 
by @dkuppitz for steps.
4c09b58 is described below

commit 4c09b58d262761269c890a4f443aece9ee2893f5
Author: Marko A. Rodriguez 
AuthorDate: Thu Mar 28 06:35:44 2019 -0600

decided to use TP3 .equals() method for functions developed by @dkuppitz 
for steps.
---
 .../apache/tinkerpop/machine/function/AbstractFunction.java   |  9 +++--
 .../tinkerpop/machine/function/branch/BranchBranch.java   |  5 -
 .../tinkerpop/machine/function/branch/RepeatBranch.java   | 11 ---
 .../tinkerpop/machine/function/filter/FilterFilter.java   |  8 
 .../tinkerpop/machine/function/filter/HasKeyFilter.java   |  8 
 .../tinkerpop/machine/function/filter/HasKeyValueFilter.java  |  8 
 .../apache/tinkerpop/machine/function/filter/IsFilter.java|  8 
 .../tinkerpop/machine/function/flatmap/FlatMapFlatMap.java|  7 ---
 .../tinkerpop/machine/function/initial/InitialInitial.java|  7 ---
 .../apache/tinkerpop/machine/function/map/ConstantMap.java|  7 ---
 .../org/apache/tinkerpop/machine/function/map/MapMap.java |  7 ---
 .../org/apache/tinkerpop/machine/function/map/PathMap.java|  8 
 .../org/apache/tinkerpop/machine/function/map/ValueMap.java   |  7 ---
 .../tinkerpop/machine/function/reduce/GroupCountReduce.java   |  7 ---
 .../org/apache/tinkerpop/machine/traverser/TraverserSet.java  | 10 +++---
 15 files changed, 10 insertions(+), 107 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/AbstractFunction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/AbstractFunction.java
index 97c6bd1..694cf78 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/AbstractFunction.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/AbstractFunction.java
@@ -51,15 +51,12 @@ public abstract class AbstractFunction implements 
CFunction {
 
 @Override
 public int hashCode() {
-return this.coefficient.hashCode() ^ (null == this.label ? 1 : 
this.label.hashCode());
+return this.getClass().hashCode() ^ this.coefficient.hashCode() ^ 
(null == this.label ? 1 : this.label.hashCode());
 }
 
 @Override
-public boolean equals(final Object object) {
-return object instanceof AbstractFunction &&
-this.coefficient.equals(((AbstractFunction) 
object).coefficient) &&
-((null == this.label && null == ((AbstractFunction) 
object).label) ||
-(null != this.label && 
this.label.equals(((AbstractFunction) object).label)));
+public boolean equals(final Object other) {
+return other != null && other.getClass().equals(this.getClass()) && 
this.hashCode() == other.hashCode();
 }
 
 @Override
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
index fbf86e3..13f182c 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
@@ -53,11 +53,6 @@ public final class BranchBranch extends 
AbstractFunction implements
 }
 
 @Override
-public boolean equals(final Object object) {
-return object instanceof BranchBranch && 
this.branches.equals(((BranchBranch) object).branches) && super.equals(object);
-}
-
-@Override
 public BranchBranch clone() {
 final BranchBranch clone = (BranchBranch) 
super.clone();
 clone.branches = new HashMap<>(this.branches.size());
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
index b49..eddb43b 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/RepeatBranch.java
@@ -109,17 +109,6 @@ public final class RepeatBranch extends 
AbstractFunction {
 }
 
 @Override
-public boolean equals(final Object object) {
-return object instanceof RepeatBranch &&
-this.repeatCompilation.equals(((RepeatBranch) 
object).repeatCompil

[tinkerpop] branch tp4 updated: Compilations can now be compared and cloned -- equals, clone, hashcode. This entailed clone, equals, and hashcode for Arguments and CFunctions. We can now cache Compila

2019-03-27 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 37e596a  Compilations can now be compared and cloned -- equals, clone, 
hashcode. This entailed clone, equals, and hashcode for Arguments and 
CFunctions. We can now cache Compilations as well as split them apart for 
processors that want to thread components.
37e596a is described below

commit 37e596a972a446ec4211b7fece432f9a717c383c
Author: Marko A. Rodriguez 
AuthorDate: Wed Mar 27 12:30:17 2019 -0600

Compilations can now be compared and cloned -- equals, clone, hashcode. 
This entailed clone, equals, and hashcode for Arguments and CFunctions. We can 
now cache Compilations as well as split them apart for processors that want to 
thread components.
---
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  4 +-
 .../machine/bytecode/compiler/Argument.java|  4 +-
 .../bytecode/compiler/BytecodeArgument.java| 23 +++-
 .../machine/bytecode/compiler/Compilation.java | 13 +++--
 .../bytecode/compiler/CompilationCircle.java   | 29 +-
 .../bytecode/compiler/ConstantArgument.java| 20 +++
 .../machine/bytecode/compiler/MethodArgument.java  | 17 ++
 .../bytecode/compiler/SourceCompilation.java   | 15 ++
 .../machine/function/AbstractFunction.java | 24 +
 .../tinkerpop/machine/function/CFunction.java  |  7 +--
 .../machine/function/branch/BranchBranch.java  | 27 +-
 .../machine/function/branch/RepeatBranch.java  | 35 +---
 .../machine/function/filter/FilterFilter.java  | 34 +---
 .../machine/function/filter/HasKeyFilter.java  | 23 +++-
 .../machine/function/filter/HasKeyValueFilter.java | 28 --
 .../machine/function/filter/IsFilter.java  | 24 +++--
 .../machine/function/flatmap/FlatMapFlatMap.java   | 22 +++-
 .../machine/function/initial/InitialInitial.java   | 14 -
 .../machine/function/map/ConstantMap.java  | 12 +
 .../tinkerpop/machine/function/map/MapMap.java | 23 ++--
 .../tinkerpop/machine/function/map/PathMap.java| 23 +++-
 .../tinkerpop/machine/function/map/ValueMap.java   | 23 ++--
 .../machine/function/reduce/GroupCountReduce.java  | 22 +++-
 .../machine/bytecode/compiler/CompilationTest.java | 62 ++
 24 files changed, 476 insertions(+), 52 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index aa6bca6..5272ce7 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -19,7 +19,9 @@
 package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.bytecode.compiler.BytecodeCompiler;
+import org.apache.tinkerpop.machine.bytecode.compiler.CommonCompiler;
 import org.apache.tinkerpop.machine.bytecode.compiler.CompositeCompiler;
+import org.apache.tinkerpop.machine.bytecode.compiler.CoreCompiler;
 import org.apache.tinkerpop.machine.bytecode.compiler.CoreCompiler.Symbols;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
@@ -86,7 +88,7 @@ public final class BytecodeUtil {
 final List compilers = new ArrayList<>();
 BytecodeUtil.getProcessorFactory(bytecode).ifPresent(f -> 
compilers.addAll(f.getCompilers()));
 BytecodeUtil.getStructureFactory(bytecode).ifPresent(f -> 
compilers.addAll(f.getCompilers()));
-return CompositeCompiler.create(compilers);
+return CompositeCompiler.create(compilers.isEmpty() ? 
List.of(CoreCompiler.instance(), CommonCompiler.instance()) : compilers);
 }
 
 public static  Optional> getCoefficient(final 
Bytecode bytecode) {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Argument.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Argument.java
index 495d9a2..f55edc0 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Argument.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Argument.java
@@ -27,7 +27,7 @@ import java.util.Arrays;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public interface Argument extends Serializable {
+public interface Argument extends Serializable, Cloneable {
 
 public  E mapArg(final Traverser traverser);
 
@@ -42,4 +42,6 @@ public interface Argument extends Serializable {
 return new C

[tinkerpop] branch tp4 updated: Staging Comilation for the ability to be cached -- hashCode, equals... and ultimately clone. Need a way to deep clone functions as they can contain compilations.

2019-03-27 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 6c60bcc  Staging Comilation for the ability to be cached -- hashCode, 
equals... and ultimately clone. Need a way to deep clone functions as they can 
contain compilations.
6c60bcc is described below

commit 6c60bcc7dc8ce2060b37051688845773a318f599
Author: Marko A. Rodriguez 
AuthorDate: Wed Mar 27 07:40:47 2019 -0600

Staging Comilation for the ability to be cached -- hashCode, equals... and 
ultimately clone. Need a way to deep clone functions as they can contain 
compilations.
---
 .../machine/bytecode/compiler/Compilation.java | 26 ++
 .../tinkerpop/machine/species/LocalMachine.java| 12 +-
 .../machine/species/remote/RemoteMachine.java  |  8 +++
 3 files changed, 36 insertions(+), 10 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
index 5c02215..9b15d95 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
@@ -129,6 +129,32 @@ public final class Compilation implements 
Serializable {
 return this.functions.toString();
 } // TODO: functions need access to compilations for nesting
 
+
+@Override
+public int hashCode() {
+return this.processorFactory.hashCode() ^ 
this.structureFactory.hashCode() ^ this.functions.hashCode();
+}
+
+@Override
+public boolean equals(final Object object) {
+return object instanceof Compilation &&
+this.functions.equals(((Compilation) object).functions) &&
+this.processorFactory.equals(((Compilation) 
object).processorFactory) &&
+this.structureFactory.equals(((Compilation) 
object).structureFactory);
+}
+
+@Override
+public Compilation clone() {
+try {
+final Compilation clone = (Compilation) 
super.clone();
+clone.processor = null;
+// clone.functions =  ... we need to do a deep clone given the 
nested compilations
+return clone;
+} catch (final CloneNotSupportedException e) {
+throw new RuntimeException(e.getMessage(), e);
+}
+}
+
 
 
 public static  Compilation compile(final 
SourceCompilation source, final Bytecode bytecode) {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
index 79b958d..cf3e221 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
@@ -73,13 +73,18 @@ public final class LocalMachine implements Machine {
 @Override
 public  Iterator> submit(Bytecode bytecode) {
 bytecode = bytecode.clone();
-final UUID sourceId = 
LocalMachine.getSourceId(bytecode).orElse(UUID.randomUUID());
+final UUID sourceId = LocalMachine.getSourceId(bytecode).orElse(null);
 final SourceCompilation source = (SourceCompilation) 
this.sources.get(sourceId);
 return null == source ?
 Compilation.compile(bytecode).getProcessor() :
 Compilation.compile(source, 
bytecode).getProcessor();
 }
 
+@Override
+public void close() {
+this.sources.clear();
+}
+
 public static Machine open() {
 return new LocalMachine();
 }
@@ -91,9 +96,4 @@ public final class LocalMachine implements Machine {
 }
 return Optional.empty();
 }
-
-@Override
-public void close() {
-this.sources.clear();
-}
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
index 3eb5b89..98f39e5 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
@@ -87,10 +87,6 @@ public final class RemoteMachine implements Machine, 
AutoCloseable {
 }
 }
 
-public static Machine open(final int traverserServerPort, final String 
machineServerLocation, final int machineServerPort) {
-return new RemoteMachine(traverserServer

[tinkerpop] branch tp4 updated: Made is so that Beam can return traversers while its pipeline is running.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4fb4b90  Made is so that Beam can return traversers while its pipeline 
is running.
4fb4b90 is described below

commit 4fb4b90c910715ae1e4825af8bec1ff5606e5c0a
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 18:50:51 2019 -0600

Made is so that Beam can return traversers while its pipeline is running.
---
 .../apache/tinkerpop/machine/processor/beam/Beam.java  | 18 --
 1 file changed, 12 insertions(+), 6 deletions(-)

diff --git 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
index fd0bf65..8cb260a 100644
--- 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
+++ 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/Beam.java
@@ -19,6 +19,7 @@
 package org.apache.tinkerpop.machine.processor.beam;
 
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -44,8 +45,10 @@ public class Beam implements Processor {
 private boolean createTraverserServer;
 private final int traverserServerPort;
 private final Pipeline pipeline;
+private PipelineResult pipelineResult;
 private Iterator> iterator = null;
 
+
 public Beam(final Compilation compilation, final String 
traverserServerLocation, final int traverserServerPort, final boolean 
createTraverserServer) {
 this.traverserServerPort = traverserServerPort;
 this.createTraverserServer = createTraverserServer;
@@ -89,12 +92,15 @@ public class Beam implements Processor {
 
 private void setupPipeline() {
 if (null == this.iterator) {
-this.iterator = this.createTraverserServer ?
-new TraverserServer<>(this.traverserServerPort) :
-Collections.emptyIterator();
-this.pipeline.run().waitUntilFinish();
-if (this.iterator instanceof TraverserServer)
-((TraverserServer) this.iterator).close();
+if (this.createTraverserServer) {
+this.iterator = new 
TraverserServer<>(this.traverserServerPort);
+this.pipelineResult = this.pipeline.run();
+} else {
+this.iterator = Collections.emptyIterator();
+this.pipeline.run().waitUntilFinish();
+}
 }
+if (this.createTraverserServer && 
this.pipelineResult.getState().isTerminal())
+((TraverserServer) this.iterator).close();
 }
 }



[tinkerpop] branch tp4 updated: renamed close to unregister in Result.Type enum.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 457293a  renamed close to unregister in Result.Type enum.
457293a is described below

commit 457293a21ac270ae96e17dc7573ee372375d6232
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 17:46:45 2019 -0600

renamed close to unregister in Result.Type enum.
---
 .../apache/tinkerpop/machine/species/remote/MachineServer.java| 2 +-
 .../apache/tinkerpop/machine/species/remote/RemoteMachine.java| 8 
 2 files changed, 5 insertions(+), 5 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index 28cf0bd..8931f40 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -103,7 +103,7 @@ public final class MachineServer implements AutoCloseable {
 
traverserOutput.writeObject(EmptyTraverser.instance()); // this tells a 
TraverserServer that there are no more traversers
 traverserOutput.flush();
 }
-} else { // Request.Type.close == request.type
+} else { // Request.Type.unregister == request.type
 
MachineServer.this.machine.unregister(request.bytecode);
 }
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
index a15d3b5..3eb5b89 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
@@ -80,7 +80,7 @@ public final class RemoteMachine implements Machine, 
AutoCloseable {
 @Override
 public  void unregister(final Bytecode sourceCode) {
 try {
-this.outputStream.writeObject(Request.close(sourceCode));
+this.outputStream.writeObject(Request.unregister(sourceCode));
 this.outputStream.flush();
 } catch (final IOException e) {
 throw new RuntimeException(e.getMessage(), e);
@@ -108,7 +108,7 @@ public final class RemoteMachine implements Machine, 
AutoCloseable {
 static final class Request implements Serializable {
 
 public enum Type {
-register, submit, close;
+register, submit, unregister;
 }
 
 public final Type type;
@@ -131,8 +131,8 @@ public final class RemoteMachine implements Machine, 
AutoCloseable {
 return new Request<>(Request.Type.submit, bytecode, 
traverserServerLocation, traverserServerPort);
 }
 
-static  Request close(final Bytecode bytecode) {
-return new Request<>(Request.Type.close, bytecode, null, -1);
+static  Request unregister(final Bytecode bytecode) {
+return new Request<>(Request.Type.unregister, bytecode, null, -1);
 }
 }
 }



[tinkerpop] branch tp4 updated: try block Closeables for more efficient error handling and socket closing.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 439cf53  try block Closeables for more efficient error handling and 
socket closing.
439cf53 is described below

commit 439cf537f70fe29862da128e37cb70683e3d2abd
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 15:05:47 2019 -0600

try block Closeables for more efficient error handling and socket closing.
---
 .../machine/species/remote/MachineServer.java  | 27 +++---
 .../machine/species/remote/TraverserServer.java|  4 +---
 2 files changed, 14 insertions(+), 17 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index e6d573c..28cf0bd 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -83,27 +83,26 @@ public final class MachineServer implements AutoCloseable {
 }
 
 public void run() {
-try {
-final ObjectInputStream input = new 
ObjectInputStream(this.clientSocket.getInputStream());
-final ObjectOutputStream output = new 
ObjectOutputStream(this.clientSocket.getOutputStream());
+try (final ObjectInputStream input = new 
ObjectInputStream(this.clientSocket.getInputStream());
+ final ObjectOutputStream output = new 
ObjectOutputStream(this.clientSocket.getOutputStream())) {
 while (true) {
 final RemoteMachine.Request request = 
(RemoteMachine.Request) input.readObject();
 if (RemoteMachine.Request.Type.register == request.type) {
 
output.writeObject(MachineServer.this.machine.register(request.bytecode));
 output.flush();
 } else if (RemoteMachine.Request.Type.submit == 
request.type) {
-final Socket traverserServerSocket = new 
Socket(request.traverserServerLocation, request.traverserServerPort);
-final ObjectOutputStream traverserOutput = new 
ObjectOutputStream(traverserServerSocket.getOutputStream());
-final Iterator> iterator = 
MachineServer.this.machine.submit(request.bytecode);
-int flushCounter = 0;
-while (iterator.hasNext()) {
-flushCounter++;
-traverserOutput.writeObject(iterator.next());
-if (0 == flushCounter % FLUSH_AMOUNT) 
traverserOutput.flush();
+try (final Socket traverserServerSocket = new 
Socket(request.traverserServerLocation, request.traverserServerPort);
+ final ObjectOutputStream traverserOutput = new 
ObjectOutputStream(traverserServerSocket.getOutputStream())) {
+final Iterator> iterator 
= MachineServer.this.machine.submit(request.bytecode);
+int flushCounter = 0;
+while (iterator.hasNext()) {
+flushCounter++;
+traverserOutput.writeObject(iterator.next());
+if (0 == flushCounter % FLUSH_AMOUNT) 
traverserOutput.flush();
+}
+
traverserOutput.writeObject(EmptyTraverser.instance()); // this tells a 
TraverserServer that there are no more traversers
+traverserOutput.flush();
 }
-
traverserOutput.writeObject(EmptyTraverser.instance()); // this tells a 
TraverserServer that there are no more traversers
-traverserOutput.flush();
-traverserOutput.close();
 } else { // Request.Type.close == request.type
 
MachineServer.this.machine.unregister(request.bytecode);
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
index 6caecc5..d647f96 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
@@ -97,8 +97,7 @@ public final class TraverserServer implements 
AutoCloseable, Iterator traverser = (Traverser) 
input.read

[tinkerpop] branch tp4 updated: removed pointless method in MachineServer.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new affa536  removed pointless method in MachineServer.
affa536 is described below

commit affa536fe12deef535281e061849ef01a9b470d8
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 14:59:43 2019 -0600

removed pointless method in MachineServer.
---
 .../org/apache/tinkerpop/machine/species/remote/MachineServer.java  | 6 +-
 1 file changed, 1 insertion(+), 5 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index 7715626..e6d573c 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -52,7 +52,7 @@ public final class MachineServer implements AutoCloseable {
 private void run() {
 try {
 this.machineServerSocket = new 
ServerSocket(this.machineServerPort);
-while (this.isAlive()) {
+while (this.serverAlive.get()) {
 final Socket clientSocket = this.machineServerSocket.accept();
 new Thread(new Worker(clientSocket)).start();
 }
@@ -62,10 +62,6 @@ public final class MachineServer implements AutoCloseable {
 }
 }
 
-private boolean isAlive() {
-return this.serverAlive.get();
-}
-
 public void close() {
 if (this.serverAlive.get()) {
 try {



[tinkerpop] branch tp4 updated: made the server workers private inner classes.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 009027a  made the server workers private inner classes.
009027a is described below

commit 009027a72718935744cf6cf4dfc21f06386fb605
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 14:52:17 2019 -0600

made the server workers private inner classes.
---
 .../org/apache/tinkerpop/machine/species/remote/MachineServer.java| 2 +-
 .../org/apache/tinkerpop/machine/species/remote/TraverserServer.java  | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index 073f664..7715626 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -82,7 +82,7 @@ public final class MachineServer implements AutoCloseable {
 
 private final Socket clientSocket;
 
-Worker(final Socket clientSocket) {
+private Worker(final Socket clientSocket) {
 this.clientSocket = clientSocket;
 }
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
index c8c4c14..6caecc5 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
@@ -88,11 +88,11 @@ public final class TraverserServer implements 
AutoCloseable, Iterator

[tinkerpop] branch tp4 updated: made Request and inner class of RemoteMachine. Its only used by RemoteMachine and MachineServer... no need to expose it.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 230dfbf  made Request and inner class of RemoteMachine. Its only used 
by RemoteMachine and MachineServer... no need to expose it.
230dfbf is described below

commit 230dfbfb2f53724b81c7bb0a1206751bb94e061b
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 14:49:26 2019 -0600

made Request and inner class of RemoteMachine. Its only used by 
RemoteMachine and MachineServer... no need to expose it.
---
 .../machine/species/remote/MachineServer.java  |  6 +--
 .../machine/species/remote/RemoteMachine.java  | 35 +
 .../tinkerpop/machine/species/remote/Request.java  | 57 --
 3 files changed, 38 insertions(+), 60 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index 18acf8e..073f664 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -91,11 +91,11 @@ public final class MachineServer implements AutoCloseable {
 final ObjectInputStream input = new 
ObjectInputStream(this.clientSocket.getInputStream());
 final ObjectOutputStream output = new 
ObjectOutputStream(this.clientSocket.getOutputStream());
 while (true) {
-final Request request = (Request) 
input.readObject();
-if (Request.Type.register == request.type) {
+final RemoteMachine.Request request = 
(RemoteMachine.Request) input.readObject();
+if (RemoteMachine.Request.Type.register == request.type) {
 
output.writeObject(MachineServer.this.machine.register(request.bytecode));
 output.flush();
-} else if (Request.Type.submit == request.type) {
+} else if (RemoteMachine.Request.Type.submit == 
request.type) {
 final Socket traverserServerSocket = new 
Socket(request.traverserServerLocation, request.traverserServerPort);
 final ObjectOutputStream traverserOutput = new 
ObjectOutputStream(traverserServerSocket.getOutputStream());
 final Iterator> iterator = 
MachineServer.this.machine.submit(request.bytecode);
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
index 156193d..a15d3b5 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/RemoteMachine.java
@@ -25,6 +25,7 @@ import org.apache.tinkerpop.machine.traverser.Traverser;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
+import java.io.Serializable;
 import java.net.InetAddress;
 import java.net.Socket;
 import java.util.Iterator;
@@ -100,4 +101,38 @@ public final class RemoteMachine implements Machine, 
AutoCloseable {
 throw new RuntimeException(e.getMessage(), e);
 }
 }
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+static final class Request implements Serializable {
+
+public enum Type {
+register, submit, close;
+}
+
+public final Type type;
+public final Bytecode bytecode;
+final String traverserServerLocation;
+final int traverserServerPort;
+
+private Request(final Type type, final Bytecode bytecode, final 
String traverserServerLocation, final int traverserServerPort) {
+this.type = type;
+this.bytecode = bytecode;
+this.traverserServerLocation = traverserServerLocation;
+this.traverserServerPort = traverserServerPort;
+}
+
+static  Request register(final Bytecode bytecode) {
+return new Request<>(Request.Type.register, bytecode, null, -1);
+}
+
+static  Request submit(final Bytecode bytecode, final String 
traverserServerLocation, final int traverserServerPort) {
+return new Request<>(Request.Type.submit, bytecode, 
traverserServerLocation, traverserServerPort);
+}
+
+static  Request close(final Bytecode bytecode) {
+return new Request<>(Request.Type.close, bytecode, null, -1);
+}
+}
 }
diff --git 
a

[tinkerpop] branch tp4 updated: more random cleanups. taking a break to do some yardwork. its sunny out.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new a9dde82  more random cleanups. taking a break to do some yardwork. its 
sunny out.
a9dde82 is described below

commit a9dde82afe8ee5107b0a880db572712eec949db2
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 14:44:29 2019 -0600

more random cleanups. taking a break to do some yardwork. its sunny out.
---
 .../apache/tinkerpop/machine/species/LocalMachine.java   |  2 +-
 .../tinkerpop/machine/species/remote/MachineServer.java  | 16 +---
 .../machine/species/remote/TraverserServer.java  |  2 --
 3 files changed, 10 insertions(+), 10 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
index b29298a..79b958d 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
@@ -84,7 +84,7 @@ public final class LocalMachine implements Machine {
 return new LocalMachine();
 }
 
-private static final  Optional getSourceId(final Bytecode 
bytecode) {
+private static  Optional getSourceId(final Bytecode bytecode) {
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
 if (sourceInstruction.op().equals(WITH_SOURCE_CODE))
 return Optional.of(UUID.fromString((String) 
sourceInstruction.args()[0]));
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
index 8c01808..18acf8e 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/MachineServer.java
@@ -66,13 +66,15 @@ public final class MachineServer implements AutoCloseable {
 return this.serverAlive.get();
 }
 
-public synchronized void close() {
-try {
-this.serverAlive.set(Boolean.FALSE);
-this.machineServerSocket.close();
-this.machine.close();
-} catch (final IOException e) {
-throw new RuntimeException(e.getMessage(), e);
+public void close() {
+if (this.serverAlive.get()) {
+try {
+this.serverAlive.set(Boolean.FALSE);
+this.machineServerSocket.close();
+this.machine.close();
+} catch (final IOException e) {
+throw new RuntimeException(e.getMessage(), e);
+}
 }
 }
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
index 9c45994..c8c4c14 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/remote/TraverserServer.java
@@ -115,6 +115,4 @@ public final class TraverserServer implements 
AutoCloseable, Iterator

[tinkerpop] branch tp4 updated: Lots of clean up on TraverserServer, RemoteMachine, and MachineServer. Figured out cool Runnable Java functional-style.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new d1b677f  Lots of clean up on TraverserServer, RemoteMachine, and 
MachineServer. Figured out cool Runnable Java functional-style.
d1b677f is described below

commit d1b677f0e7bd47f1d75c464a57c82f8ca15cc400
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 14:33:16 2019 -0600

Lots of clean up on TraverserServer, RemoteMachine, and MachineServer. 
Figured out cool Runnable Java functional-style.
---
 .../language/gremlin/TraversalSource.java  |  2 +-
 .../java/org/apache/tinkerpop/machine/Machine.java |  4 +-
 .../tinkerpop/machine/species/BasicMachine.java|  6 +--
 .../tinkerpop/machine/species/LocalMachine.java|  7 ++--
 .../machine/species/remote/MachineServer.java  | 43 +++---
 .../machine/species/remote/RemoteMachine.java  | 41 +++--
 .../tinkerpop/machine/species/remote/Request.java  |  4 +-
 .../machine/species/remote/TraverserServer.java| 43 +++---
 .../machine/species/LocalMachineTest.java  |  2 +-
 .../tinkerpop/machine/processor/beam/Beam.java | 12 +++---
 .../tinkerpop/machine/processor/beam/OutputFn.java |  7 ++--
 .../tinkerpop/machine/processor/beam/ReduceFn.java |  4 +-
 .../{serialization => io}/CoefficientCoder.java|  2 +-
 .../beam/{serialization => io}/ReducerCoder.java   |  2 +-
 .../beam/{serialization => io}/TraverserCoder.java |  2 +-
 .../processor/beam/sideeffect/InMemoryReducer.java |  2 +-
 .../machine/processor/beam/util/TopologyUtil.java  |  2 +-
 .../tinkerpop/machine/processor/beam/BeamTest.java |  8 +++-
 .../machine/processor/pipes/PipesTest.java |  2 +-
 19 files changed, 101 insertions(+), 94 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index e27f680..e8c0bd1 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
@@ -112,7 +112,7 @@ public class TraversalSource implements Cloneable {
 }
 
 public void close() {
-this.machine.close(this.bytecode);
+this.machine.unregister(this.bytecode);
 }
 
 @Override
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
index fb9074f..8e4a781 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
@@ -33,5 +33,7 @@ public interface Machine extends Closeable {
 
 public  Iterator> submit(final Bytecode 
bytecode); // TODO: should return a "ShellTraverser"
 
-public  void close(final Bytecode sourceCode);
+public  void unregister(final Bytecode sourceCode);
+
+public void close();
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
index 2bf8b2c..5dbe4aa 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
@@ -23,7 +23,6 @@ import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
-import java.io.IOException;
 import java.util.Iterator;
 
 /**
@@ -41,7 +40,7 @@ public final class BasicMachine implements Machine {
 }
 
 @Override
-public  void close(final Bytecode sourceCode) {
+public  void unregister(final Bytecode sourceCode) {
 
 }
 
@@ -55,7 +54,6 @@ public final class BasicMachine implements Machine {
 }
 
 @Override
-public void close() throws IOException {
-
+public void close() {
 }
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
index 9e192ee..b29298a 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
@@ -26,7 +26,6 @@ import 
org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.bytecode.compiler.Sour

[tinkerpop] branch tp4 updated: big moves. MachineServer, RemoteMachine, TraverserServer. We now have general support for over the network traverser iterators.

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new da887c2  big moves. MachineServer, RemoteMachine, TraverserServer. We 
now have general support for over the network traverser iterators.
da887c2 is described below

commit da887c23c1f18617ff77ede74d69d93a4895fb83
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 11:45:03 2019 -0600

big moves. MachineServer, RemoteMachine, TraverserServer. We now have 
general support for over the network traverser iterators.
---
 .../language/gremlin/TraversalSource.java  |   8 ++
 .../java/org/apache/tinkerpop/machine/Machine.java |   3 +-
 .../tinkerpop/machine/bytecode/Bytecode.java   |   3 +-
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |   5 +-
 .../tinkerpop/machine/bytecode/Instruction.java|   3 +-
 .../machine/bytecode/SourceInstruction.java|   3 +-
 .../tinkerpop/machine/function/map/PathMap.java|   2 +-
 .../tinkerpop/machine/species/BasicMachine.java|   6 ++
 .../tinkerpop/machine/species/LocalMachine.java|   6 ++
 .../tinkerpop/machine/species/RemoteMachine.java   |  52 ---
 .../MachineServer.java}|  73 ---
 .../machine/species/remote/RemoteMachine.java  | 100 +
 .../tinkerpop/machine/species/remote/Request.java  |  57 
 .../species/{io => remote}/TraverserServer.java|  17 ++--
 .../tinkerpop/machine/structure/data/JTuple2.java  |   4 +-
 .../tinkerpop/machine/structure/data/TMap.java |   3 +-
 .../tinkerpop/machine/processor/beam/Beam.java |  29 +++---
 .../machine/processor/beam/BeamProcessor.java  |  28 +-
 .../tinkerpop/machine/processor/beam/OutputFn.java |  26 +++---
 .../tinkerpop/machine/processor/beam/BeamTest.java |  14 ++-
 .../machine/processor/pipes/PipesTest.java |  12 ++-
 21 files changed, 310 insertions(+), 144 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index fa2d29f..e27f680 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
@@ -33,6 +33,8 @@ import 
org.apache.tinkerpop.machine.strategy.verification.CoefficientVerificatio
 import org.apache.tinkerpop.machine.structure.StructureFactory;
 import org.apache.tinkerpop.machine.structure.data.TVertex;
 
+import java.util.Map;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -64,6 +66,12 @@ public class TraversalSource implements Cloneable {
 return clone;
 }
 
+public TraversalSource withProcessor(final Class processor, final Map configuration) {
+final TraversalSource clone = this.clone();
+clone.bytecode.addSourceInstruction(Symbols.WITH_PROCESSOR, processor, 
configuration);
+return clone;
+}
+
 public TraversalSource withStructure(final Class structure) {
 final TraversalSource clone = this.clone();
 clone.bytecode.addSourceInstruction(Symbols.WITH_STRUCTURE, structure);
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
index a4f84e7..fb9074f 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
@@ -21,12 +21,13 @@ package org.apache.tinkerpop.machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
+import java.io.Closeable;
 import java.util.Iterator;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public interface Machine {
+public interface Machine extends Closeable {
 
 public  Bytecode register(final Bytecode sourceCode);
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
index bcc1dab..ba77f99 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
@@ -20,13 +20,14 @@ package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class Bytec

[tinkerpop] branch tp4 updated: Beam now support TraverserServer which allows the output steps to write (in parallel and over the network) to a multi-threaded server backed by a TraverserSet and thus,

2019-03-26 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 195d0da  Beam now support TraverserServer which allows the output 
steps to write (in parallel and over the network) to a multi-threaded server 
backed by a TraverserSet and thus, is an Iterator.
195d0da is described below

commit 195d0da36507c72e4faa0913d6977be0dd6103ec
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 26 08:27:20 2019 -0600

Beam now support TraverserServer which allows the output steps to write (in 
parallel and over the network) to a multi-threaded server backed by a 
TraverserSet and thus, is an Iterator.
---
 .../machine/{ => species}/BasicMachine.java|   3 +-
 .../machine/{ => species}/LocalMachine.java|   3 +-
 .../RemoteMachine.java}|  24 ++--
 .../machine/species/io/TraverserServer.java| 122 +
 .../machine/{ => species}/LocalMachineTest.java|   2 +-
 .../tinkerpop/machine/processor/beam/Beam.java |  14 +--
 .../tinkerpop/machine/processor/beam/OutputFn.java |  61 ++-
 .../tinkerpop/machine/processor/beam/BeamTest.java |   2 +-
 .../machine/processor/pipes/PipesTest.java |   2 +-
 9 files changed, 203 insertions(+), 30 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
similarity index 94%
copy from 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
copy to 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
index 27cfc58..43b4946 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/BasicMachine.java
@@ -16,8 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine;
+package org.apache.tinkerpop.machine.species;
 
+import org.apache.tinkerpop.machine.Machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.traverser.Traverser;
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
similarity index 97%
rename from 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
rename to 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
index a5d4e27..042124c 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/LocalMachine.java
@@ -16,8 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine;
+package org.apache.tinkerpop.machine.species;
 
+import org.apache.tinkerpop.machine.Machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.BytecodeUtil;
 import org.apache.tinkerpop.machine.bytecode.SourceInstruction;
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/RemoteMachine.java
similarity index 76%
rename from 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
rename to 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/RemoteMachine.java
index 27cfc58..00da1c6 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/species/RemoteMachine.java
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine;
+package org.apache.tinkerpop.machine.species;
 
+import org.apache.tinkerpop.machine.Machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
-import org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
 import java.util.Iterator;
@@ -27,28 +27,26 @@ import java.util.Iterator;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class BasicMachine implements Machine {
+public class RemoteMachine implements Machine {
 
-private BasicMachine() {
-// use open();
+private final int port;
+
+ 

[tinkerpop] branch tp4 updated: some code clean up on LocalMachine>

2019-03-25 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 21c098d  some code clean up on LocalMachine>
21c098d is described below

commit 21c098d1e0c1fb0cf13f2fdae07a97bb7db4a54d
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 25 08:59:13 2019 -0600

some code clean up on LocalMachine>
---
 .../org/apache/tinkerpop/machine/LocalMachine.java | 25 --
 .../machine/bytecode/compiler/Compilation.java |  4 ++--
 2 files changed, 16 insertions(+), 13 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
index 774a3aa..a5d4e27 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/LocalMachine.java
@@ -47,18 +47,21 @@ public final class LocalMachine implements Machine {
 public  Bytecode register(Bytecode sourceCode) {
 sourceCode = sourceCode.clone(); // if the connection is local, don't 
mutate original
 final Optional id = LocalMachine.getSourceId(sourceCode);
-if (id.isPresent() && this.sources.containsKey(id.get())) {
-if (1 == sourceCode.getSourceInstructions().size())
-return sourceCode;
-final SourceCompilation source = (SourceCompilation) 
this.sources.get(id.get());
-BytecodeUtil.mergeSourceInstructions(source.getSourceCode(), 
sourceCode);
+if (id.isPresent()) {
+if (this.sources.containsKey(id.get())) {
+if (1 == sourceCode.getSourceInstructions().size())
+return sourceCode;
+final SourceCompilation source = (SourceCompilation) 
this.sources.get(id.get());
+BytecodeUtil.mergeSourceInstructions(source.getSourceCode(), 
sourceCode);
+}
 sourceCode.getInstructions().removeIf(i -> 
i.op().equals(WITH_SOURCE_CODE));
 }
 final UUID uuid = UUID.randomUUID();
 this.sources.put(uuid, new SourceCompilation<>(sourceCode));
-final Bytecode newSource = new Bytecode<>();
-newSource.addUniqueSourceInstruction(WITH_SOURCE_CODE, 
uuid.toString());
-return newSource;
+final Bytecode registeredBytecode = new Bytecode<>();
+registeredBytecode.addSourceInstruction(WITH_SOURCE_CODE, 
uuid.toString());
+
registeredBytecode.getInstructions().addAll(sourceCode.getInstructions()); // 
all bytecode is returned
+return registeredBytecode;
 }
 
 @Override
@@ -69,9 +72,9 @@ public final class LocalMachine implements Machine {
 @Override
 public  Iterator> submit(Bytecode bytecode) {
 bytecode = bytecode.clone();
-final UUID sourceId = LocalMachine.getSourceId(bytecode).orElse(null);
-final SourceCompilation source = null == sourceId ? null : 
(SourceCompilation) this.sources.get(sourceId);
-return null == sourceId ?
+final UUID sourceId = 
LocalMachine.getSourceId(bytecode).orElse(UUID.randomUUID());
+final SourceCompilation source = (SourceCompilation) 
this.sources.get(sourceId);
+return null == source ?
 Compilation.compile(bytecode).getProcessor() :
 Compilation.compile(source, 
bytecode).getProcessor();
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
index 3193685..5c02215 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
@@ -62,10 +62,10 @@ public final class Compilation implements 
Serializable {
 }
 
 public Compilation(final ProcessorFactory processorFactory) {
-this.functions = Collections.emptyList(); // TODO: somehow strings for 
primitive processors
-this.structureFactory = null;
+this.structureFactory = EmptyStructure.instance();
 this.processorFactory = processorFactory;
 this.traverserFactory = null;
+this.functions = Collections.emptyList(); // TODO: somehow strings for 
primitive processors
 }
 
 public Processor getProcessor() {



[tinkerpop] branch tp4 updated: LocalMachineTest added. TraversalSource cleaned up -- no more worrying about strategy compilation on the client side -- makes no sense anymore with Machine.register(sou

2019-03-25 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 596caf3  LocalMachineTest added. TraversalSource cleaned up -- no more 
worrying about strategy compilation on the client side -- makes no sense 
anymore with Machine.register(sourceCode). Sweet.
596caf3 is described below

commit 596caf3ab82f3b15c2c343af87be6d03f26d6d6e
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 25 06:08:48 2019 -0600

LocalMachineTest added. TraversalSource cleaned up -- no more worrying 
about strategy compilation on the client side -- makes no sense anymore with 
Machine.register(sourceCode). Sweet.
---
 .../language/gremlin/TraversalSource.java  | 10 ---
 .../machine/{Machine.java => BasicMachine.java}| 26 +--
 .../org/apache/tinkerpop/machine/LocalMachine.java | 22 --
 .../java/org/apache/tinkerpop/machine/Machine.java |  4 +-
 .../machine/bytecode/compiler/Compilation.java |  2 +-
 .../bytecode/compiler/SourceCompilation.java   |  5 +-
 .../machine/processor/EmptyProcessor.java  | 64 +
 .../apache/tinkerpop/machine/LocalMachineTest.java | 81 ++
 8 files changed, 188 insertions(+), 26 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
index 2eed977..fa2d29f 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
@@ -19,7 +19,6 @@
 package org.apache.tinkerpop.language.gremlin;
 
 import org.apache.tinkerpop.language.gremlin.common.CommonTraversal;
-import org.apache.tinkerpop.machine.LocalMachine;
 import org.apache.tinkerpop.machine.Machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.BytecodeUtil;
@@ -43,8 +42,6 @@ public class TraversalSource implements Cloneable {
 private Bytecode bytecode;
 private Coefficient coefficient = (Coefficient) 
LongCoefficient.create();
 private boolean registered = false;
-// private Set sortedStrategies (will be more efficient to 
precompute sort order)
-// private Machine machine (will be more efficient for remote connections)
 
 TraversalSource(final Machine machine) {
 this.machine = machine;
@@ -52,7 +49,6 @@ public class TraversalSource implements Cloneable {
 this.bytecode.addSourceInstruction(Symbols.WITH_STRATEGY, 
CoefficientStrategy.class); // TODO: remove when strategies full integrated
 this.bytecode.addSourceInstruction(Symbols.WITH_STRATEGY, 
CoefficientVerificationStrategy.class);
 this.bytecode.addSourceInstruction(Symbols.WITH_STRATEGY, 
ExplainStrategy.class);
-this.bytecode.addSourceInstruction(Symbols.WITH_MACHINE, 
LocalMachine.class);
 }
 
 public TraversalSource withCoefficient(final Class> coefficient) {
@@ -62,12 +58,6 @@ public class TraversalSource implements Cloneable {
 return clone;
 }
 
-public TraversalSource withMachine(final Class 
machine) {
-final TraversalSource clone = this.clone();
-clone.bytecode.addUniqueSourceInstruction(Symbols.WITH_MACHINE, 
machine);
-return clone;
-}
-
 public TraversalSource withProcessor(final Class processor) {
 final TraversalSource clone = this.clone();
 clone.bytecode.addSourceInstruction(Symbols.WITH_PROCESSOR, processor);
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
similarity index 66%
copy from 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
copy to 
java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
index 34d8403..27cfc58 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/Machine.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/BasicMachine.java
@@ -19,6 +19,7 @@
 package org.apache.tinkerpop.machine;
 
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
+import org.apache.tinkerpop.machine.bytecode.compiler.Compilation;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
 import java.util.Iterator;
@@ -26,11 +27,28 @@ import java.util.Iterator;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public interface Machine {
+public final class BasicMachine implements Machine {
 
-public  Bytecode register(final Bytecode sourceCode);
+private BasicMachine() {
+// use open();
+}
 
-public  void close(final By

[tinkerpop] branch tp4 updated: I have worked out the concept of always submitting Bytecode to a Machine. The Machine is able to precompile source instructions (e.g. strategies, database and processor

2019-03-25 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new a9fb2e0  I have worked out the concept of always submitting Bytecode 
to a Machine. The Machine is able to precompile source instructions (e.g. 
strategies, database and processor connections). LocalMachine is simple thread 
safe machine for use on the local computer. Very neat model.
a9fb2e0 is described below

commit a9fb2e0db252a2fc08e2e1809c7afd2a80ce8b4d
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 25 05:30:29 2019 -0600

I have worked out the concept of always submitting Bytecode to a Machine. 
The Machine is able to precompile source instructions (e.g. strategies, 
database and processor connections). LocalMachine is simple thread safe machine 
for use on the local computer. Very neat model.
---
 .../language/gremlin/AbstractTraversal.java| 10 ++--
 .../apache/tinkerpop/language/gremlin/Gremlin.java |  6 +-
 .../language/gremlin/TraversalSource.java  | 23 +++-
 .../language/gremlin/common/CommonTraversal.java   |  7 ++-
 .../language/gremlin/core/CoreTraversal.java   |  7 ++-
 .../org/apache/tinkerpop/machine/LocalMachine.java | 48 ++-
 .../java/org/apache/tinkerpop/machine/Machine.java |  4 ++
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   | 27 -
 .../machine/bytecode/compiler/Compilation.java | 14 -
 .../bytecode/compiler/CompositeCompiler.java   |  5 ++
 .../bytecode/compiler/SourceCompilation.java   | 69 ++
 .../strategy/decoration/ExplainStrategy.java   |  4 +-
 .../CoefficientVerificationStrategy.java   |  3 +-
 .../tinkerpop/machine/processor/beam/BeamTest.java | 11 +++-
 .../machine/processor/pipes/PipesTest.java | 11 +++-
 15 files changed, 207 insertions(+), 42 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
index 5c35849..3d2811c 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
@@ -18,8 +18,8 @@
  */
 package org.apache.tinkerpop.language.gremlin;
 
+import org.apache.tinkerpop.machine.Machine;
 import org.apache.tinkerpop.machine.bytecode.Bytecode;
-import org.apache.tinkerpop.machine.bytecode.BytecodeUtil;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
@@ -32,8 +32,9 @@ import java.util.List;
  */
 public abstract class AbstractTraversal implements Traversal 
{
 
-protected Coefficient currentCoefficient;
+private final Machine machine;
 protected final Bytecode bytecode;
+protected Coefficient currentCoefficient;
 private Iterator> traversers = null;
 private boolean locked = false;
 
@@ -41,7 +42,8 @@ public abstract class AbstractTraversal implements 
Traversal {
 private long lastCount = 0L;
 private E lastObject = null;
 
-public AbstractTraversal(final Bytecode bytecode, final Coefficient 
unity) {
+public AbstractTraversal(final Machine machine, final Bytecode 
bytecode, final Coefficient unity) {
+this.machine = machine;
 this.bytecode = bytecode;
 this.currentCoefficient = unity.clone();
 }
@@ -59,7 +61,7 @@ public abstract class AbstractTraversal implements 
Traversal {
 private final void prepareTraversal() {
 if (!this.locked) {
 this.locked = true;
-this.traversers = 
BytecodeUtil.getMachine(this.bytecode).get().submit(this.bytecode);
+this.traversers = this.machine.submit(this.bytecode);
 }
 }
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
index fdd4981..b739a16 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
@@ -18,12 +18,14 @@
  */
 package org.apache.tinkerpop.language.gremlin;
 
+import org.apache.tinkerpop.machine.Machine;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public final class Gremlin {
 
-public static  TraversalSource traversal() {
-return new TraversalSource<>();
+public static  TraversalSource traversal(final Machine machine) {
+return new TraversalSource<>(machine);
 }
 }
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/sr

[tinkerpop] branch tp4 updated: minor nothing. done for the day.

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new ac8cc73  minor nothing. done for the day.
ac8cc73 is described below

commit ac8cc7349a5401df18d94a7dcf5d78b7677b1bbf
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 17:28:38 2019 -0600

minor nothing. done for the day.
---
 .../main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java | 5 +
 1 file changed, 1 insertion(+), 4 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
index e271484..6df4a20 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
@@ -56,10 +56,7 @@ public final class Instruction {
 }
 
 public void setLabel(final String label) {
-if (null == this.label)
-this.label = label;
-else
-throw new RuntimeException("Can't set a label twice");
+this.label = label;
 }
 
 public void addArg(final Object arg) {



[tinkerpop] branch tp4 updated: After discussing with @dkuppitz, Instructions can only have zero or one label. Path entries can only have zero or one label. This really cleans things up.

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 9fc2c75  After discussing with @dkuppitz, Instructions can only have 
zero or one label. Path entries can only have zero or one label. This really 
cleans things up.
9fc2c75 is described below

commit 9fc2c75e706370c2ecb24291b1be735dc8d05587
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 17:26:07 2019 -0600

After discussing with @dkuppitz, Instructions can only have zero or one 
label. Path entries can only have zero or one label. This really cleans things 
up.
---
 .../language/gremlin/common/CommonTraversal.java   |  2 +-
 .../language/gremlin/core/CoreTraversal.java   |  2 +-
 .../tinkerpop/machine/bytecode/Instruction.java| 19 ++--
 .../machine/function/AbstractFunction.java | 12 
 .../tinkerpop/machine/function/CFunction.java  |  2 +-
 .../machine/function/barrier/JoinBarrier.java  |  7 ++---
 .../machine/function/barrier/StallBarrier.java |  6 ++--
 .../machine/function/branch/BranchBranch.java  |  6 ++--
 .../machine/function/branch/RepeatBranch.java  |  6 ++--
 .../machine/function/filter/FilterFilter.java  |  6 ++--
 .../machine/function/filter/HasKeyFilter.java  |  6 ++--
 .../machine/function/filter/HasKeyValueFilter.java |  6 ++--
 .../machine/function/filter/IdentityFilter.java|  6 ++--
 .../machine/function/filter/IsFilter.java  |  6 ++--
 .../machine/function/flatmap/FlatMapFlatMap.java   |  8 ++---
 .../machine/function/flatmap/UnfoldFlatMap.java|  6 ++--
 .../machine/function/initial/InitialInitial.java   |  6 ++--
 .../machine/function/map/ConstantMap.java  |  8 ++---
 .../tinkerpop/machine/function/map/IncrMap.java|  6 ++--
 .../tinkerpop/machine/function/map/LoopsMap.java   |  6 ++--
 .../tinkerpop/machine/function/map/MapMap.java |  6 ++--
 .../tinkerpop/machine/function/map/PathMap.java| 11 ---
 .../tinkerpop/machine/function/map/ValueMap.java   |  6 ++--
 .../machine/function/reduce/CountReduce.java   |  6 ++--
 .../machine/function/reduce/GroupCountReduce.java  |  6 ++--
 .../machine/function/reduce/ReduceReduce.java  |  6 ++--
 .../machine/function/reduce/SumReduce.java |  6 ++--
 .../strategy/optimization/IdentityStrategy.java|  2 +-
 .../tinkerpop/machine/structure/data/JMap.java |  7 +
 .../tinkerpop/machine/structure/data/JTuple2.java  |  4 +--
 .../machine/traverser/path/BasicPath.java  | 21 +
 .../machine/traverser/path/EmptyPath.java  |  9 ++
 .../tinkerpop/machine/traverser/path/Path.java |  5 ++--
 .../machine/traverser/species/COP_Traverser.java   |  2 +-
 .../traverser/species/COP_TraverserFactory.java|  2 +-
 .../tinkerpop/machine/util/StringFactory.java  |  8 ++---
 .../tinkerpop/machine/traverser/path/PathTest.java | 35 +-
 .../bytecode/compiler/BlueprintsCompiler.java  |  8 ++---
 .../function/initial/VerticesFlatMap.java  |  4 +--
 39 files changed, 129 insertions(+), 157 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index 1d3831e..a9c2a74 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
@@ -48,7 +48,7 @@ public class CommonTraversal extends 
AbstractTraversal {
 
 @Override
 public Traversal as(final String label) {
-this.bytecode.lastInstruction().addLabel(label);
+this.bytecode.lastInstruction().setLabel(label);
 return this;
 }
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
index deffa5a..eec864e 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
@@ -51,7 +51,7 @@ public class CoreTraversal extends 
AbstractTraversal {
 
 @Override
 public Traversal as(final String label) {
-this.bytecode.lastInstruction().addLabel(label);
+this.bytecode.lastInstruction().setLabel(label);
 return this;
 }
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
index 88f2d62..e271484 100644
--- 
a/java/machine/machine

[tinkerpop] branch tp4 updated: added PathTest and converted lots of list and set creations to List.of and Set.of

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 9b765b0  added PathTest and converted lots of list and set creations 
to List.of and Set.of
9b765b0 is described below

commit 9b765b034182432048d310d5b59d0f72d52d7f63
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 16:01:42 2019 -0600

added PathTest and converted lots of list and set creations to List.of and 
Set.of
---
 .../tinkerpop/machine/function/map/PathMap.java|   2 +-
 .../machine/traverser/path/BasicPath.java  |  17 ++-
 .../tinkerpop/machine/traverser/path/PathTest.java | 128 +
 .../machine/processor/beam/BeamProcessor.java  |   4 +-
 .../tinkerpop/machine/processor/beam/BeamTest.java |   6 +-
 .../machine/processor/pipes/PipesProcessor.java|   4 +-
 .../machine/processor/pipes/PipesTest.java |  10 +-
 .../structure/blueprints/BlueprintsStructure.java  |   5 +-
 8 files changed, 155 insertions(+), 21 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
index 6628a8a..3f09029 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
@@ -61,7 +61,7 @@ public final class PathMap extends AbstractFunction 
implements MapFunct
 final Path newPath = new BasicPath();
 if (this.hasPathLabels) {
 for (final String label : this.pathLabels) {
-newPath.add(Collections.singleton(label), 
this.byCompilations.process(oldPath.get(Path.Pop.last, label)));
+newPath.add(Set.of(label), 
this.byCompilations.process(oldPath.get(Path.Pop.last, label)));
 }
 } else {
 for (int i = 0; i < oldPath.size(); i++) {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
index cbf8a7a..a3583f9 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/traverser/path/BasicPath.java
@@ -34,11 +34,6 @@ public final class BasicPath implements Path {
 public BasicPath() {
 }
 
-public BasicPath(final BasicPath path) {
-this.objects.addAll(path.objects);
-this.labels.addAll(path.labels);
-}
-
 @Override
 public void add(final Set labels, final Object object) {
 this.labels.add(labels);
@@ -84,6 +79,18 @@ public final class BasicPath implements Path {
 }
 
 @Override
+public int hashCode() {
+return this.labels.hashCode() ^ this.objects.hashCode();
+}
+
+@Override
+public boolean equals(final Object object) {
+return object instanceof BasicPath &&
+this.labels.equals(((BasicPath) object).labels) &&
+this.objects.equals(((BasicPath) object).objects);
+}
+
+@Override
 public String toString() {
 return this.objects.toString();
 }
diff --git 
a/java/machine/machine-core/src/test/java/org/apache/tinkerpop/machine/traverser/path/PathTest.java
 
b/java/machine/machine-core/src/test/java/org/apache/tinkerpop/machine/traverser/path/PathTest.java
new file mode 100644
index 000..7424fd4
--- /dev/null
+++ 
b/java/machine/machine-core/src/test/java/org/apache/tinkerpop/machine/traverser/path/PathTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tinkerpop.machine.traverser.path;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+import java.util.function.Supplier;
+
+import static org.junit.jupit

[tinkerpop] branch tp4 updated: cleaned up Compilation methods.

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4c6ff25  cleaned up Compilation methods.
4c6ff25 is described below

commit 4c6ff252218723c520b2f76d579bcb30360a98b8
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 12:47:04 2019 -0600

cleaned up Compilation methods.
---
 .../tinkerpop/machine/bytecode/compiler/BytecodeArgument.java |  2 +-
 .../tinkerpop/machine/bytecode/compiler/Compilation.java  | 11 +--
 .../tinkerpop/machine/function/barrier/JoinBarrier.java   |  2 +-
 .../tinkerpop/machine/function/branch/BranchBranch.java   |  4 ++--
 .../org/apache/tinkerpop/machine/function/map/PathMap.java|  2 +-
 5 files changed, 6 insertions(+), 15 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeArgument.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeArgument.java
index 1daae88..5711ab5 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeArgument.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/BytecodeArgument.java
@@ -29,7 +29,7 @@ public class BytecodeArgument implements Argument {
 private final Compilation compilation;
 
 public BytecodeArgument(final Bytecode arg) {
-this.compilation = Compilation.compileOne(arg);
+this.compilation = Compilation.compile(arg);
 }
 
 @Override
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
index 8474b5a..a60a804 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/compiler/Compilation.java
@@ -127,7 +127,7 @@ public final class Compilation implements 
Serializable {
 return new Compilation<>(bytecode);
 }
 
-public static  Compilation compileOne(final Object arg) {
+public static  Compilation compile(final Object arg) {
 return new Compilation<>((Bytecode) arg);
 }
 
@@ -136,13 +136,4 @@ public final class Compilation implements 
Serializable {
 new Compilation<>((Bytecode) args[index]) :
 null;
 }
-
-public static  List> compile(final Object... 
args) {
-final List> compilations = new ArrayList<>();
-for (final Object arg : args) {
-if (arg instanceof Bytecode)
-compilations.add(new Compilation<>((Bytecode) arg));
-}
-return compilations;
-}
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/JoinBarrier.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/JoinBarrier.java
index ffa3c7b..7dd7779 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/JoinBarrier.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/barrier/JoinBarrier.java
@@ -95,7 +95,7 @@ public final class JoinBarrier extends 
AbstractFunction implements B
 public static  JoinBarrier compile(final Instruction 
instruction) {
 return new JoinBarrier<>(instruction.coefficient(), 
instruction.labels(),
 CoreCompiler.Symbols.Tokens.valueOf((String) 
instruction.args()[0]),
-Compilation.compileOne(instruction.args()[1]),
+Compilation.compile(instruction.args()[1]),
 Argument.create(instruction.args()[2]));
 }
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
index 2113d01..b425e43 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/branch/BranchBranch.java
@@ -52,8 +52,8 @@ public final class BranchBranch extends 
AbstractFunction implements
 final Object[] args = instruction.args();
 final Map, List>> branches = 
new HashMap<>();
 for (int i = 0; i < args.length; i = i + 2) {
-final Compilation predicate = 
Symbols.DEFAULT.equals(args[i]) ? null : Compilation.compileOne(args[i]);
-final Compilation branch = Compilation.compileOne(args[i 
+ 1]);
+final Compil

[tinkerpop] branch tp4 updated: made it so path() compilation is easier using tokens in the instruction>

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 3d84979  made it so path() compilation is easier using tokens in the 
instruction>
3d84979 is described below

commit 3d84979ee25d7298d5f68475920bedd268d6bc64
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 12:40:44 2019 -0600

made it so path() compilation is easier using tokens in the instruction>
---
 .../java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java  | 7 +++
 .../apache/tinkerpop/language/gremlin/common/CommonTraversal.java  | 4 +---
 .../org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java  | 4 +---
 3 files changed, 9 insertions(+), 6 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
index 88ffde5..cc1c2d4 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
@@ -59,4 +59,11 @@ public final class TraversalUtil {
 }
 return args;
 }
+
+public static Object[] addObjects(final Object[] original, final Object... 
updates) {
+final Object[] objects = new Object[original.length + updates.length];
+System.arraycopy(original, 0, objects, 0, original.length);
+System.arraycopy(updates, 0, objects, original.length, updates.length);
+return objects;
+}
 }
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index c352b00..37b1b53 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
@@ -193,9 +193,7 @@ public class CommonTraversal extends 
AbstractTraversal {
 
 @Override
 public Traversal path(final String... labels) {
-this.addInstruction(Symbols.PATH, labels);
-this.bytecode.lastInstruction().addArg("|");
-return (Traversal) this;
+return this.addInstruction(Symbols.PATH, 
TraversalUtil.addObjects(labels, "|"));
 }
 
 @Override
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
index 6e8e97a..203fcec 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/core/CoreTraversal.java
@@ -220,9 +220,7 @@ public class CoreTraversal extends 
AbstractTraversal {
 
 @Override
 public Traversal path(final String... labels) {
-this.addInstruction(Symbols.PATH, (Object[]) labels);
-this.bytecode.lastInstruction().addArg("|");
-return (Traversal) this;
+return this.addInstruction(Symbols.PATH, 
TraversalUtil.addObjects(labels, "|"));
 }
 
 @Override



[tinkerpop] branch tp4 updated: All function types are responsible for their own compilation. They have static methods and private constructors.

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 53ea847  All function types are responsible for their own compilation. 
They have static methods and private constructors.
53ea847 is described below

commit 53ea847380296938a1890489baa0d7ad162a2809
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 12:33:41 2019 -0600

All function types are responsible for their own compilation. They have 
static methods and private constructors.
---
 .../language/gremlin/AbstractTraversal.java|  5 +-
 .../language/gremlin/common/CommonTraversal.java   |  5 +-
 .../language/gremlin/core/CoreTraversal.java   |  9 ++--
 .../org/apache/tinkerpop/machine/LocalMachine.java |  2 +-
 .../java/org/apache/tinkerpop/machine/Machine.java |  2 +-
 .../tinkerpop/machine/bytecode/Instruction.java|  6 ++-
 .../machine/bytecode/compiler/CommonCompiler.java  | 55 ++
 .../machine/bytecode/compiler/Compilation.java | 17 ---
 .../bytecode/compiler/CompilationCircle.java   | 14 +++---
 .../machine/bytecode/compiler/CoreCompiler.java| 36 ++
 .../machine/function/barrier/JoinBarrier.java  | 18 +--
 .../machine/function/barrier/StallBarrier.java |  9 +++-
 .../machine/function/branch/BranchBranch.java  |  8 ++--
 .../machine/function/branch/RepeatBranch.java  | 20 
 .../machine/function/filter/FilterFilter.java  | 10 +++-
 .../machine/function/filter/HasKeyFilter.java  |  7 ++-
 .../machine/function/filter/HasKeyValueFilter.java |  7 ++-
 .../machine/function/filter/IdentityFilter.java|  7 ++-
 .../machine/function/filter/IsFilter.java  |  7 ++-
 .../machine/function/flatmap/FlatMapFlatMap.java   |  9 +++-
 .../machine/function/flatmap/UnfoldFlatMap.java|  9 +++-
 .../{InjectInitial.java => InitialInitial.java}|  9 +++-
 .../machine/function/map/ConstantMap.java  |  9 +++-
 .../tinkerpop/machine/function/map/IncrMap.java|  9 +++-
 .../tinkerpop/machine/function/map/LoopsMap.java   |  9 +++-
 .../tinkerpop/machine/function/map/MapMap.java |  9 +++-
 .../tinkerpop/machine/function/map/PathMap.java| 54 +
 .../tinkerpop/machine/function/map/ValueMap.java   |  9 +++-
 .../machine/function/reduce/CountReduce.java   |  9 +++-
 .../machine/function/reduce/GroupCountReduce.java  |  9 +++-
 .../machine/function/reduce/ReduceReduce.java  | 10 +++-
 .../machine/function/reduce/SumReduce.java |  9 +++-
 .../machine/processor/ConstantProcessor.java   | 39 ---
 .../machine/processor/HasNextProcessor.java| 39 ---
 .../machine/traverser/path/BasicPath.java  | 54 +
 .../machine/traverser/path/EmptyPath.java  | 15 --
 .../tinkerpop/machine/traverser/path/Path.java | 22 +++--
 .../machine/traverser/species/COP_Traverser.java   |  7 +--
 .../tinkerpop/machine/processor/beam/BeamTest.java |  2 +-
 .../machine/processor/pipes/PipesTest.java |  6 +++
 40 files changed, 356 insertions(+), 235 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
index 53eebb9..5c35849 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
@@ -35,7 +35,7 @@ public abstract class AbstractTraversal implements 
Traversal {
 protected Coefficient currentCoefficient;
 protected final Bytecode bytecode;
 private Iterator> traversers = null;
-private boolean locked = false; // TODO: when a traversal has been 
submitted, we need to make sure new modulations can't happen.
+private boolean locked = false;
 
 // iteration helpers
 private long lastCount = 0L;
@@ -57,7 +57,7 @@ public abstract class AbstractTraversal implements 
Traversal {
 }
 
 private final void prepareTraversal() {
-if (null == this.traversers) {
+if (!this.locked) {
 this.locked = true;
 this.traversers = 
BytecodeUtil.getMachine(this.bytecode).get().submit(this.bytecode);
 }
@@ -90,6 +90,7 @@ public abstract class AbstractTraversal implements 
Traversal {
 }
 }
 
+@Override
 public List toList() {
 final List list = new ArrayList<>();
 while (this.hasNext()) {
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/common/CommonTraversal.java
index c505d2d..c352b00 100644

[tinkerpop] branch tp4 updated: AbstractTraversal now has addInstruction() which does coefficient unity reseting and making sure the traverasl isn't locked (submitted).

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new a38e078  AbstractTraversal now has addInstruction() which does 
coefficient unity reseting and making sure the traverasl isn't locked 
(submitted).
a38e078 is described below

commit a38e07834a7ad7d206d91cac86d2a03f4e10a698
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 09:57:01 2019 -0600

AbstractTraversal now has addInstruction() which does coefficient unity 
reseting and making sure the traverasl isn't locked (submitted).
---
 .../language/gremlin/AbstractTraversal.java|  25 ++-
 .../org/apache/tinkerpop/language/gremlin/P.java   |   2 +-
 .../language/gremlin/TraversalSource.java  |  24 ++-
 .../tinkerpop/language/gremlin/TraversalUtil.java  |  12 +-
 .../language/gremlin/common/CommonTraversal.java   | 107 
 .../language/gremlin/core/CoreTraversal.java   | 183 +
 .../apache/tinkerpop/language/gremlin/core/__.java |  22 +--
 .../ConstantArgument.java => LocalMachine.java}|  23 +--
 .../{bytecode/FunctionType.java => Machine.java}   |  12 +-
 .../tinkerpop/machine/bytecode/Bytecode.java   |   1 -
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  31 +++-
 .../machine/bytecode/{ => compiler}/Argument.java  |   3 +-
 .../bytecode/{ => compiler}/BytecodeArgument.java  |   3 +-
 .../bytecode/{ => compiler}/BytecodeCompiler.java  |   4 +-
 .../{ => bytecode}/compiler/CommonCompiler.java|   8 +-
 .../bytecode/{ => compiler}/Compilation.java   |  13 +-
 .../bytecode/{ => compiler}/CompilationCircle.java |   2 +-
 .../bytecode/{ => compiler}/CompositeCompiler.java |   4 +-
 .../bytecode/{ => compiler}/ConstantArgument.java  |   2 +-
 .../{ => bytecode}/compiler/CoreCompiler.java  |   9 +-
 .../bytecode/{ => compiler}/FunctionType.java  |   2 +-
 .../bytecode/{ => compiler}/MethodArgument.java|   2 +-
 .../machine/bytecode/{ => compiler}/Oper.java  |   2 +-
 .../machine/bytecode/{ => compiler}/Pred.java  |   2 +-
 .../tinkerpop/machine/function/BranchFunction.java |   2 +-
 .../machine/function/barrier/JoinBarrier.java  |   6 +-
 .../machine/function/branch/BranchBranch.java  |   4 +-
 .../machine/function/branch/RepeatBranch.java  |   2 +-
 .../machine/function/filter/FilterFilter.java  |   4 +-
 .../machine/function/filter/HasKeyFilter.java  |   4 +-
 .../machine/function/filter/HasKeyValueFilter.java |   2 +-
 .../machine/function/filter/IsFilter.java  |   4 +-
 .../machine/function/flatmap/FlatMapFlatMap.java   |   2 +-
 .../tinkerpop/machine/function/map/MapMap.java |   2 +-
 .../tinkerpop/machine/function/map/PathMap.java|   4 +-
 .../tinkerpop/machine/function/map/ValueMap.java   |   2 +-
 .../machine/function/reduce/GroupCountReduce.java  |   2 +-
 .../machine/processor/HasNextProcessor.java|   2 +-
 .../machine/processor/ProcessorFactory.java|   4 +-
 .../machine/processor/SimpleProcessor.java |   6 +-
 .../strategy/decoration/ExplainStrategy.java   |   4 +-
 .../strategy/finalization/CoefficientStrategy.java |   2 +-
 .../strategy/optimization/IdentityStrategy.java|   2 +-
 .../CoefficientVerificationStrategy.java   |   4 +-
 .../machine/structure/StructureFactory.java|   2 +-
 .../tinkerpop/machine/processor/beam/Beam.java |   2 +-
 .../machine/processor/beam/BeamProcessor.java  |   8 +-
 .../tinkerpop/machine/processor/beam/BranchFn.java |   2 +-
 .../machine/processor/beam/RepeatEndFn.java|   2 +-
 .../machine/processor/beam/RepeatStartFn.java  |   2 +-
 .../processor/beam/strategy/BeamStrategy.java  |   2 +-
 .../machine/processor/beam/util/TopologyUtil.java  |   2 +-
 .../machine/processor/pipes/BranchStep.java|   2 +-
 .../tinkerpop/machine/processor/pipes/Pipes.java   |   2 +-
 .../machine/processor/pipes/PipesProcessor.java|   8 +-
 .../machine/processor/pipes/RepeatStep.java|   2 +-
 .../processor/pipes/strategy/PipesStrategy.java|   2 +-
 .../structure/blueprints/BlueprintsStructure.java  |   4 +-
 .../compiler/BlueprintsCompiler.java   |   6 +-
 .../provider/BlueprintsVerticesStrategy.java   |   4 +-
 60 files changed, 279 insertions(+), 335 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
index 5430215..53eebb9 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
@@ -19,11 +19,12 @@
 package org

[tinkerpop] branch tp4 updated: ProcessorFactory and StructureFactory are required to return a list of support BytecodeCompilers.

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 2eb1bbd  ProcessorFactory and StructureFactory are required to return 
a list of support BytecodeCompilers.
2eb1bbd is described below

commit 2eb1bbd13a5d433bd4f8d882db7e746c82b96ca5
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 08:58:16 2019 -0600

ProcessorFactory and StructureFactory are required to return a list of 
support BytecodeCompilers.
---
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   | 17 +---
 .../tinkerpop/machine/bytecode/Compilation.java| 11 +---
 .../machine/processor/ProcessorFactory.java| 11 ++--
 .../machine/processor/SimpleProcessor.java |  5 
 .../machine/structure/EmptyStructure.java  | 16 ---
 .../machine/structure/StructureFactory.java| 11 ++--
 .../tinkerpop/machine/traverser/COTraverser.java}  | 32 ++
 .../{COPTraverser.java => COP_Traverser.java}  | 10 +++
 ...erserFactory.java => COP_TraverserFactory.java} | 10 +++
 .../{CORTraverser.java => COR_Traverser.java}  |  6 ++--
 ...erserFactory.java => COR_TraverserFactory.java} | 10 +++
 .../{COTraverser.java => CO_Traverser.java}| 12 
 ...verserFactory.java => CO_TraverserFactory.java} | 10 +++
 .../machine/traverser/TraverserSetTest.java|  8 +++---
 java/machine/processor/beam/pom.xml|  2 +-
 .../tinkerpop/machine/processor/beam/Beam.java |  8 +++---
 .../machine/processor/beam/BeamProcessor.java  | 14 +-
 .../tinkerpop/machine/processor/beam/BeamTest.java |  4 +--
 java/machine/processor/pipes/pom.xml   |  2 +-
 .../machine/processor/pipes/EmptyStep.java |  4 +--
 .../machine/processor/pipes/PipesProcessor.java| 14 +-
 .../machine/processor/pipes/PipesTest.java |  4 +--
 .../structure/blueprints/BlueprintsStructure.java  | 12 +---
 .../{bytecode => compiler}/BlueprintsCompiler.java | 14 --
 .../provider/BlueprintsVerticesStrategy.java   |  2 +-
 25 files changed, 119 insertions(+), 130 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index 825d863..64e2827 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -19,16 +19,14 @@
 package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
-import org.apache.tinkerpop.machine.compiler.CommonCompiler;
-import org.apache.tinkerpop.machine.compiler.CoreCompiler;
 import org.apache.tinkerpop.machine.compiler.CoreCompiler.Symbols;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
 import org.apache.tinkerpop.machine.strategy.Strategy;
 import org.apache.tinkerpop.machine.strategy.StrategyUtil;
 import org.apache.tinkerpop.machine.structure.StructureFactory;
 import org.apache.tinkerpop.machine.traverser.TraverserFactory;
-import org.apache.tinkerpop.machine.traverser.species.COPTraverserFactory;
-import org.apache.tinkerpop.machine.traverser.species.CORTraverserFactory;
+import org.apache.tinkerpop.machine.traverser.species.COP_TraverserFactory;
+import org.apache.tinkerpop.machine.traverser.species.COR_TraverserFactory;
 
 import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
@@ -79,9 +77,8 @@ public final class BytecodeUtil {
 
 public static  CompositeCompiler getCompilers(final Bytecode 
bytecode) {
 final List compilers = new ArrayList<>();
-compilers.add(CoreCompiler.instance());
-compilers.add(CommonCompiler.instance()); // TODO: this needs to be 
part of source instruction
-BytecodeUtil.getStructureFactory(bytecode).ifPresent(f -> 
f.getCompiler().ifPresent(compilers::add));
+BytecodeUtil.getProcessorFactory(bytecode).ifPresent(f -> 
compilers.addAll(f.getCompilers()));
+BytecodeUtil.getStructureFactory(bytecode).ifPresent(f -> 
compilers.addAll(f.getCompilers()));
 return CompositeCompiler.create(compilers);
 }
 
@@ -150,10 +147,10 @@ public final class BytecodeUtil {
 // TODO: make this real
 for (final Instruction instruction : bytecode.getInstructions()) {
 if (instruction.op().equals(Symbols.PATH))
-return Optional.of(COPTraverserFactory.instance());
+return Optional.of(COP_TraverserFactory.instance());
 else if (instruction.op().equals(Symbols.REPEAT))
-return Opti

[tinkerpop] branch tp4 updated: we now have CoreCompiler and CommonCompiler. Likewise CoreTraversal and CommonTraversal in Gremlin. The idea is that if you know the underling VM supports the common in

2019-03-24 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new cf4dd8d  we now have CoreCompiler and CommonCompiler. Likewise 
CoreTraversal and CommonTraversal in Gremlin. The idea is that if you know the 
underling VM supports the common instruction set, you can create instructions 
for it accordingly. Still thinking through this...
cf4dd8d is described below

commit cf4dd8dd9d368a5916526f9043bc63f483b98e2c
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 24 08:22:22 2019 -0600

we now have CoreCompiler and CommonCompiler. Likewise CoreTraversal and 
CommonTraversal in Gremlin. The idea is that if you know the underling VM 
supports the common instruction set, you can create instructions for it 
accordingly. Still thinking through this...
---
 .../language/gremlin/AbstractTraversal.java|  94 +
 .../tinkerpop/language/gremlin/Traversal.java  | 385 +
 .../language/gremlin/TraversalSource.java  |  11 +-
 .../tinkerpop/language/gremlin/TraversalUtil.java  |  18 +-
 .../CommonTraversal.java}  | 243 +
 .../language/gremlin/{ => common}/__.java  |   8 +-
 .../language/gremlin/core/CoreTraversal.java   | 296 
 .../tinkerpop/language/gremlin/{ => core}/__.java  |  24 +-
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |   9 +-
 .../tinkerpop/machine/bytecode/Compilation.java|   4 +-
 .../CommonCompiler.java}   |  94 +++--
 .../{bytecode => compiler}/CoreCompiler.java   |  15 +-
 .../machine/function/barrier/JoinBarrier.java  |   2 +-
 .../machine/function/branch/BranchBranch.java  |   2 +-
 .../machine/function/filter/HasKeyFilter.java  |  65 
 .../machine/function/filter/HasKeyValueFilter.java |  55 +++
 .../filter/IdentityFilter.java}|  27 +-
 .../machine/function/filter/IsFilter.java  |  54 +++
 .../{FlatmapFlatmap.java => FlatMapFlatMap.java}   |   4 +-
 .../{FlatmapFlatmap.java => UnfoldFlatMap.java}|  12 +-
 .../map/ConstantMap.java}  |  41 +--
 .../map/IncrMap.java}  |  26 +-
 .../map/LoopsMap.java} |  26 +-
 .../map/ValueMap.java} |  43 +--
 .../reduce/CountReduce.java}   |  38 +-
 .../reduce/SumReduce.java} |  39 +--
 .../strategy/decoration/ExplainStrategy.java   |   2 +-
 .../strategy/finalization/CoefficientStrategy.java |   2 +-
 .../strategy/optimization/IdentityStrategy.java|   2 +-
 .../{COTraverserFactory.java => CTraverser.java}   |  20 +-
 .../{COTraverserFactory.java => OTraverser.java}   |  19 +-
 .../{COTraverserFactory.java => PTraverser.java}   |  21 +-
 .../{COTraverserFactory.java => RTraverser.java}   |  19 +-
 .../machine/traverser/ShellTraverser.java  |  84 -
 .../traverser/{ => species}/COPTraverser.java  |   5 +-
 .../{ => species}/COPTraverserFactory.java |   4 +-
 .../traverser/{ => species}/CORTraverser.java  |   3 +-
 .../{ => species}/CORTraverserFactory.java |   4 +-
 .../traverser/{ => species}/COTraverser.java   |   3 +-
 .../{ => species}/COTraverserFactory.java  |   4 +-
 .../traverser/{ => species}/EmptyTraverser.java|   3 +-
 .../machine/traverser/TraverserSetTest.java|   1 +
 .../tinkerpop/machine/processor/beam/Beam.java |   2 +-
 .../processor/beam/strategy/BeamStrategy.java  |   2 +-
 .../tinkerpop/machine/processor/beam/BeamTest.java |   9 +-
 .../machine/processor/pipes/EmptyStep.java |   2 +-
 .../processor/pipes/strategy/PipesStrategy.java|   2 +-
 .../machine/processor/pipes/PipesTest.java |   9 +-
 .../provider/BlueprintsVerticesStrategy.java   |   2 +-
 49 files changed, 1036 insertions(+), 823 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.java
new file mode 100644
index 000..5430215
--- /dev/null
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/AbstractTraversal.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
+ *
+ *

[tinkerpop] branch tp4 updated: incr is gone. random other tweaks. taking a break.

2019-03-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 201d03d  incr is gone. random other tweaks. taking a break.
201d03d is described below

commit 201d03d8ef1e398c49079bd0e432f876f3ae4107
Author: Marko A. Rodriguez 
AuthorDate: Sat Mar 23 10:38:32 2019 -0600

incr is gone. random other tweaks. taking a break.
---
 .../tinkerpop/language/gremlin/Traversal.java  |  7 +++-
 .../tinkerpop/machine/bytecode/CoreCompiler.java   |  9 ++---
 .../tinkerpop/machine/bytecode/MethodArgument.java |  8 -
 .../machine/function/filter/FilterFilter.java  |  4 ++-
 .../tinkerpop/machine/function/map/IncrMap.java| 42 --
 .../machine/processor/pipes/PipesTest.java |  9 +++--
 6 files changed, 26 insertions(+), 53 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index c7214bd..8bc4e92 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -211,7 +211,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal incr() {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.INCR);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.MAP, 
"number::add", 1L);
 return (Traversal) this;
 }
 
@@ -280,6 +280,11 @@ public class Traversal implements Iterator {
 return this;
 }
 
+public  Traversal value(final K key) {
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.MAP, 
"dictionary::get", key);
+return (Traversal) this;
+}
+
 ///
 
 private final void prepareTraversal() {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
index 359e6e7..32590a0 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
@@ -27,7 +27,6 @@ import 
org.apache.tinkerpop.machine.function.branch.RepeatBranch;
 import org.apache.tinkerpop.machine.function.filter.FilterFilter;
 import org.apache.tinkerpop.machine.function.flatmap.FlatmapFlatmap;
 import org.apache.tinkerpop.machine.function.initial.InjectInitial;
-import org.apache.tinkerpop.machine.function.map.IncrMap;
 import org.apache.tinkerpop.machine.function.map.MapMap;
 import org.apache.tinkerpop.machine.function.map.PathMap;
 import org.apache.tinkerpop.machine.function.reduce.GroupCountReduce;
@@ -59,7 +58,6 @@ public final class CoreCompiler implements BytecodeCompiler {
 put(Symbols.FILTER, FunctionType.FILTER);
 put(Symbols.FLATMAP, FunctionType.FLATMAP);
 put(Symbols.GROUP_COUNT, FunctionType.REDUCE);
-put(Symbols.INCR, FunctionType.MAP);
 put(Symbols.INITIAL, FunctionType.INITIAL);
 put(Symbols.JOIN, FunctionType.BARRIER);
 put(Symbols.MAP, FunctionType.MAP);
@@ -80,15 +78,15 @@ public final class CoreCompiler implements BytecodeCompiler 
{
 case Symbols.BRANCH:
 return new BranchBranch<>(coefficient, labels, 
BranchBranch.makeBranches(instruction.args()));
 case Symbols.FILTER:
-return new FilterFilter<>(coefficient, labels, 
Pred.valueOf(instruction.args()[0]), 
Argument.create(Arrays.copyOfRange(instruction.args(), 1, 
instruction.args().length)));
+return instruction.args().length == 1 ?
+new FilterFilter<>(coefficient, labels, null, 
Argument.create(instruction.args())) :
+new FilterFilter<>(coefficient, labels, 
Pred.valueOf(instruction.args()[0]), 
Argument.create(Arrays.copyOfRange(instruction.args(), 1, 
instruction.args().length)));
 case Symbols.FLATMAP:
 return new FlatmapFlatmap<>(coefficient, labels, 
Argument.create(instruction.args()));
 case Symbols.GROUP_COUNT:
 return new GroupCountReduce<>(coefficient, labels, 
Compilation.compileOrNull(0, instruction.args()));
 case Symbols.INITIAL:
 return new InjectInitial<>(coefficient, labels, 
instruction.args());
-case Symbols.INCR:
-return new IncrMap<>(coefficient, labels);
 case Symbols.JOIN:
 return new JoinBarrier<>(coefficient, labels, (Symbols.Tokens) 
instruction.

[tinkerpop] branch tp4 updated: identity() is now a map(traverser::object). Now Map, Filter, FlatMap, Reduce, and Branch are self-consistent.

2019-03-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 635eeca  identity() is now a map(traverser::object). Now Map, Filter, 
FlatMap, Reduce, and Branch are self-consistent.
635eeca is described below

commit 635eecaf3be3cb37a550ef284ff54c65ab27dae9
Author: Marko A. Rodriguez 
AuthorDate: Sat Mar 23 09:06:42 2019 -0600

identity() is now a map(traverser::object). Now Map, Filter, FlatMap, 
Reduce, and Branch are self-consistent.
---
 .../main/java/org/apache/tinkerpop/language/gremlin/Traversal.java   | 2 +-
 .../java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java | 4 +---
 .../org/apache/tinkerpop/machine/function/filter/FilterFilter.java   | 4 +---
 .../tinkerpop/machine/strategy/optimization/IdentityStrategy.java| 5 +++--
 4 files changed, 6 insertions(+), 9 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 8525d92..c7214bd 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -191,7 +191,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal identity() {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
Boolean.TRUE);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.MAP, 
"traverser::object");
 return this;
 }
 
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
index c692651..359e6e7 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
@@ -80,9 +80,7 @@ public final class CoreCompiler implements BytecodeCompiler {
 case Symbols.BRANCH:
 return new BranchBranch<>(coefficient, labels, 
BranchBranch.makeBranches(instruction.args()));
 case Symbols.FILTER:
-return instruction.args().length == 1 ?
-new FilterFilter<>(coefficient, labels, null, 
Argument.create(instruction.args())) :
-new FilterFilter<>(coefficient, labels, 
Pred.valueOf(instruction.args()[0]), 
Argument.create(Arrays.copyOfRange(instruction.args(), 1, 
instruction.args().length)));
+return new FilterFilter<>(coefficient, labels, 
Pred.valueOf(instruction.args()[0]), 
Argument.create(Arrays.copyOfRange(instruction.args(), 1, 
instruction.args().length)));
 case Symbols.FLATMAP:
 return new FlatmapFlatmap<>(coefficient, labels, 
Argument.create(instruction.args()));
 case Symbols.GROUP_COUNT:
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/filter/FilterFilter.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/filter/FilterFilter.java
index d93a420..df39450 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/filter/FilterFilter.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/function/filter/FilterFilter.java
@@ -45,9 +45,7 @@ public final class FilterFilter extends 
AbstractFunction implements Fil
 
 @Override
 public boolean test(final Traverser traverser) {
-return null == this.pred ?
-this.argument.filterArg(traverser) :
-this.pred.test(traverser.object(), 
this.argument.mapArg(traverser));
+return this.pred.test(traverser.object(), 
this.argument.mapArg(traverser));
 }
 
 @Override
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/optimization/IdentityStrategy.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/optimization/IdentityStrategy.java
index 6c1bd99..db223b9 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/optimization/IdentityStrategy.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/optimization/IdentityStrategy.java
@@ -31,8 +31,9 @@ public final class IdentityStrategy extends 
AbstractStrategy void apply(final Bytecode bytecode) {
 bytecode.getInstructions().removeIf(instruction ->
-instruction.op().equals(CoreCompiler.Symbols.FILTER) &&
-Boolean.TRUE.equals(instruction.arg

[tinkerpop] branch tp4 updated: has() and hasKey() are now primitive instructions. All that is left is groupCount and path.

2019-03-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 99ca3bd  has() and hasKey() are now primitive instructions. All that 
is left is groupCount and path.
99ca3bd is described below

commit 99ca3bd36237ba1779474989fa064de8b0a48f9e
Author: Marko A. Rodriguez 
AuthorDate: Sat Mar 23 08:53:13 2019 -0600

has() and hasKey() are now primitive instructions. All that is left is 
groupCount and path.
---
 .../tinkerpop/language/gremlin/Traversal.java  | 46 +++
 .../tinkerpop/machine/bytecode/Argument.java   | 13 +++--
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 22 ++--
 .../tinkerpop/machine/bytecode/MethodArgument.java | 19 ++-
 .../machine/function/filter/FilterFilter.java  |  8 +--
 .../machine/function/filter/HasKeyFilter.java  | 65 --
 .../machine/function/filter/HasKeyValueFilter.java | 56 ---
 .../machine/function/reduce/ReduceReduce.java  |  7 +--
 .../machine/processor/pipes/PipesTest.java |  5 ++
 9 files changed, 75 insertions(+), 166 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 28a93c1..8525d92 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -103,7 +103,8 @@ public class Traversal implements Iterator {
 }
 
 public Traversal count() {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.REDUCE, 
"traverser::count", Oper.sum.name(), 0L);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.MAP, 
"traverser::count");
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.REDUCE, 
Oper.sum.name(), 0L);
 return (Traversal) this;
 }
 
@@ -134,37 +135,58 @@ public class Traversal implements Iterator {
 }
 
 public  Traversal> hasKey(final P predicate) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.HAS_KEY, 
predicate.type().name(), TraversalUtil.tryToGetBytecode(predicate.object()));
+final Bytecode internal = new Bytecode<>();
+internal.addInstruction(this.currentCoefficient, Symbols.FLATMAP, 
"dictionary::keys");
+internal.addInstruction(this.currentCoefficient, Symbols.FILTER, 
predicate.type().name(), TraversalUtil.tryToGetBytecode(predicate.object()));
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
internal);
 return (Traversal) this;
 }
 
 public  Traversal> hasKey(final K key) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.HAS_KEY, 
Pred.eq.name(), key);
+final Bytecode internal = new Bytecode<>();
+internal.addInstruction(this.currentCoefficient, Symbols.FLATMAP, 
"dictionary::keys");
+internal.addInstruction(this.currentCoefficient, Symbols.FILTER, 
Pred.eq.name(), key);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
internal);
 return (Traversal) this;
 }
 
 public  Traversal> hasKey(final Traversal, K> keyTraversal) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.HAS_KEY, 
Pred.eq.name(), keyTraversal.bytecode);
+final Bytecode internal = new Bytecode<>();
+internal.addInstruction(this.currentCoefficient, Symbols.FLATMAP, 
"dictionary::keys");
+internal.addInstruction(this.currentCoefficient, Symbols.FILTER, 
Pred.eq.name(), keyTraversal.bytecode);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
internal);
 return (Traversal) this;
 }
 
 public  Traversal> has(final K key, final V value) {
-this.bytecode.addInstruction(this.currentCoefficient, 
Symbols.HAS_KEY_VALUE, key, value);
+final Bytecode internal = new Bytecode<>();
+internal.addInstruction(this.currentCoefficient, Symbols.MAP, 
"dictionary::get", TraversalUtil.tryToGetBytecode(key));
+internal.addInstruction(this.currentCoefficient, Symbols.FILTER, 
Pred.eq.name(), value);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
internal);
 return (Traversal) this;
 }
 
 public  Traversal> has(final Traversal, 
K> keyTraversal, final V value) {
-this.bytecode.addInstruction(this.currentCoefficient, 
Symbols.HAS_KEY_VALUE, keyTraversal.bytecode, value);
+final Bytecode internal = new Bytecode<>();
+internal.addInstruction(this.currentCoefficient, Sy

[tinkerpop] branch tp4 updated: really widdling down the functions... this is so crazy.

2019-03-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 38d8f8e  really widdling down the functions... this is so crazy.
38d8f8e is described below

commit 38d8f8e316522578cab85c61c664301182b0d22d
Author: Marko A. Rodriguez 
AuthorDate: Sat Mar 23 08:04:38 2019 -0600

really widdling down the functions... this is so crazy.
---
 .../tinkerpop/language/gremlin/Traversal.java  | 19 
 .../tinkerpop/machine/bytecode/Argument.java   | 36 ---
 .../BytecodeArgument.java} | 28 ++-
 .../ConstantArgument.java} | 23 -
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 47 ++-
 .../MethodArgument.java}   | 31 ++---
 .../map/LoopsMap.java => bytecode/Oper.java}   | 29 ++--
 .../machine/function/filter/FilterFilter.java  | 12 -
 .../machine/function/filter/HasKeyValueFilter.java |  1 +
 .../machine/function/filter/IsFilter.java  | 54 --
 .../{UnfoldFlatMap.java => FlatmapFlatmap.java}| 12 +++--
 .../reduce/{SumReduce.java => ReduceReduce.java}   | 22 ++---
 .../machine/processor/pipes/PipesTest.java |  6 +--
 13 files changed, 124 insertions(+), 196 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 783f35f..28a93c1 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -22,6 +22,7 @@ import org.apache.tinkerpop.machine.bytecode.Bytecode;
 import org.apache.tinkerpop.machine.bytecode.BytecodeUtil;
 import org.apache.tinkerpop.machine.bytecode.Compilation;
 import org.apache.tinkerpop.machine.bytecode.CoreCompiler.Symbols;
+import org.apache.tinkerpop.machine.bytecode.Oper;
 import org.apache.tinkerpop.machine.bytecode.Pred;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.coefficient.LongCoefficient;
@@ -86,12 +87,12 @@ public class Traversal implements Iterator {
 return this;
 }
 
-public  Traversal choose(final Traversal predicate, 
final Traversal trueTraversal, final Traversal 
falseTraversal) {
+public  Traversal choose(final Traversal predicate, 
final Traversal trueTraversal, final Traversal 
falseTraversal) {
 this.bytecode.addInstruction(this.currentCoefficient, Symbols.BRANCH, 
predicate.bytecode, trueTraversal.bytecode, Symbols.DEFAULT, 
falseTraversal.bytecode);
 return (Traversal) this;
 }
 
-public  Traversal choose(final Traversal predicate, 
final Traversal trueTraversal) {
+public  Traversal choose(final Traversal predicate, 
final Traversal trueTraversal) {
 this.bytecode.addInstruction(this.currentCoefficient, Symbols.BRANCH, 
predicate.bytecode, trueTraversal.bytecode);
 return (Traversal) this;
 }
@@ -102,7 +103,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal count() {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.COUNT);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.REDUCE, 
"traverser::count", Oper.sum.name(), 0L);
 return (Traversal) this;
 }
 
@@ -173,17 +174,17 @@ public class Traversal implements Iterator {
 }
 
 public Traversal is(final E object) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.IS, 
Pred.eq.name(), object);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
"traverser::object", Pred.eq.name(), object);
 return this;
 }
 
 public Traversal is(final Traversal objectTraversal) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.IS, 
Pred.eq.name(), objectTraversal.bytecode);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
"traverser::object", Pred.eq.name(), objectTraversal.bytecode);
 return this;
 }
 
 public Traversal is(final P predicate) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.IS, 
predicate.type().name(), TraversalUtil.tryToGetBytecode(predicate.object()));
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
"traverser::object", predicate.type().name(), 
TraversalUtil.tryToGetBytecode(predicate.object()));
 return this;
 }
 
@@ -198,7 +199,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal loops() {
-this.bytecode.addInstruction(this.currentCoefficient, Symb

[tinkerpop] branch tp4 updated: per discussions with @dkuppitz, working towards making bytecode map, flatmap, branch, reduce, filter instructions ONLY. The TP4 Machine supports a very tiny instruction

2019-03-23 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 5b2934a  per discussions with @dkuppitz, working towards making 
bytecode map, flatmap, branch, reduce, filter instructions ONLY. The TP4 
Machine supports a very tiny instruction set that can, at compile time, be 
strategized into processor specific functions (optimizations). This is crazy 
stuff, but if it works, it will be brilliant.
5b2934a is described below

commit 5b2934af091e5acdccfeaa8b3e602dc1456a1cd7
Author: Marko A. Rodriguez 
AuthorDate: Sat Mar 23 05:37:26 2019 -0600

per discussions with @dkuppitz, working towards making bytecode map, 
flatmap, branch, reduce, filter instructions ONLY. The TP4 Machine supports a 
very tiny instruction set that can, at compile time, be strategized into 
processor specific functions (optimizations). This is crazy stuff, but if it 
works, it will be brilliant.
---
 .../tinkerpop/language/gremlin/Traversal.java  | 12 ++--
 .../language/gremlin/TraversalSource.java  |  2 +-
 .../tinkerpop/machine/bytecode/Argument.java   |  6 +-
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 43 -
 .../tinkerpop/machine/function/BranchFunction.java |  5 +-
 .../machine/function/barrier/JoinBarrier.java  |  2 +-
 .../branch/{UnionBranch.java => BranchBranch.java} | 39 ++--
 .../machine/function/branch/IfBranch.java  | 72 --
 .../machine/function/filter/FilterFilter.java  | 12 ++--
 .../machine/function/filter/HasKeyFilter.java  |  4 +-
 .../machine/function/filter/HasKeyValueFilter.java |  2 +-
 .../machine/function/filter/IdentityFilter.java| 41 
 .../machine/function/filter/IsFilter.java  |  2 +-
 .../tinkerpop/machine/function/map/MapMap.java | 12 ++--
 .../strategy/decoration/ExplainStrategy.java   |  2 +-
 .../strategy/optimization/IdentityStrategy.java|  3 +-
 .../machine/bytecode/InstructionTest.java  | 10 +--
 .../machine/bytecode/SourceInstructionTest.java|  8 +--
 .../tinkerpop/machine/processor/beam/BranchFn.java | 36 +++
 .../machine/processor/beam/util/TopologyUtil.java  | 45 --
 .../tinkerpop/machine/processor/beam/BeamTest.java |  5 +-
 .../machine/processor/pipes/BranchStep.java| 30 +
 .../tinkerpop/machine/processor/pipes/Pipes.java   |  2 +-
 .../machine/processor/pipes/PipesTest.java |  7 ++-
 24 files changed, 152 insertions(+), 250 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 15adece..783f35f 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -87,17 +87,17 @@ public class Traversal implements Iterator {
 }
 
 public  Traversal choose(final Traversal predicate, 
final Traversal trueTraversal, final Traversal 
falseTraversal) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.IF, 
predicate.bytecode, trueTraversal.bytecode, falseTraversal.bytecode);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.BRANCH, 
predicate.bytecode, trueTraversal.bytecode, Symbols.DEFAULT, 
falseTraversal.bytecode);
 return (Traversal) this;
 }
 
 public  Traversal choose(final Traversal predicate, 
final Traversal trueTraversal) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.IF, 
predicate.bytecode, trueTraversal.bytecode);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.BRANCH, 
predicate.bytecode, trueTraversal.bytecode);
 return (Traversal) this;
 }
 
 public  Traversal constant(final R constant) {
-this.bytecode.addInstruction(this.currentCoefficient, 
Symbols.CONSTANT, constant);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.MAP, 
constant);
 return (Traversal) this;
 }
 
@@ -168,7 +168,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal identity() {
-this.bytecode.addInstruction(this.currentCoefficient, 
Symbols.IDENTITY);
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
Boolean.TRUE);
 return this;
 }
 
@@ -243,12 +243,12 @@ public class Traversal implements Iterator {
 // TODO: for some reason var args are not working...Java11
 
 public  Traversal union(final Traversal traversalA, 
final Traversal traversalB) {
-this.bytecode.addInstruction(this.currentCoefficient, Symbols.UNION, 
traversalA.bytecode, traversalB.bytecode);
+this.bytecode.addInstruct

[tinkerpop] branch tp4 updated: added hashCode() and equals() to Instruction and SourceInstruction. Wrote unit tests cases validating methods and hash/equals.

2019-03-22 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 43ab07f  added hashCode() and equals() to Instruction and 
SourceInstruction. Wrote unit tests cases validating methods and hash/equals.
43ab07f is described below

commit 43ab07f5cd525897ae81184e8cdd75421f5cb4ff
Author: Marko A. Rodriguez 
AuthorDate: Fri Mar 22 14:12:13 2019 -0600

added hashCode() and equals() to Instruction and SourceInstruction. Wrote 
unit tests cases validating methods and hash/equals.
---
 .../tinkerpop/machine/bytecode/Bytecode.java   | 16 +-
 .../tinkerpop/machine/bytecode/Instruction.java| 16 ++
 .../machine/bytecode/SourceInstruction.java| 17 +-
 .../machine/bytecode/InstructionTest.java  | 61 ++
 .../machine/bytecode/SourceInstructionTest.java| 57 
 5 files changed, 164 insertions(+), 3 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
index b0bf82f..bfcb930 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
@@ -26,10 +26,10 @@ import java.util.List;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class Bytecode implements Cloneable {
+public final class Bytecode implements Cloneable { // todo: serializable?
 
-private List> instructions = new ArrayList<>();
 private List sourceInstructions = new ArrayList<>();
+private List> instructions = new ArrayList<>();
 
 public void addSourceInstruction(final String op, final Object... args) {
 this.sourceInstructions.add(new SourceInstruction(op, args));
@@ -60,6 +60,18 @@ public final class Bytecode implements Cloneable {
 }
 
 @Override
+public int hashCode() {
+return this.sourceInstructions.hashCode() ^ 
this.instructions.hashCode();
+}
+
+@Override
+public boolean equals(final Object object) {
+return object instanceof Bytecode &&
+this.instructions.equals(((Bytecode) object).instructions) &&
+this.sourceInstructions.equals(((Bytecode) 
object).sourceInstructions);
+}
+
+@Override
 public String toString() {
 return this.instructions.toString();
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
index 4136d2e..1c08a58 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Instruction.java
@@ -73,6 +73,22 @@ public final class Instruction {
 }
 
 @Override
+public int hashCode() {
+return this.coefficient.hashCode() ^ this.op.hashCode() ^ 
Arrays.hashCode(this.args) ^ this.labels.hashCode();
+}
+
+@Override
+public boolean equals(final Object object) {
+if (!(object instanceof Instruction))
+return false;
+final Instruction other = (Instruction) object;
+return this.op.equals(other.op) &&
+Arrays.equals(this.args, other.args) &&
+this.coefficient.equals(other.coefficient) &&
+this.labels.equals(other.labels);
+}
+
+@Override
 public String toString() {
 return StringFactory.makeInstructionString(this);
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/SourceInstruction.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/SourceInstruction.java
index ce6c944..6b484fc 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/SourceInstruction.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/SourceInstruction.java
@@ -20,6 +20,8 @@ package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.util.StringFactory;
 
+import java.util.Arrays;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -28,7 +30,7 @@ public final class SourceInstruction {
 private final String op;
 private final Object[] args;
 
-public SourceInstruction(final String op, final Object[] args) {
+public SourceInstruction(final String op, final Object... args) {
 this.op = op;
 this.args = args;
 }
@@ -42,6 +44,19 @@ public final class Sour

[tinkerpop] branch tp4 updated: SourceStep toString() and TODO regarding how to get functions vs. execution plans from Compilations.

2019-03-22 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 11169d5  SourceStep toString() and TODO regarding how to get functions 
vs. execution plans from Compilations.
11169d5 is described below

commit 11169d55f1a4116096846d9f675e43de8c512619
Author: Marko A. Rodriguez 
AuthorDate: Fri Mar 22 08:16:45 2019 -0600

SourceStep toString() and TODO regarding how to get functions vs. execution 
plans from Compilations.
---
 .../main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java| 2 +-
 .../java/org/apache/tinkerpop/machine/processor/pipes/SourceStep.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 087a75d..46cec90 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -127,7 +127,7 @@ public final class Compilation implements 
Serializable {
 @Override
 public String toString() {
 return this.functions.toString();
-}
+} // TODO: functions need access to compilations for nesting
 
 
 
diff --git 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/SourceStep.java
 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/SourceStep.java
index 1229234..8bb0168 100644
--- 
a/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/SourceStep.java
+++ 
b/java/machine/processor/pipes/src/main/java/org/apache/tinkerpop/machine/processor/pipes/SourceStep.java
@@ -52,6 +52,6 @@ final class SourceStep implements Step {
 
 @Override
 public String toString() {
-return "Source";
+return SourceStep.class.getSimpleName();
 }
 }



[tinkerpop] branch tp4 updated: Coefficient.toString() is standardized using line brackets. Some minor tweaks. Done for the day.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 209279c  Coefficient.toString() is standardized using line brackets. 
Some minor tweaks. Done for the day.
209279c is described below

commit 209279c9eb264f590ff5cdf661e16c644701aaec
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 15:49:39 2019 -0600

Coefficient.toString() is standardized using line brackets. Some minor 
tweaks. Done for the day.
---
 .../tinkerpop/machine/bytecode/Compilation.java  |  2 +-
 .../tinkerpop/machine/bytecode/CoreCompiler.java |  4 
 .../machine/coefficient/LongCoefficient.java |  4 +++-
 .../apache/tinkerpop/machine/util/StringFactory.java | 11 ---
 .../tinkerpop/machine/processor/beam/ReduceFn.java   | 20 ++--
 .../processor/beam/serialization/ReducerCoder.java   | 10 +-
 .../{BasicReducer.java => InMemoryReducer.java}  |  8 
 .../processor/beam/util/ExecutionPlanner.java|  2 +-
 .../machine/processor/beam/util/TopologyUtil.java|  2 +-
 9 files changed, 37 insertions(+), 26 deletions(-)

diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 397c53a..087a75d 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -52,7 +52,7 @@ public final class Compilation implements 
Serializable {
 this.structureFactory = 
BytecodeUtil.getStructureFactory(bytecode).orElse(EmptyStructure.instance());
 this.processorFactory = 
BytecodeUtil.getProcessorFactory(bytecode).get();
 this.traverserFactory = 
BytecodeUtil.getTraverserFactory(bytecode).get();
-this.functions = CompositeCompiler.compile(bytecode, 
Arrays.asList(CoreCompiler.instance(), 
this.structureFactory.getCompiler().orElse(new CoreCompiler(;
+this.functions = CompositeCompiler.compile(bytecode, 
Arrays.asList(CoreCompiler.instance(), 
this.structureFactory.getCompiler().orElse(CoreCompiler.instance(;
 }
 
 public Compilation(final ProcessorFactory processorFactory) {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
index 4b541b7..e875ff1 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
@@ -52,6 +52,10 @@ public final class CoreCompiler implements BytecodeCompiler {
 
 private static final CoreCompiler INSTANCE = new CoreCompiler();
 
+private CoreCompiler() {
+// static instance
+}
+
 public static final CoreCompiler instance() {
 return INSTANCE;
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/coefficient/LongCoefficient.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/coefficient/LongCoefficient.java
index d3a28b1..cd12836 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/coefficient/LongCoefficient.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/coefficient/LongCoefficient.java
@@ -18,6 +18,8 @@
  */
 package org.apache.tinkerpop.machine.coefficient;
 
+import org.apache.tinkerpop.machine.util.StringFactory;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -80,7 +82,7 @@ public class LongCoefficient implements Coefficient {
 
 @Override
 public String toString() {
-return Long.toString(this.value);
+return StringFactory.makeCoefficientString(this);
 }
 
 @Override
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/StringFactory.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/StringFactory.java
index 8d29250..c333885 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/StringFactory.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/util/StringFactory.java
@@ -20,6 +20,7 @@ package org.apache.tinkerpop.machine.util;
 
 import org.apache.tinkerpop.machine.bytecode.Instruction;
 import org.apache.tinkerpop.machine.bytecode.SourceInstruction;
+import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.CFunction;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 
@@ -36,10 +37,14 @@ public final cl

[tinkerpop] branch tp4 updated: fixed up the toString() of Beams execution plan. Tweaked on ExplainStrategy some more.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new cf832e7  fixed up the toString() of Beams execution plan. Tweaked on 
ExplainStrategy some more.
cf832e7 is described below

commit cf832e7a3eb1b9659098593c0bb838201c1822dc
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 15:01:16 2019 -0600

fixed up the toString() of Beams execution plan. Tweaked on ExplainStrategy 
some more.
---
 .../org/apache/tinkerpop/language/gremlin/Traversal.java  |  2 +-
 .../machine/strategy/decoration/ExplainStrategy.java  |  9 ++---
 .../org/apache/tinkerpop/machine/processor/beam/Beam.java |  4 ++--
 .../util/{PrimitiveVisitor.java => ExecutionPlanner.java} | 15 ---
 .../apache/tinkerpop/machine/processor/beam/BeamTest.java |  6 ++
 5 files changed, 19 insertions(+), 17 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 6b1dd97..15adece 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -117,7 +117,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal explain() {
-this.bytecode.addSourceInstruction(Symbols.WITH_STRATEGY, 
ExplainStrategy.class);
+this.bytecode.addSourceInstruction(Symbols.WITH_STRATEGY, 
ExplainStrategy.class); // TODO: maybe its best to have this in the global cache
 this.bytecode.addInstruction(this.currentCoefficient, Symbols.EXPLAIN);
 return (Traversal) this;
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/decoration/ExplainStrategy.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/decoration/ExplainStrategy.java
index 44d7ded..9550274 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/decoration/ExplainStrategy.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/strategy/decoration/ExplainStrategy.java
@@ -41,16 +41,19 @@ public final class ExplainStrategy extends 
AbstractStrategy void apply(final Bytecode bytecode) {
 if (bytecode.lastInstruction().op().equals(Symbols.EXPLAIN)) {
 bytecode.getInstructions().remove(bytecode.lastInstruction());
+bytecode.getSourceInstructions().removeIf(instruction ->
+instruction.op().equals(Symbols.WITH_STRATEGY) &&
+
instruction.args()[0].equals(ExplainStrategy.class));
 final Bytecode clone = bytecode.clone();
 bytecode.getInstructions().clear();
 bytecode.addInstruction(
 BytecodeUtil.getCoefficient(clone).get(),
 Symbols.INJECT,
-ExplainStrategy.processBytecode(clone));
+ExplainStrategy.explainBytecode(clone));
 }
 }
 
-private static  String processBytecode(final Bytecode bytecode) {
+private static  String explainBytecode(final Bytecode bytecode) {
 final Map explain = new LinkedHashMap<>();
 explain.put(ORIGINAL, bytecode.toString());
 for (final Strategy strategy : BytecodeUtil.getStrategies(bytecode)) {
@@ -78,7 +81,7 @@ public final class ExplainStrategy extends 
AbstractStrategy implements Processor {
 
 @Override
 public String toString() {
-final PrimitiveVisitor visitor = new PrimitiveVisitor();
+final ExecutionPlanner visitor = new ExecutionPlanner();
 this.pipeline.traverseTopologically(visitor);
 return visitor.toString();
 }
diff --git 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/PrimitiveVisitor.java
 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/ExecutionPlanner.java
similarity index 70%
rename from 
java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/PrimitiveVisitor.java
rename to 
java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/ExecutionPlanner.java
index 6a5f4e2..9162044 100644
--- 
a/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/PrimitiveVisitor.java
+++ 
b/java/machine/processor/beam/src/main/java/org/apache/tinkerpop/machine/processor/beam/util/ExecutionPlanner.java
@@ -20,10 +20,7 @@ package org.apache.tinkerpop.machine.processor.beam.util;
 
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms

[tinkerpop] branch tp4 updated: a better explain() model using DecorationStrategy.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 33d48d2  a better explain() model using DecorationStrategy.
33d48d2 is described below

commit 33d48d21d82d5b772a96fd00fc3ab7e596d01c7f
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 14:36:49 2019 -0600

a better explain() model using DecorationStrategy.
---
 .../tinkerpop/language/gremlin/Traversal.java  |  2 +
 .../tinkerpop/machine/bytecode/Compilation.java| 20 ++---
 .../machine/processor/ExplainProcessor.java| 82 ---
 .../strategy/decoration/ExplainStrategy.java   | 92 ++
 .../machine/processor/pipes/AbstractStep.java  |  2 +-
 5 files changed, 100 insertions(+), 98 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index af444ec..6b1dd97 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -25,6 +25,7 @@ import 
org.apache.tinkerpop.machine.bytecode.CoreCompiler.Symbols;
 import org.apache.tinkerpop.machine.bytecode.Pred;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.coefficient.LongCoefficient;
+import org.apache.tinkerpop.machine.strategy.decoration.ExplainStrategy;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.path.Path;
 
@@ -116,6 +117,7 @@ public class Traversal implements Iterator {
 }
 
 public Traversal explain() {
+this.bytecode.addSourceInstruction(Symbols.WITH_STRATEGY, 
ExplainStrategy.class);
 this.bytecode.addInstruction(this.currentCoefficient, Symbols.EXPLAIN);
 return (Traversal) this;
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 2e9b8df..397c53a 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -19,14 +19,12 @@
 package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.function.CFunction;
-import org.apache.tinkerpop.machine.processor.ExplainProcessor;
 import org.apache.tinkerpop.machine.processor.FilterProcessor;
 import org.apache.tinkerpop.machine.processor.LoopsProcessor;
 import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
 import org.apache.tinkerpop.machine.structure.EmptyStructure;
 import org.apache.tinkerpop.machine.structure.StructureFactory;
-import org.apache.tinkerpop.machine.traverser.COTraverserFactory;
 import org.apache.tinkerpop.machine.traverser.Traverser;
 import org.apache.tinkerpop.machine.traverser.TraverserFactory;
 
@@ -50,19 +48,11 @@ public final class Compilation implements 
Serializable {
 private transient Processor processor;
 
 public Compilation(final Bytecode bytecode) {
-if 
(bytecode.lastInstruction().op().equals(CoreCompiler.Symbols.EXPLAIN)) {
-this.processorFactory = new ExplainProcessor(bytecode);
-this.traverserFactory = null;
-this.functions = Collections.emptyList();
-this.structureFactory = null;
-
-} else {
-BytecodeUtil.strategize(bytecode);
-this.structureFactory = 
BytecodeUtil.getStructureFactory(bytecode).orElse(EmptyStructure.instance());
-this.processorFactory = 
BytecodeUtil.getProcessorFactory(bytecode).get();
-this.traverserFactory = 
BytecodeUtil.getTraverserFactory(bytecode).get();
-this.functions = CompositeCompiler.compile(bytecode, 
Arrays.asList(CoreCompiler.instance(), 
this.structureFactory.getCompiler().orElse(new CoreCompiler(;
-}
+BytecodeUtil.strategize(bytecode);
+this.structureFactory = 
BytecodeUtil.getStructureFactory(bytecode).orElse(EmptyStructure.instance());
+this.processorFactory = 
BytecodeUtil.getProcessorFactory(bytecode).get();
+this.traverserFactory = 
BytecodeUtil.getTraverserFactory(bytecode).get();
+this.functions = CompositeCompiler.compile(bytecode, 
Arrays.asList(CoreCompiler.instance(), 
this.structureFactory.getCompiler().orElse(new CoreCompiler(;
 }
 
 public Compilation(final ProcessorFactory processorFactory) {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/processor

[tinkerpop] branch tp4 updated: implemented a basic implementation of explain(). Some smart cleanup and it will be pretty slammin'.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 9e21a74  implemented a basic implementation of explain(). Some smart 
cleanup and it will be pretty slammin'.
9e21a74 is described below

commit 9e21a74e583b58c387e276ac8d8d436053bd8af5
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 12:40:24 2019 -0600

implemented a basic implementation of explain(). Some smart cleanup and it 
will be pretty slammin'.
---
 .../tinkerpop/language/gremlin/Traversal.java  |  9 ++-
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  8 +--
 .../tinkerpop/machine/bytecode/Compilation.java| 30 +---
 .../tinkerpop/machine/bytecode/CoreCompiler.java   |  1 +
 .../machine/processor/ExplainProcessor.java| 82 +
 .../machine/traverser/ShellTraverser.java  | 84 ++
 .../machine/processor/pipes/PipesTest.java |  5 +-
 .../blueprints/bytecode/BlueprintsCompiler.java|  7 +-
 .../provider/BlueprintsVerticesStrategy.java   |  2 +-
 9 files changed, 205 insertions(+), 23 deletions(-)

diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 7c27824..af444ec 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -42,6 +42,7 @@ public class Traversal implements Iterator {
 protected final Bytecode bytecode;
 private Compilation compilation;
 private Coefficient currentCoefficient;
+private boolean locked = false; // TODO: when a traversal has been 
submitted, we need to make sure new modulations can't happen.
 
 // iteration helpers
 private long lastCount = 0L;
@@ -114,6 +115,11 @@ public class Traversal implements Iterator {
 return this;
 }
 
+public Traversal explain() {
+this.bytecode.addInstruction(this.currentCoefficient, Symbols.EXPLAIN);
+return (Traversal) this;
+}
+
 public Traversal filter(final Traversal filterTraversal) 
{
 this.bytecode.addInstruction(this.currentCoefficient, Symbols.FILTER, 
filterTraversal.bytecode);
 return this;
@@ -293,7 +299,6 @@ public class Traversal implements Iterator {
 
 @Override
 public String toString() {
-this.prepareTraversal();
-return this.compilation.getProcessor().toString();
+return this.bytecode.toString();
 }
 }
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index d267b5c..e1ee051 100644
--- 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -46,7 +46,7 @@ public final class BytecodeUtil {
 }
 }
 
-private static  void strategize(final Bytecode bytecode, Strategy 
strategy) {
+public static  void strategize(final Bytecode bytecode, Strategy 
strategy) {
 strategy.apply(bytecode);
 for (final Instruction instruction : bytecode.getInstructions()) {
 for (Object arg : instruction.args()) {
@@ -56,7 +56,7 @@ public final class BytecodeUtil {
 }
 }
 
-private static  Set> getStrategies(final Bytecode 
bytecode) {
+public static  Set> getStrategies(final Bytecode 
bytecode) {
 try {
 final Set> strategies = new HashSet<>();
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
@@ -97,7 +97,7 @@ public final class BytecodeUtil {
 }
 }
 
-static  Optional getProcessorFactory(final 
Bytecode bytecode) {
+public static  Optional getProcessorFactory(final 
Bytecode bytecode) {
 try {
 ProcessorFactory processor = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
@@ -111,7 +111,7 @@ public final class BytecodeUtil {
 }
 }
 
-static  Optional getStructureFactory(final 
Bytecode bytecode) {
+public static  Optional getStructureFactory(final 
Bytecode bytecode) {
 try {
 StructureFactory structure = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
diff --git 
a/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/machine/machine-core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 6732443.

[tinkerpop] branch tp4 updated: renamed tinkergraph to blueprints.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new dc628b8  renamed tinkergraph to blueprints.
dc628b8 is described below

commit dc628b8e14883f7c1cb089f074a679754ae7086a
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 11:35:12 2019 -0600

renamed tinkergraph to blueprints.
---
 .gitignore   |   1 +
 java/machine/structure/.DS_Store | Bin 6148 -> 0 bytes
 .../structure/{tinkergraph => blueprints}/pom.xml|   4 ++--
 .../machine/structure/blueprints/Blueprints.java}|   4 ++--
 .../structure/blueprints/BlueprintsStructure.java}   |  14 +++---
 .../blueprints/bytecode/BlueprintsCompiler.java} |   6 +++---
 .../structure/blueprints/data/BlueprintsVertex.java} |   4 ++--
 .../blueprints}/function/initial/VerticesFlatMap.java|   6 +++---
 .../strategy/provider/BlueprintsVerticesStrategy.java}   |   8 
 java/machine/structure/pom.xml   |   2 +-
 java/machine/structure/tinkergraph/.DS_Store | Bin 6148 -> 0 bytes
 11 files changed, 25 insertions(+), 24 deletions(-)

diff --git a/.gitignore b/.gitignore
index ce14a15..a50e5ea 100644
--- a/.gitignore
+++ b/.gitignore
@@ -4,3 +4,4 @@
 .idea/
 target/
 fileTemplates/
+.DS_Store
diff --git a/java/machine/structure/.DS_Store b/java/machine/structure/.DS_Store
deleted file mode 100644
index d5b57da..000
Binary files a/java/machine/structure/.DS_Store and /dev/null differ
diff --git a/java/machine/structure/tinkergraph/pom.xml 
b/java/machine/structure/blueprints/pom.xml
similarity index 95%
rename from java/machine/structure/tinkergraph/pom.xml
rename to java/machine/structure/blueprints/pom.xml
index 99bf3d4..7afa479 100644
--- a/java/machine/structure/tinkergraph/pom.xml
+++ b/java/machine/structure/blueprints/pom.xml
@@ -21,8 +21,8 @@ limitations under the License.
 org.apache.tinkerpop
 4.0.0-SNAPSHOT
 
-Apache TinkerPop :: Machine :: TinkerGraph
-tinkergraph
+Apache TinkerPop :: Machine :: Blueprints
+blueprints
 
 
 org.apache.tinkerpop
diff --git 
a/java/machine/structure/tinkergraph/src/main/java/org/apache/tinkerpop/machine/structure/tinkergraph/TinkerGraph.java
 
b/java/machine/structure/blueprints/src/main/java/org/apache/tinkerpop/machine/structure/blueprints/Blueprints.java
similarity index 89%
rename from 
java/machine/structure/tinkergraph/src/main/java/org/apache/tinkerpop/machine/structure/tinkergraph/TinkerGraph.java
rename to 
java/machine/structure/blueprints/src/main/java/org/apache/tinkerpop/machine/structure/blueprints/Blueprints.java
index 58c24cd..8d57792 100644
--- 
a/java/machine/structure/tinkergraph/src/main/java/org/apache/tinkerpop/machine/structure/tinkergraph/TinkerGraph.java
+++ 
b/java/machine/structure/blueprints/src/main/java/org/apache/tinkerpop/machine/structure/blueprints/Blueprints.java
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine.structure.tinkergraph;
+package org.apache.tinkerpop.machine.structure.blueprints;
 
 import org.apache.tinkerpop.machine.structure.Structure;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public class TinkerGraph implements Structure {
+public class Blueprints implements Structure {
 
 
 }
diff --git 
a/java/machine/structure/tinkergraph/src/main/java/org/apache/tinkerpop/machine/structure/tinkergraph/TinkerGraphStructure.java
 
b/java/machine/structure/blueprints/src/main/java/org/apache/tinkerpop/machine/structure/blueprints/BlueprintsStructure.java
similarity index 75%
rename from 
java/machine/structure/tinkergraph/src/main/java/org/apache/tinkerpop/machine/structure/tinkergraph/TinkerGraphStructure.java
rename to 
java/machine/structure/blueprints/src/main/java/org/apache/tinkerpop/machine/structure/blueprints/BlueprintsStructure.java
index 4f00303..3fda204 100644
--- 
a/java/machine/structure/tinkergraph/src/main/java/org/apache/tinkerpop/machine/structure/tinkergraph/TinkerGraphStructure.java
+++ 
b/java/machine/structure/blueprints/src/main/java/org/apache/tinkerpop/machine/structure/blueprints/BlueprintsStructure.java
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine.structure.tinkergraph;
+package org.apache.tinkerpop.machine.structure.blueprints;
 
 import org.apache.tinkerpop.machine.bytecode.BytecodeCompiler;
 import org.apache.tinkerpop.machine.strategy.Strategy;
 import org.apache.tinkerpop.machine.structure.Structure;
 import org.apache.tinkerpop.machine.structure.StructureFactory

[tinkerpop] branch tp4 updated: removed .DS_Store Mac hidden files. Added it to gitignore too.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 2ad56a4  removed .DS_Store Mac hidden files. Added it to gitignore too.
2ad56a4 is described below

commit 2ad56a48430223bbfcff3a42772003270899954d
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 11:35:47 2019 -0600

removed .DS_Store Mac hidden files. Added it to gitignore too.
---
 java/machine/processor/.DS_Store   | Bin 6148 -> 0 bytes
 java/machine/processor/beam/.DS_Store  | Bin 6148 -> 0 bytes
 java/machine/processor/pipes/.DS_Store | Bin 6148 -> 0 bytes
 3 files changed, 0 insertions(+), 0 deletions(-)

diff --git a/java/machine/processor/.DS_Store b/java/machine/processor/.DS_Store
deleted file mode 100644
index b769da3..000
Binary files a/java/machine/processor/.DS_Store and /dev/null differ
diff --git a/java/machine/processor/beam/.DS_Store 
b/java/machine/processor/beam/.DS_Store
deleted file mode 100644
index 5008ddf..000
Binary files a/java/machine/processor/beam/.DS_Store and /dev/null differ
diff --git a/java/machine/processor/pipes/.DS_Store 
b/java/machine/processor/pipes/.DS_Store
deleted file mode 100644
index 5008ddf..000
Binary files a/java/machine/processor/pipes/.DS_Store and /dev/null differ



[tinkerpop] branch tp4 updated: removed core module. it is now machine/machine-core.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new c134a52  removed core module. it is now machine/machine-core.
c134a52 is described below

commit c134a526542e76b31706aefbf2f7e9200ed3da5b
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 11:23:42 2019 -0600

removed core module. it is now machine/machine-core.
---
 .../apache/tinkerpop/machine/TraversalTest.java| 35 --
 .../machine/bytecode/BytecodeUtilTest.java | 35 --
 java/language/gremlin/pom.xml  |  2 +-
 java/{core => machine/machine-core}/pom.xml| 13 ++--
 .../tinkerpop/machine/bytecode/Argument.java   |  0
 .../tinkerpop/machine/bytecode/Bytecode.java   |  0
 .../machine/bytecode/BytecodeCompiler.java |  0
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  0
 .../tinkerpop/machine/bytecode/Compilation.java|  0
 .../machine/bytecode/CompilationCircle.java|  0
 .../machine/bytecode/CompositeCompiler.java|  0
 .../tinkerpop/machine/bytecode/CoreCompiler.java   |  0
 .../tinkerpop/machine/bytecode/FunctionType.java   |  0
 .../tinkerpop/machine/bytecode/Instruction.java|  2 +-
 .../apache/tinkerpop/machine/bytecode/Pred.java|  2 +-
 .../machine/bytecode/SourceInstruction.java|  2 +-
 .../tinkerpop/machine/coefficient/Coefficient.java |  0
 .../machine/coefficient/LongCoefficient.java   |  0
 .../machine/function/AbstractFunction.java |  2 +-
 .../machine/function/BarrierFunction.java  |  0
 .../tinkerpop/machine/function/BranchFunction.java |  0
 .../tinkerpop/machine/function/CFunction.java  |  0
 .../tinkerpop/machine/function/FilterFunction.java |  0
 .../machine/function/FlatMapFunction.java  |  0
 .../machine/function/InitialFunction.java  |  0
 .../tinkerpop/machine/function/MapFunction.java|  0
 .../tinkerpop/machine/function/ReduceFunction.java |  0
 .../machine/function/barrier/JoinBarrier.java  |  2 +-
 .../machine/function/barrier/StallBarrier.java |  0
 .../machine/function/branch/IfBranch.java  |  2 +-
 .../machine/function/branch/RepeatBranch.java  |  2 +-
 .../machine/function/branch/UnionBranch.java   |  2 +-
 .../machine/function/filter/FilterFilter.java  |  3 +-
 .../machine/function/filter/HasKeyFilter.java  |  2 +-
 .../machine/function/filter/HasKeyValueFilter.java |  2 +-
 .../machine/function/filter/IdentityFilter.java|  0
 .../machine/function/filter/IsFilter.java  |  2 +-
 .../machine/function/flatmap/UnfoldFlatMap.java|  4 +--
 .../machine/function/initial/InjectInitial.java|  4 +--
 .../machine/function/map/ConstantMap.java  |  2 +-
 .../tinkerpop/machine/function/map/IncrMap.java|  0
 .../tinkerpop/machine/function/map/LoopsMap.java   |  0
 .../tinkerpop/machine/function/map/MapMap.java |  2 +-
 .../tinkerpop/machine/function/map/PathMap.java|  0
 .../machine/function/reduce/CountReduce.java   |  0
 .../machine/function/reduce/GroupCountReduce.java  |  2 +-
 .../machine/function/reduce/SumReduce.java |  2 +-
 .../machine/processor/ConstantProcessor.java   |  0
 .../machine/processor/FilterProcessor.java |  0
 .../machine/processor/HasNextProcessor.java|  0
 .../machine/processor/LoopsProcessor.java  |  0
 .../tinkerpop/machine/processor/Processor.java |  0
 .../machine/processor/ProcessorFactory.java|  0
 .../machine/processor/SimpleProcessor.java |  2 +-
 .../machine/strategy/AbstractStrategy.java |  0
 .../tinkerpop/machine/strategy/Strategy.java   |  0
 .../tinkerpop/machine/strategy/StrategyUtil.java   |  2 +-
 .../strategy/finalization/CoefficientStrategy.java |  0
 .../finalization/TraverserFactoryStrategy.java |  0
 .../strategy/optimization/IdentityStrategy.java|  0
 .../CoefficientVerificationStrategy.java   |  0
 .../machine/structure/EmptyStructure.java  |  0
 .../tinkerpop/machine/structure/Structure.java |  0
 .../machine/structure/StructureFactory.java|  0
 .../tinkerpop/machine/structure/data/TEdge.java|  2 +-
 .../tinkerpop/machine/structure/data/TElement.java |  0
 .../tinkerpop/machine/structure/data/TKV.java  |  0
 .../tinkerpop/machine/structure/data/TMap.java |  0
 .../tinkerpop/machine/structure/data/TVertex.java  |  2 +-
 .../tinkerpop/machine/traverser/COPTraverser.java  |  0
 .../machine/traverser/COPTraverserFactory.java |  0
 .../tinkerpop/machine/traverser/CORTraverser.java  |  0
 .../machine/traverser/CORTraverserFactory.java |  0
 .../tinkerpop/machine/traverser/COTraverser.java   |  2 +-
 .../machine/traverser/COTraverserFactory.java  |  0
 .../machine/traverser/EmptyTraverser.java  |  0
 .../tinker

[tinkerpop] branch tp4 updated: moved BeamTest to processor/beam package.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 438c706  moved BeamTest to processor/beam package.
438c706 is described below

commit 438c7065263040e855517a6e4f597072112d190e
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 10:13:24 2019 -0600

moved BeamTest to processor/beam package.
---
 .../beam/functions/TraversalSourceLibrary.java | 36 --
 .../machine/{ => processor}/beam/BeamTest.java |  2 +-
 2 files changed, 1 insertion(+), 37 deletions(-)

diff --git 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/beam/functions/TraversalSourceLibrary.java
 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/beam/functions/TraversalSourceLibrary.java
deleted file mode 100644
index efa826f..000
--- 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/beam/functions/TraversalSourceLibrary.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.tinkerpop.machine.beam.functions;
-
-import org.apache.tinkerpop.language.gremlin.Gremlin;
-import org.apache.tinkerpop.language.gremlin.TraversalSource;
-import org.apache.tinkerpop.machine.processor.beam.BeamProcessor;
-import org.apache.tinkerpop.machine.coefficient.LongCoefficient;
-import org.apache.tinkerpop.machine.strategy.optimization.IdentityStrategy;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class TraversalSourceLibrary {
-
-public static final TraversalSource[] LONG_SOURCES = new 
TraversalSource[]{
-Gremlin.traversal().withProcessor(BeamProcessor.class),
-
Gremlin.traversal().withCoefficient(LongCoefficient.class).withProcessor(BeamProcessor.class),
-
Gremlin.traversal().withProcessor(BeamProcessor.class).withStrategy(IdentityStrategy.class)};
-}
diff --git 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/beam/BeamTest.java
 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/BeamTest.java
similarity index 99%
rename from 
java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/beam/BeamTest.java
rename to 
java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/BeamTest.java
index 29c5369..111d189 100644
--- 
a/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/beam/BeamTest.java
+++ 
b/java/machine/processor/beam/src/test/java/org/apache/tinkerpop/machine/processor/beam/BeamTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine.beam;
+package org.apache.tinkerpop.machine.processor.beam;
 
 import org.apache.tinkerpop.language.gremlin.Gremlin;
 import org.apache.tinkerpop.language.gremlin.P;



[tinkerpop] branch tp4 updated: beam and pipes are now off of machine.processor package. Removed the (Java) postfix on all the pom.xmls.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 95e16c8  beam and pipes are now off of machine.processor package. 
Removed the (Java) postfix on all the pom.xmls.
95e16c8 is described below

commit 95e16c8f7f2311f9fadbccdb051516c883ca9644
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 10:06:50 2019 -0600

beam and pipes are now off of machine.processor package. Removed the (Java) 
postfix on all the pom.xmls.
---
 java/core/pom.xml  |  2 +-
 java/language/gremlin/pom.xml  |  2 +-
 java/language/pom.xml  |  2 +-
 java/machine/pom.xml   |  2 +-
 java/machine/processor/beam/pom.xml|  2 +-
 .../machine/{ => processor}/beam/AbstractFn.java   |  2 +-
 .../machine/{ => processor}/beam/Beam.java |  8 ++---
 .../{ => processor}/beam/BeamProcessor.java|  4 +--
 .../machine/{ => processor}/beam/BranchFn.java |  2 +-
 .../machine/{ => processor}/beam/FilterFn.java |  2 +-
 .../machine/{ => processor}/beam/FlatMapFn.java|  2 +-
 .../tinkerpop/machine/{ => processor}/beam/Fn.java |  2 +-
 .../machine/{ => processor}/beam/InitialFn.java|  2 +-
 .../machine/{ => processor}/beam/MapFn.java|  2 +-
 .../machine/{ => processor}/beam/OutputFn.java |  2 +-
 .../machine/{ => processor}/beam/ReduceFn.java |  8 ++---
 .../{ => processor}/beam/RepeatDeadEndFn.java  |  2 +-
 .../machine/{ => processor}/beam/RepeatEndFn.java  |  2 +-
 .../{ => processor}/beam/RepeatStartFn.java|  2 +-
 .../beam/serialization/CoefficientCoder.java   |  2 +-
 .../beam/serialization/ReducerCoder.java   |  4 +--
 .../beam/serialization/TraverserCoder.java |  2 +-
 .../beam/sideeffect/BasicReducer.java  |  4 +--
 .../beam/strategy/BeamStrategy.java|  4 +--
 .../beam/util/PrimitiveVisitor.java|  6 ++--
 .../{ => processor}/beam/util/TopologyUtil.java| 24 +++---
 .../apache/tinkerpop/machine/beam/BeamTest.java|  1 +
 .../beam/functions/TraversalSourceLibrary.java |  2 +-
 java/machine/processor/pipes/pom.xml   |  2 +-
 .../{ => processor}/pipes/AbstractStep.java|  2 +-
 .../machine/{ => processor}/pipes/BarrierStep.java |  6 ++--
 .../machine/{ => processor}/pipes/BranchStep.java  |  2 +-
 .../machine/{ => processor}/pipes/EmptyStep.java   |  2 +-
 .../machine/{ => processor}/pipes/FilterStep.java  |  2 +-
 .../machine/{ => processor}/pipes/FlatMapStep.java |  2 +-
 .../machine/{ => processor}/pipes/InitialStep.java |  2 +-
 .../machine/{ => processor}/pipes/MapStep.java |  2 +-
 .../machine/{ => processor}/pipes/Pipes.java   |  4 +--
 .../{ => processor}/pipes/PipesProcessor.java  |  4 +--
 .../machine/{ => processor}/pipes/ReduceStep.java  |  4 +--
 .../machine/{ => processor}/pipes/RepeatStep.java  |  2 +-
 .../machine/{ => processor}/pipes/SourceStep.java  |  2 +-
 .../machine/{ => processor}/pipes/Step.java|  2 +-
 .../pipes/strategy/PipesStrategy.java  |  4 +--
 .../{ => processor}/pipes/util/Barrier.java|  2 +-
 .../pipes/util/InMemoryBarrier.java|  2 +-
 .../pipes/util/InMemoryReducer.java|  2 +-
 .../{ => processor}/pipes/util/Reducer.java|  2 +-
 .../pipes/function/TraversalSourceLibrary.java | 37 --
 .../machine/{ => processor}/pipes/PipesTest.java   |  3 +-
 java/machine/processor/pom.xml |  2 +-
 java/machine/structure/pom.xml |  2 +-
 java/machine/structure/tinkergraph/pom.xml |  2 +-
 java/pom.xml   |  2 +-
 54 files changed, 83 insertions(+), 118 deletions(-)

diff --git a/java/core/pom.xml b/java/core/pom.xml
index 64c2071..bdb543f 100644
--- a/java/core/pom.xml
+++ b/java/core/pom.xml
@@ -21,7 +21,7 @@ limitations under the License.
 tinkerpop
 4.0.0-SNAPSHOT
 
-Apache TinkerPop (Java) :: Core
+Apache TinkerPop :: Core
 core
 
 ${basedir}/target
diff --git a/java/language/gremlin/pom.xml b/java/language/gremlin/pom.xml
index 30ee8dc..1c09cc6 100644
--- a/java/language/gremlin/pom.xml
+++ b/java/language/gremlin/pom.xml
@@ -21,7 +21,7 @@ limitations under the License.
 org.apache.tinkerpop
 4.0.0-SNAPSHOT
 
-Apache TinkerPop (Java) :: Language :: Gremlin
+Apache TinkerPop :: Language :: Gremlin
 gremlin
 
 
diff --git a/java/language/pom.xml b/java/language/pom.xml
index 3aaf916..3ef169e 100644
--- a/java/language/pom.xml
+++ b/java/language/pom.xml
@@ -22,7 +22,7 @@ limitations

[tinkerpop] branch tp4 updated: AbstractStrategy equals(), hashCode() implementations. other minor tweaks.

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new ea10595  AbstractStrategy equals(), hashCode() implementations. other 
minor tweaks.
ea10595 is described below

commit ea10595b1172a9ce9ae58b1aeaedb30967e7e91d
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 07:33:33 2019 -0600

AbstractStrategy equals(), hashCode() implementations. other minor tweaks.
---
 .../tinkerpop/machine/strategy/AbstractStrategy.java  | 10 ++
 .../apache/tinkerpop/machine/strategy/Strategy.java   |  4 ++--
 .../tinkerpop/machine/strategy/StrategyUtil.java  |  7 +--
 .../apache/tinkerpop/language/gremlin/Traversal.java  | 19 +++
 .../org/apache/tinkerpop/language/gremlin/__.java |  4 +---
 5 files changed, 29 insertions(+), 15 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/AbstractStrategy.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/AbstractStrategy.java
index 81118cc..fbe281d 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/AbstractStrategy.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/AbstractStrategy.java
@@ -27,4 +27,14 @@ public abstract class AbstractStrategy 
implements Strategy extends 
Serializable, Comparable getStrategyCategory() {
 return (Class) Strategy.class;
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/StrategyUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/StrategyUtil.java
index 48b51d7..9db77d4 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/StrategyUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/strategy/StrategyUtil.java
@@ -32,7 +32,11 @@ import java.util.stream.Collectors;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public class StrategyUtil {
+public final class StrategyUtil {
+
+private StrategyUtil() {
+// static instance
+}
 
 private final static List> STRATEGY_CATEGORIES = 
List.of(
 Strategy.DecorationStrategy.class,
@@ -51,7 +55,6 @@ public class StrategyUtil {
 MultiMap.put(strategiesByCategory, s.getStrategyCategory(), 
s.getClass());
 });
 
-
 //Initialize all the dependencies
 strategies.forEach(strategy -> {
 strategy.applyPrior().forEach(s -> {
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
index 3fba0e8..7c27824 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
@@ -42,18 +42,21 @@ public class Traversal implements Iterator {
 protected final Bytecode bytecode;
 private Compilation compilation;
 private Coefficient currentCoefficient;
-//
+
+// iteration helpers
 private long lastCount = 0L;
 private E lastObject = null;
 
-Traversal(final Coefficient unity, final Bytecode bytecode) {
-this.bytecode = bytecode;
-this.currentCoefficient = unity;
+// used by __
+Traversal() {
+this.bytecode = new Bytecode<>();
+this.currentCoefficient = 
BytecodeUtil.getCoefficient(this.bytecode).orElse((Coefficient) 
LongCoefficient.create()); // TODO: this will cause __ problems
 }
 
-Traversal(final Bytecode bytecode) {
+// used by TraversalSource
+Traversal(final Coefficient unity, final Bytecode bytecode) {
 this.bytecode = bytecode;
-this.currentCoefficient = 
BytecodeUtil.getCoefficient(this.bytecode).orElse((Coefficient) 
LongCoefficient.create()); // TODO: this will cause __ problems
+this.currentCoefficient = unity;
 }
 
 public Traversal as(final String label) {
@@ -71,8 +74,8 @@ public class Traversal implements Iterator {
 return this;
 }
 
-public  Traversal by(final R byObject) {
-this.bytecode.lastInstruction().addArg(byObject);
+public Traversal by(final String byString) {
+this.bytecode.lastInstruction().addArg(byString);
 return this;
 }
 
diff --git 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
index 5d8d0c5..73db277 100644
--- 
a/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
+++ 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
@@ -18,8 +18,6 @@
  */
 package org.apache.tinkerpop.language.gremlin;
 
-import org.apache.tinkerpop.machine.by

[tinkerpop] branch tp4 updated: fixed a cloning bug in TraversalSource. Made inject() and V() proper initial instructions. Added a LongCoefficientTest. Fixed a equals()/hashCode() bug in LongCoefficie

2019-03-19 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4afc6cd  fixed a cloning bug in TraversalSource. Made inject() and V() 
proper initial instructions. Added a LongCoefficientTest. Fixed a 
equals()/hashCode() bug in LongCoefficient.
4afc6cd is described below

commit 4afc6cdcf0f147f77c477857542f4ac897ae6c4a
Author: Marko A. Rodriguez 
AuthorDate: Tue Mar 19 07:16:41 2019 -0600

fixed a cloning bug in TraversalSource. Made inject() and V() proper 
initial instructions. Added a LongCoefficientTest. Fixed a equals()/hashCode() 
bug in LongCoefficient.
---
 .../tinkerpop/machine/bytecode/Bytecode.java   |  5 ++
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  4 ++
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 17 ++
 .../machine/coefficient/LongCoefficient.java   | 14 -
 .../machine/function/initial/InjectInitial.java|  4 +-
 .../{compiler => bytecode}/BytecodeUtilTest.java   |  2 +-
 .../machine/coefficient/LongCoefficientTest.java   | 69 ++
 .../tinkerpop/language/gremlin/Traversal.java  | 16 ++---
 .../language/gremlin/TraversalSource.java  | 59 +-
 9 files changed, 158 insertions(+), 32 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
index 2737b52..b0bf82f 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Bytecode.java
@@ -35,6 +35,11 @@ public final class Bytecode implements Cloneable {
 this.sourceInstructions.add(new SourceInstruction(op, args));
 }
 
+public void addUniqueSourceInstruction(final String op, final Object... 
args) {
+this.sourceInstructions.removeIf(instruction -> 
instruction.op().equals(op));
+this.sourceInstructions.add(new SourceInstruction(op, args));
+}
+
 public List getSourceInstructions() {
 return this.sourceInstructions;
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index 37f1f9f..d267b5c 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -139,6 +139,10 @@ public final class BytecodeUtil {
 bytecode.getInstructions().add(index, newInstruction);
 }
 
+public static  void removeSourceInstruction(final Bytecode bytecode, 
final String op) {
+bytecode.getSourceInstructions().removeIf(instruction -> 
instruction.op().equals(op));
+}
+
 static  Optional> getTraverserFactory(final 
Bytecode bytecode) {
 // TODO: make this real
 for (final Instruction instruction : bytecode.getInstructions()) {
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
index 8eeb5c0..b4b60e4 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
@@ -167,25 +167,42 @@ public final class CoreCompiler implements 
BytecodeCompiler {
 
 // INSTRUCTION OPS
 public static final String BARRIER = "barrier";
+// [if,[bc],[bc],?[bc]]
 public static final String IF = "if";
+// [constant, o]
 public static final String CONSTANT = "constant";
+// [count]
 public static final String COUNT = "count";
+// [filter, [bc]]
 public static final String FILTER = "filter";
+// [groupCount, ?[bc]]
 public static final String GROUP_COUNT = "groupCount";
+// [hasKey, pred, o | [bc]]
 public static final String HAS_KEY = "hasKey";
 public static final String HAS_KEY_VALUE = "hasKeyValue";
+// [identity]
 public static final String IDENTITY = "identity";
+// [incr]
 public static final String INCR = "incr";
+// [inject, o*]
 public static final String INJECT = "inject";
+// [is, pred, o | [bc]]
 public static final String IS = "is";
 public static final String JOIN = "join";
+// [loops]
 public static final String LOOPS = "loops";
+// [map, [bc]]
 public static final String MAP = "map";
 public static final String PATH =

[tinkerpop] branch tp4 updated: lots of reorg, cleanup, toStringing, ....

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 84a069c  lots of reorg, cleanup, toStringing, 
84a069c is described below

commit 84a069cb8d2bcb6129141d2cfc85ff683954517b
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 18:53:58 2019 -0600

lots of reorg, cleanup, toStringing, 
---
 .../tinkerpop/machine/bytecode/Compilation.java|  6 ++
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 25 --
 .../apache/tinkerpop/machine/bytecode/Pred.java|  2 +-
 .../branch/{ChooseBranch.java => IfBranch.java}| 10 -
 .../machine/function/reduce/GroupCountReduce.java  |  6 ++
 ...rFactoryStrategy.java => AbstractStrategy.java} | 14 
 .../strategy/finalization/CoefficientStrategy.java |  3 ++-
 .../finalization/TraverserFactoryStrategy.java |  3 ++-
 .../strategy/optimization/IdentityStrategy.java|  3 ++-
 .../CoefficientVerificationStrategy.java   |  3 ++-
 .../machine/traverser/path/BasicPath.java  |  2 +-
 .../org/apache/tinkerpop/util/StringFactory.java   | 14 +---
 .../tinkerpop/language/gremlin/Traversal.java  |  4 ++--
 .../org/apache/tinkerpop/language/gremlin/__.java  |  2 +-
 .../machine/beam/strategy/BeamStrategy.java|  3 ++-
 .../apache/tinkerpop/machine/pipes/ReduceStep.java |  5 -
 .../apache/tinkerpop/machine/pipes/SourceStep.java | 12 +--
 .../machine/pipes/strategy/PipesStrategy.java  |  3 ++-
 .../apache/tinkerpop/machine/pipes/PipesTest.java  |  2 +-
 .../tinkergraph/TinkerGraphStructure.java  |  4 ++--
 ...esStrategy.java => TinkerVerticesStrategy.java} |  3 ++-
 21 files changed, 73 insertions(+), 56 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 82f71dc..6732443 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -147,6 +147,12 @@ public final class Compilation implements 
Serializable {
 return Optional.empty();
 }
 
+public static  Compilation compileOrNull(final int 
index, final Object... args) {
+return args.length > index && args[index] instanceof Bytecode ?
+new Compilation<>((Bytecode) args[index]) :
+null;
+}
+
 public static  List> compile(final Object... 
args) {
 final List> compilations = new ArrayList<>();
 for (final Object arg : args) {
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
index 16d5823..8eeb5c0 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
@@ -22,7 +22,7 @@ import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.CFunction;
 import org.apache.tinkerpop.machine.function.barrier.JoinBarrier;
 import org.apache.tinkerpop.machine.function.barrier.StallBarrier;
-import org.apache.tinkerpop.machine.function.branch.ChooseBranch;
+import org.apache.tinkerpop.machine.function.branch.IfBranch;
 import org.apache.tinkerpop.machine.function.branch.RepeatBranch;
 import org.apache.tinkerpop.machine.function.branch.UnionBranch;
 import org.apache.tinkerpop.machine.function.filter.FilterFilter;
@@ -58,8 +58,7 @@ public final class CoreCompiler implements BytecodeCompiler {
 
 private static final Map OP_TYPES = new HashMap<>() 
{{
 put(Symbols.BARRIER, FunctionType.BARRIER);
-put(Symbols.CHOOSE_IF_THEN, FunctionType.BRANCH);
-put(Symbols.CHOOSE_IF_THEN_ELSE, FunctionType.BRANCH);
+put(Symbols.IF, FunctionType.BRANCH);
 put(Symbols.CONSTANT, FunctionType.MAP);
 put(Symbols.COUNT, FunctionType.REDUCE);
 put(Symbols.FILTER, FunctionType.FILTER);
@@ -89,16 +88,11 @@ public final class CoreCompiler implements BytecodeCompiler 
{
 switch (op) {
 case Symbols.BARRIER:
 return new StallBarrier<>(coefficient, labels, 1000);
-case Symbols.CHOOSE_IF_THEN:
-return new ChooseBranch<>(coefficient, labels,
+case Symbols.IF:
+return new IfBranch<>(coefficient, labels,
 Compilation.compileOne(instruction.args()[0]),
 Compilation.compileOne(instruction.args()[1]),
-null);
-case Symbols.CHOOSE_IF_THEN_ELSE:

[tinkerpop] branch tp4 updated: BytecodeCompiler.getOpType() added which says the function type of the instruction by its opcode.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 3ae25c9  BytecodeCompiler.getOpType() added which says the function 
type of the instruction by its opcode.
3ae25c9 is described below

commit 3ae25c99664004d44268059cda69d20f2d0bd194
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 15:43:18 2019 -0600

BytecodeCompiler.getOpType() added which says the function type of the 
instruction by its opcode.
---
 .../machine/bytecode/BytecodeCompiler.java |  2 +
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   | 15 +++-
 .../tinkerpop/machine/bytecode/Compilation.java|  3 +-
 .../machine/bytecode/CompositeCompiler.java| 22 +-
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 86 --
 .../{BytecodeCompiler.java => FunctionType.java}   | 16 +---
 .../CoefficientVerificationStrategy.java   | 13 +++-
 .../tinkergraph/bytecode/TinkerGraphCompiler.java  |  6 ++
 8 files changed, 83 insertions(+), 80 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
index 32af547..bd68642 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
@@ -37,4 +37,6 @@ public interface BytecodeCompiler {
 }
 
 public  CFunction compile(final Instruction instruction);
+
+public FunctionType getFunctionType(final String op);
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index 421285f..37f1f9f 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -29,7 +29,9 @@ import 
org.apache.tinkerpop.machine.traverser.CORTraverserFactory;
 import org.apache.tinkerpop.machine.traverser.TraverserFactory;
 
 import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Optional;
 import java.util.Set;
 
@@ -73,6 +75,13 @@ public final class BytecodeUtil {
 }
 }
 
+public static  CompositeCompiler getCompilers(final Bytecode 
bytecode) {
+final List compilers = new ArrayList<>();
+compilers.add(CoreCompiler.instance());
+BytecodeUtil.getStructureFactory(bytecode).ifPresent(f -> 
f.getCompiler().ifPresent(compilers::add));
+return CompositeCompiler.create(compilers);
+}
+
 public static  Optional> getCoefficient(final 
Bytecode bytecode) {
 try {
 Coefficient coefficient = null;
@@ -88,7 +97,7 @@ public final class BytecodeUtil {
 }
 }
 
-public static  Optional getProcessorFactory(final 
Bytecode bytecode) {
+static  Optional getProcessorFactory(final 
Bytecode bytecode) {
 try {
 ProcessorFactory processor = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
@@ -102,7 +111,7 @@ public final class BytecodeUtil {
 }
 }
 
-public static  Optional getStructureFactory(final 
Bytecode bytecode) {
+static  Optional getStructureFactory(final 
Bytecode bytecode) {
 try {
 StructureFactory structure = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
@@ -130,7 +139,7 @@ public final class BytecodeUtil {
 bytecode.getInstructions().add(index, newInstruction);
 }
 
-public static  Optional> getTraverserFactory(final 
Bytecode bytecode) {
+static  Optional> getTraverserFactory(final 
Bytecode bytecode) {
 // TODO: make this real
 for (final Instruction instruction : bytecode.getInstructions()) {
 if (instruction.op().equals(Symbols.PATH))
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index d4283f4..82f71dc 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -30,6 +30,7 @@ import 
org.apache.tinkerpop.machine.traverser.TraverserFactory;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -51,7 +52,7 @@ public final class Compilation implements 
Serializable {
 

[tinkerpop] branch tp4 updated: Strategies are not sorted prior to application.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 1812efd  Strategies are not sorted prior to application.
1812efd is described below

commit 1812efdae8bce89752daab7969218a14078cbb22
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 15:10:51 2019 -0600

Strategies are not sorted prior to application.
---
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  14 +--
 .../machine/processor/ProcessorFactory.java|   6 +-
 .../machine/processor/SimpleProcessor.java |   6 +-
 .../tinkerpop/machine/strategy/StrategyUtil.java   | 118 +
 .../CoefficientVerificationStrategy.java   |   9 +-
 .../machine/structure/EmptyStructure.java  |   6 +-
 .../machine/structure/StructureFactory.java|   6 +-
 .../java/org/apache/tinkerpop/util/MultiMap.java   |  65 
 .../tinkerpop/machine/beam/BeamProcessor.java  |   6 +-
 .../tinkerpop/machine/pipes/PipesProcessor.java|   6 +-
 .../tinkergraph/TinkerGraphStructure.java  |   8 +-
 11 files changed, 217 insertions(+), 33 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index bfae7ba..421285f 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -22,22 +22,23 @@ import 
org.apache.tinkerpop.machine.bytecode.CoreCompiler.Symbols;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
 import org.apache.tinkerpop.machine.strategy.Strategy;
+import org.apache.tinkerpop.machine.strategy.StrategyUtil;
 import org.apache.tinkerpop.machine.structure.StructureFactory;
 import org.apache.tinkerpop.machine.traverser.COPTraverserFactory;
 import org.apache.tinkerpop.machine.traverser.CORTraverserFactory;
 import org.apache.tinkerpop.machine.traverser.TraverserFactory;
 
 import java.lang.reflect.InvocationTargetException;
-import java.util.ArrayList;
-import java.util.List;
+import java.util.HashSet;
 import java.util.Optional;
+import java.util.Set;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public final class BytecodeUtil {
 
-public static  void strategize(final Bytecode bytecode) {
+static  void strategize(final Bytecode bytecode) {
 for (final Strategy strategy : BytecodeUtil.getStrategies(bytecode)) {
 BytecodeUtil.strategize(bytecode, strategy);
 }
@@ -53,9 +54,9 @@ public final class BytecodeUtil {
 }
 }
 
-public static  List getStrategies(final Bytecode bytecode) 
{
+private static  Set> getStrategies(final Bytecode 
bytecode) {
 try {
-final List strategies = new ArrayList<>();
+final Set> strategies = new HashSet<>();
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
 if (sourceInstruction.op().equals(Symbols.WITH_STRATEGY)) {
 strategies.add(((Class) 
sourceInstruction.args()[0]).getConstructor().newInstance());
@@ -65,7 +66,8 @@ public final class BytecodeUtil {
 
strategies.addAll(StructureFactory.structureStrategies(((Class) sourceInstruction.args()[0])));
 }
 }
-return strategies;
+return StrategyUtil.sortStrategies(strategies);
+
 } catch (NoSuchMethodException | IllegalAccessException | 
InstantiationException | InvocationTargetException e) {
 throw new RuntimeException(e.getMessage(), e);
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
index cd7fb94..20e623a 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
@@ -23,7 +23,7 @@ import org.apache.tinkerpop.machine.strategy.Strategy;
 
 import java.io.Serializable;
 import java.lang.reflect.InvocationTargetException;
-import java.util.List;
+import java.util.Set;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -32,11 +32,11 @@ public interface ProcessorFactory extends Serializable {
 
 public  Processor mint(final Compilation 
compilation);
 
-public List getStrategies();
+public Set> getStrategies();
 
 // public Optional getCompiler();
 
-public static List processorStrategies(final Class processFactoryClass) {
+public static Set> processorStrategies(final C

[tinkerpop] branch tp4 updated: starting to flush out strategies -- decoration, optimization, provider, finalization, verification.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 3daa44f  starting to flush out strategies -- decoration, optimization, 
provider, finalization, verification.
3daa44f is described below

commit 3daa44fd4203651210b6157d1f0f2918c22ed0a6
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 14:45:33 2019 -0600

starting to flush out strategies -- decoration, optimization, provider, 
finalization, verification.
---
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  19 ++-
 .../tinkerpop/machine/bytecode/Compilation.java|   2 +-
 .../tinkerpop/machine/bytecode/CoreCompiler.java   |   6 +
 .../tinkerpop/machine/strategy/Strategy.java   | 179 -
 .../{ => finalization}/CoefficientStrategy.java|   9 +-
 .../TraverserFactoryStrategy.java  |   6 +-
 .../{ => optimization}/IdentityStrategy.java   |   7 +-
 .../CoefficientVerificationStrategy.java   |   5 +-
 .../language/gremlin/TraversalSource.java  |  10 +-
 .../machine/beam/strategy/BeamStrategy.java|   2 +-
 .../apache/tinkerpop/machine/beam/BeamTest.java|   2 +-
 .../beam/functions/TraversalSourceLibrary.java |   2 +-
 .../machine/pipes/strategy/PipesStrategy.java  |   2 +-
 .../apache/tinkerpop/machine/pipes/PipesTest.java  |   2 +-
 .../pipes/function/TraversalSourceLibrary.java |   2 +-
 .../tinkergraph/TinkerGraphStructure.java  |   2 +-
 .../strategy/{ => provider}/VerticesStrategy.java  |   6 +-
 17 files changed, 224 insertions(+), 39 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index b9a750c..bfae7ba 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.machine.bytecode;
 
+import org.apache.tinkerpop.machine.bytecode.CoreCompiler.Symbols;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
 import org.apache.tinkerpop.machine.strategy.Strategy;
@@ -56,10 +57,14 @@ public final class BytecodeUtil {
 try {
 final List strategies = new ArrayList<>();
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_STRATEGY))
+if (sourceInstruction.op().equals(Symbols.WITH_STRATEGY)) {
 strategies.add(((Class) 
sourceInstruction.args()[0]).getConstructor().newInstance());
+} else if 
(sourceInstruction.op().equals(Symbols.WITH_PROCESSOR)) {
+
strategies.addAll(ProcessorFactory.processorStrategies(((Class) sourceInstruction.args()[0])));
+} else if 
(sourceInstruction.op().equals(Symbols.WITH_STRUCTURE)) {
+
strategies.addAll(StructureFactory.structureStrategies(((Class) sourceInstruction.args()[0])));
+}
 }
-// TODO: sort strategies
 return strategies;
 } catch (NoSuchMethodException | IllegalAccessException | 
InstantiationException | InvocationTargetException e) {
 throw new RuntimeException(e.getMessage(), e);
@@ -70,7 +75,7 @@ public final class BytecodeUtil {
 try {
 Coefficient coefficient = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_COEFFICIENT)) {
+if (sourceInstruction.op().equals(Symbols.WITH_COEFFICIENT)) {
 coefficient = ((Class>) 
sourceInstruction.args()[0]).getConstructor().newInstance();
 }
 }
@@ -85,7 +90,7 @@ public final class BytecodeUtil {
 try {
 ProcessorFactory processor = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_PROCESSOR)) {
+if (sourceInstruction.op().equals(Symbols.WITH_PROCESSOR)) {
 processor = (ProcessorFactory) ((Class>) sourceInstruction.args()[0]).getConstructor().newInstance();
 }
 }
@@ -99,7 +104,7 @@ public final class BytecodeUtil {
 try {
 StructureFactory structure = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if 
(sourceInstruction.op().equals(CoreCompiler.S

[tinkerpop] branch tp4 updated: P is a Gremlin language construct. We will not be passing around complex objects in Bytecode. P simply puts the enum name in the instruction which is then generated bac

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 2dea0fc  P is a Gremlin language construct. We will not be passing 
around complex objects in Bytecode. P simply puts the enum name in the 
instruction which is then generated back to an enum at compile-time.
2dea0fc is described below

commit 2dea0fc196f1f7dde8e2a07d5f3e47099fbe0782
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 14:21:28 2019 -0600

P is a Gremlin language construct. We will not be passing around complex 
objects in Bytecode. P simply puts the enum name in the instruction which is 
then generated back to an enum at compile-time.
---
 .../tinkerpop/machine/bytecode/CoreCompiler.java   |   4 +-
 .../org/apache/tinkerpop/machine/bytecode/P.java   | 125 -
 .../apache/tinkerpop/machine/bytecode/Pred.java|  71 
 .../machine/function/filter/HasKeyFilter.java  |   8 +-
 .../machine/function/filter/IsFilter.java  |   6 +-
 .../org/apache/tinkerpop/language/gremlin/P.java   |  78 +
 .../tinkerpop/language/gremlin/Traversal.java  |  14 +--
 .../apache/tinkerpop/machine/beam/BeamTest.java|   3 +-
 .../apache/tinkerpop/machine/pipes/PipesTest.java  |   2 +-
 9 files changed, 167 insertions(+), 144 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
index 9832d58..9e90ed2 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
@@ -75,7 +75,7 @@ public final class CoreCompiler implements BytecodeCompiler {
 case Symbols.GROUP_COUNT:
 return new GroupCountReduce<>(coefficient, labels, 
Compilation.compileMaybe(instruction.args()).orElse(null));
 case Symbols.HAS_KEY:
-return new HasKeyFilter<>(coefficient, labels, 
P.Type.get(instruction.args()[0]), Argument.create(instruction.args()[1]));
+return new HasKeyFilter<>(coefficient, labels, 
Pred.get(instruction.args()[0]), Argument.create(instruction.args()[1]));
 case Symbols.HAS_KEY_VALUE:
 return new HasKeyValueFilter<>(coefficient, labels, 
Argument.create(instruction.args()[0]), Argument.create(instruction.args()[1]));
 case Symbols.IDENTITY:
@@ -83,7 +83,7 @@ public final class CoreCompiler implements BytecodeCompiler {
 case Symbols.INJECT:
 return new InjectInitial<>(coefficient, labels, 
instruction.args());
 case Symbols.IS:
-return new IsFilter<>(coefficient, labels, 
P.Type.get(instruction.args()[0]), Argument.create(instruction.args()[1]));
+return new IsFilter<>(coefficient, labels, 
Pred.get(instruction.args()[0]), Argument.create(instruction.args()[1]));
 case Symbols.INCR:
 return new IncrMap<>(coefficient, labels);
 case Symbols.JOIN:
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
deleted file mode 100644
index 2c3a889..000
--- a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.tinkerpop.machine.bytecode;
-
-import org.apache.tinkerpop.util.NumberHelper;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class P {
-
-public enum Type {
-eq {
-public boolean test(final Object first, final Object second) {
-return null == first ? null == second : (first instanceof 
Number && second instanceof Number
-? NumberHelper.compare((Number) first, (Number) 
second) == 0
-   

[tinkerpop] branch tp4 updated: CoreCompiler.Symbols now exists. Likewise TinkerGraphCompiler.Symbols.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 80c88fe  CoreCompiler.Symbols now exists. Likewise 
TinkerGraphCompiler.Symbols.
80c88fe is described below

commit 80c88fe09885035c9f9205089325b5decafe6d1c
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 14:13:56 2019 -0600

CoreCompiler.Symbols now exists. Likewise TinkerGraphCompiler.Symbols.
---
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |  12 +-
 .../tinkerpop/machine/bytecode/CoreCompiler.java   | 106 +
 .../apache/tinkerpop/machine/bytecode/Symbols.java | 128 -
 .../machine/function/barrier/JoinBarrier.java  |   6 +-
 .../machine/strategy/CoefficientStrategy.java  |   8 +-
 .../strategy/CoefficientVerificationStrategy.java  |   4 +-
 .../machine/strategy/IdentityStrategy.java |   4 +-
 .../tinkerpop/language/gremlin/Traversal.java  |   2 +-
 .../language/gremlin/TraversalSource.java  |   2 +-
 .../tinkerpop/language/gremlin/TraversalUtil.java  |   2 +-
 .../machine/beam/strategy/BeamStrategy.java|   2 +-
 .../machine/pipes/strategy/PipesStrategy.java  |   2 +-
 .../apache/tinkerpop/machine/pipes/PipesTest.java  |   2 +-
 .../tinkergraph/bytecode/TinkerGraphCompiler.java  |  11 +-
 .../tinkergraph/strategy/VerticesStrategy.java |   7 +-
 15 files changed, 143 insertions(+), 155 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index a46b756..b9a750c 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -56,7 +56,7 @@ public final class BytecodeUtil {
 try {
 final List strategies = new ArrayList<>();
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if (sourceInstruction.op().equals(Symbols.WITH_STRATEGY))
+if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_STRATEGY))
 strategies.add(((Class) 
sourceInstruction.args()[0]).getConstructor().newInstance());
 }
 // TODO: sort strategies
@@ -70,7 +70,7 @@ public final class BytecodeUtil {
 try {
 Coefficient coefficient = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if (sourceInstruction.op().equals(Symbols.WITH_COEFFICIENT)) {
+if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_COEFFICIENT)) {
 coefficient = ((Class>) 
sourceInstruction.args()[0]).getConstructor().newInstance();
 }
 }
@@ -85,7 +85,7 @@ public final class BytecodeUtil {
 try {
 ProcessorFactory processor = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if (sourceInstruction.op().equals(Symbols.WITH_PROCESSOR)) {
+if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_PROCESSOR)) {
 processor = (ProcessorFactory) ((Class>) sourceInstruction.args()[0]).getConstructor().newInstance();
 }
 }
@@ -99,7 +99,7 @@ public final class BytecodeUtil {
 try {
 StructureFactory structure = null;
 for (final SourceInstruction sourceInstruction : 
bytecode.getSourceInstructions()) {
-if (sourceInstruction.op().equals(Symbols.WITH_STRUCTURE)) {
+if 
(sourceInstruction.op().equals(CoreCompiler.Symbols.WITH_STRUCTURE)) {
 structure = (StructureFactory) ((Class>) sourceInstruction.args()[0]).getConstructor().newInstance();
 }
 }
@@ -126,9 +126,9 @@ public final class BytecodeUtil {
 public static  Optional> getTraverserFactory(final 
Bytecode bytecode) {
 // TODO: make this real
 for (final Instruction instruction : bytecode.getInstructions()) {
-if (instruction.op().equals(Symbols.PATH))
+if (instruction.op().equals(CoreCompiler.Symbols.PATH))
 return Optional.of(COPTraverserFactory.instance());
-else if (instruction.op().equals(Symbols.REPEAT))
+else if (instruction.op().equals(CoreCompiler.Symbols.REPEAT))
 return Optional.of(CORTraverserFactory.instance());
 }
 return Optional.of(COPTraverserFactory.instance());
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreCompiler.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/CoreC

[tinkerpop] branch tp4 updated: moved tinkergraph to its own module. we now have machine/processor/pipes ... machine/structure/tinkergraph. Don't know if we need such module nesting, but can always fl

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 9e2351b  moved tinkergraph to its own module. we now have 
machine/processor/pipes ... machine/structure/tinkergraph. Don't know if we 
need such module nesting, but can always flatten it out later.
9e2351b is described below

commit 9e2351b0ecab9189dff36bfdae1d897f9c7c3a6f
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 14:00:54 2019 -0600

moved tinkergraph to its own module. we now have machine/processor/pipes 
... machine/structure/tinkergraph. Don't know if we need such module nesting, 
but can always flatten it out later.
---
 java/machine/pom.xml |   4 ++--
 java/machine/processor/.DS_Store | Bin 0 -> 6148 bytes
 java/machine/processor/beam/.DS_Store| Bin 0 -> 6148 bytes
 java/machine/{ => processor}/beam/pom.xml|   8 +++-
 .../org/apache/tinkerpop/machine/beam/AbstractFn.java|   0
 .../java/org/apache/tinkerpop/machine/beam/Beam.java |   0
 .../org/apache/tinkerpop/machine/beam/BeamProcessor.java |   0
 .../java/org/apache/tinkerpop/machine/beam/BranchFn.java |   0
 .../java/org/apache/tinkerpop/machine/beam/FilterFn.java |   0
 .../org/apache/tinkerpop/machine/beam/FlatMapFn.java |   0
 .../main/java/org/apache/tinkerpop/machine/beam/Fn.java  |   0
 .../org/apache/tinkerpop/machine/beam/InitialFn.java |   0
 .../java/org/apache/tinkerpop/machine/beam/MapFn.java|   0
 .../java/org/apache/tinkerpop/machine/beam/OutputFn.java |   0
 .../java/org/apache/tinkerpop/machine/beam/ReduceFn.java |   0
 .../apache/tinkerpop/machine/beam/RepeatDeadEndFn.java   |   0
 .../org/apache/tinkerpop/machine/beam/RepeatEndFn.java   |   0
 .../org/apache/tinkerpop/machine/beam/RepeatStartFn.java |   0
 .../machine/beam/serialization/CoefficientCoder.java |   0
 .../machine/beam/serialization/ReducerCoder.java |   0
 .../machine/beam/serialization/TraverserCoder.java   |   0
 .../tinkerpop/machine/beam/sideeffect/BasicReducer.java  |   0
 .../tinkerpop/machine/beam/strategy/BeamStrategy.java|   0
 .../tinkerpop/machine/beam/util/PrimitiveVisitor.java|   0
 .../apache/tinkerpop/machine/beam/util/TopologyUtil.java |   0
 .../java/org/apache/tinkerpop/machine/beam/BeamTest.java |   0
 .../machine/beam/functions/TraversalSourceLibrary.java   |   0
 java/machine/processor/pipes/.DS_Store   | Bin 0 -> 6148 bytes
 java/machine/{ => processor}/pipes/pom.xml   |   8 +++-
 .../org/apache/tinkerpop/machine/pipes/AbstractStep.java |   0
 .../org/apache/tinkerpop/machine/pipes/BarrierStep.java  |   0
 .../org/apache/tinkerpop/machine/pipes/BranchStep.java   |   0
 .../org/apache/tinkerpop/machine/pipes/EmptyStep.java|   0
 .../org/apache/tinkerpop/machine/pipes/FilterStep.java   |   0
 .../org/apache/tinkerpop/machine/pipes/FlatMapStep.java  |   0
 .../org/apache/tinkerpop/machine/pipes/InitialStep.java  |   0
 .../java/org/apache/tinkerpop/machine/pipes/MapStep.java |   0
 .../java/org/apache/tinkerpop/machine/pipes/Pipes.java   |   0
 .../apache/tinkerpop/machine/pipes/PipesProcessor.java   |   0
 .../org/apache/tinkerpop/machine/pipes/ReduceStep.java   |   0
 .../org/apache/tinkerpop/machine/pipes/RepeatStep.java   |   0
 .../org/apache/tinkerpop/machine/pipes/SourceStep.java   |   0
 .../java/org/apache/tinkerpop/machine/pipes/Step.java|   0
 .../tinkerpop/machine/pipes/strategy/PipesStrategy.java  |   0
 .../org/apache/tinkerpop/machine/pipes/util/Barrier.java |   0
 .../tinkerpop/machine/pipes/util/InMemoryBarrier.java|   0
 .../tinkerpop/machine/pipes/util/InMemoryReducer.java|   0
 .../org/apache/tinkerpop/machine/pipes/util/Reducer.java |   0
 .../org/apache/tinkerpop/machine/pipes/PipesTest.java|   0
 .../machine/pipes/function/TraversalSourceLibrary.java   |   0
 java/machine/{ => processor}/pom.xml |   6 +++---
 java/machine/structure/.DS_Store | Bin 0 -> 6148 bytes
 java/machine/{ => structure}/pom.xml |   9 -
 java/machine/structure/tinkergraph/.DS_Store | Bin 0 -> 6148 bytes
 java/machine/{pipes => structure/tinkergraph}/pom.xml|   6 +++---
 .../machine/structure/tinkergraph/TinkerGraph.java   |   0
 .../structure/tinkergraph/TinkerGraphStructure.java  |   0
 .../tinkergraph/bytecode/TinkerGraphCompiler.java|   0
 .../machine/structure/tinkergraph/data/TinkerVertex.java |   0
 .../tinkergraph/function/initial/VerticesFlatMap.java|   0
 .../structure/tinkergraph/strategy/VerticesStrategy.java |   0
 61 files changed, 26 insertions(+), 15 deletions(-)

diff --git a/java/machine/pom.xml b/java/machine/pom.xml
index 18c4a95..e190872 100644
--

[tinkerpop] branch tp4 updated: minor cleanups and tweaks. taking a break.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 4370488  minor cleanups and tweaks. taking a break.
4370488 is described below

commit 43704888f20da9dcf62b8e96763640d40651d376
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 12:39:03 2019 -0600

minor cleanups and tweaks. taking a break.
---
 .../org/apache/tinkerpop/machine/bytecode/Compilation.java   |  2 +-
 .../{language => machine/structure}/data/TEdge.java  |  2 +-
 .../{language => machine/structure}/data/TElement.java   |  2 +-
 .../tinkerpop/{language => machine/structure}/data/TKV.java  |  2 +-
 .../tinkerpop/{language => machine/structure}/data/TMap.java |  2 +-
 .../{language => machine/structure}/data/TVertex.java|  2 +-
 .../machine/structure/tinkergraph/data/TinkerVertex.java |  6 +++---
 .../tinkergraph/function/initial/VerticesFlatMap.java|  2 +-
 .../org/apache/tinkerpop/language/gremlin/Traversal.java |  2 +-
 .../apache/tinkerpop/language/gremlin/TraversalSource.java   | 12 ++--
 .../java/org/apache/tinkerpop/machine/pipes/PipesTest.java   |  2 +-
 11 files changed, 18 insertions(+), 18 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 4c1cfe0..1368cf0 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -159,7 +159,7 @@ public final class Compilation implements 
Serializable {
 final List objects = new ArrayList<>();
 for (final Object arg : args) {
 if (arg instanceof Bytecode)
-objects.add(new Compilation<>((Bytecode) arg));
+objects.add(new Compilation<>((Bytecode) arg));
 else if (arg instanceof Character)
 objects.add(arg);
 else if (arg instanceof Integer)
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/data/TEdge.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TEdge.java
similarity index 95%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/data/TEdge.java
rename to 
java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TEdge.java
index b5e37d2..8412851 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/language/data/TEdge.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TEdge.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.language.data;
+package org.apache.tinkerpop.machine.structure.data;
 
 import org.apache.tinkerpop.util.IteratorUtils;
 
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/data/TElement.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TElement.java
similarity index 94%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/data/TElement.java
rename to 
java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TElement.java
index 6b056cc..b29423f 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/language/data/TElement.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TElement.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.language.data;
+package org.apache.tinkerpop.machine.structure.data;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/data/TKV.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TKV.java
similarity index 94%
rename from java/core/src/main/java/org/apache/tinkerpop/language/data/TKV.java
rename to 
java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TKV.java
index 62bc553..f6376d2 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/language/data/TKV.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TKV.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.language.data;
+package org.apache.tinkerpop.machine.structure.data;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/data/TMap.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/structure/data/TMap.java
similarity index 95%
rename from java/core/src/main/java/org/apache/tinkerpop/language/data/TMap.java
rename to 
java/core/src/main/java/org/apache/tinkerpop

[tinkerpop] branch tp4 updated: we officially have StructureFactory and Structure interfaces. Users can do withStructure(). A structure, like a processor, has strategies. Thus, there are now multiple

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new d932867  we officially have StructureFactory and Structure interfaces. 
Users can do withStructure(). A structure, like a processor, has strategies. 
Thus, there are now multiple compilers in play. If CoreCompiler doesn't return 
an instruction, then another is tried. Have basic g.V() working with 
TinkerGraph.
d932867 is described below

commit d9328673084c1c470e73dc77a1f659df1325fcda
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 12:11:05 2019 -0600

we officially have StructureFactory and Structure interfaces. Users can do 
withStructure(). A structure, like a processor, has strategies. Thus, there are 
now multiple compilers in play. If CoreCompiler doesn't return an instruction, 
then another is tried. Have basic g.V() working with TinkerGraph.
---
 .../BytecodeCompiler.java} |  26 +++--
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   | 109 -
 .../tinkerpop/machine/bytecode/Compilation.java|   7 +-
 .../CompositeCompiler.java}|  31 +++---
 .../{BytecodeUtil.java => CoreCompiler.java}   | 106 ++--
 .../org/apache/tinkerpop/machine/bytecode/P.java   |   6 +-
 .../apache/tinkerpop/machine/bytecode/Symbols.java |  13 +++
 .../machine/processor/ProcessorFactory.java|   2 +
 .../EmptyStructure.java}   |  40 +---
 .../tinkerpop/machine/structure/Structure.java |  25 +
 .../StructureFactory.java} |  16 +--
 .../machine/structure/tinkergraph/TinkerGraph.java |  29 ++
 .../tinkergraph/TinkerGraphStructure.java} |  35 ---
 .../tinkergraph/bytecode/TinkerGraphCompiler.java  |  43 
 .../structure/tinkergraph/data/TinkerVertex.java   |  85 
 .../function/initial/VerticesFlatMap.java  |  45 +
 .../tinkergraph/strategy/VerticesStrategy.java}|  31 +++---
 .../tinkerpop/language/gremlin/Traversal.java  |  41 +---
 .../language/gremlin/TraversalSource.java  |  15 +++
 .../apache/tinkerpop/machine/beam/BeamTest.java|  23 -
 .../apache/tinkerpop/machine/pipes/PipesTest.java  |  19 +++-
 21 files changed, 463 insertions(+), 284 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
similarity index 51%
copy from 
java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
copy to 
java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
index 9ab3557..32af547 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/processor/ProcessorFactory.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeCompiler.java
@@ -16,29 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.machine.processor;
+package org.apache.tinkerpop.machine.bytecode;
 
-import org.apache.tinkerpop.machine.bytecode.Compilation;
-import org.apache.tinkerpop.machine.strategy.Strategy;
+import org.apache.tinkerpop.machine.function.CFunction;
 
-import java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public interface ProcessorFactory extends Serializable {
+public interface BytecodeCompiler {
 
-public  Processor mint(final Compilation 
compilation);
-
-public List getStrategies();
-
-public static List processorStrategies(final Class processFactoryClass) {
-try {
-return 
processFactoryClass.getConstructor().newInstance().getStrategies();
-} catch (final NoSuchMethodException | IllegalAccessException | 
IllegalArgumentException | InvocationTargetException | InstantiationException 
e) {
-throw new RuntimeException(e.getMessage(), e);
+public default  List> compile(final Bytecode bytecode) {
+final List> functions = new ArrayList<>();
+for (final Instruction instruction : bytecode.getInstructions()) {
+functions.add(this.compile(instruction));
 }
+return functions;
 }
+
+public  CFunction compile(final Instruction instruction);
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index 99cafc6..a46b756 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/By

[tinkerpop] branch tp4 updated: super easy to add new P predicates. Just added P.regex. HasKeyFilter now supports P predicate for key.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 1b71231  super easy to add new P predicates. Just added P.regex. 
HasKeyFilter now supports P predicate for key.
1b71231 is described below

commit 1b7123139a6cc07bf80f84c00255b8cd58378792
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 09:39:03 2019 -0600

super easy to add new P predicates. Just added P.regex. HasKeyFilter now 
supports P predicate for key.
---
 .../apache/tinkerpop/machine/bytecode/BytecodeUtil.java  |  2 +-
 .../java/org/apache/tinkerpop/machine/bytecode/P.java|  8 
 .../tinkerpop/machine/function/filter/HasKeyFilter.java  | 16 ++--
 .../org/apache/tinkerpop/language/gremlin/Traversal.java |  7 ++-
 .../org/apache/tinkerpop/machine/pipes/PipesTest.java|  2 +-
 5 files changed, 30 insertions(+), 5 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index 7a81cd1..99cafc6 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -169,7 +169,7 @@ public final class BytecodeUtil {
 case Symbols.GROUP_COUNT:
 return new GroupCountReduce<>(coefficient, labels, 
Compilation.compileMaybe(instruction.args()).orElse(null));
 case Symbols.HAS_KEY:
-return new HasKeyFilter<>(coefficient, labels, 
Argument.create(instruction.args()[0]));
+return new HasKeyFilter<>(coefficient, labels, (P.Type) 
instruction.args()[0], Argument.create(instruction.args()[1]));
 case Symbols.HAS_KEY_VALUE:
 return new HasKeyValueFilter<>(coefficient, labels, 
Argument.create(instruction.args()[0]), Argument.create(instruction.args()[1]));
 case Symbols.IDENTITY:
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
index 32dc6c7..35313c4 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
+++ b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
@@ -58,6 +58,10 @@ public final class P {
 public boolean test(final Object first, final Object second) {
 return null == first ? null == second : (null != second && 
!lt.test(first, second));
 }
+}, regex {
+public boolean test(final Object first, final Object second) {
+return first.toString().matches(second.toString());
+}
 };
 
 public abstract boolean test(final Object first, final Object second);
@@ -104,6 +108,10 @@ public final class P {
 return new P<>(Type.gte, (S) object);
 }
 
+public static  P regex(final Object object) {
+return new P<>(Type.regex, (S) object);
+}
+
 @Override
 public String toString() {
 return this.type.toString();
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/filter/HasKeyFilter.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/filter/HasKeyFilter.java
index 9fbc915..2deac98 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/filter/HasKeyFilter.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/filter/HasKeyFilter.java
@@ -19,6 +19,7 @@
 package org.apache.tinkerpop.machine.function.filter;
 
 import org.apache.tinkerpop.machine.bytecode.Argument;
+import org.apache.tinkerpop.machine.bytecode.P;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.AbstractFunction;
 import org.apache.tinkerpop.machine.function.FilterFunction;
@@ -33,17 +34,28 @@ import java.util.Set;
  */
 public final class HasKeyFilter extends AbstractFunction 
implements FilterFunction> {
 
+private final P.Type predicate;
 private final Argument key;
 
-public HasKeyFilter(final Coefficient coefficient, final Set 
labels, final Argument key) {
+public HasKeyFilter(final Coefficient coefficient, final Set 
labels, final P.Type predicate, final Argument key) {
 super(coefficient, labels);
+this.predicate = predicate;
 this.key = key;
 }
 
 @Override
 public boolean test(final Traverser> traverser) {
 final Map object = traverser.object();
-return object.containsKey(this.key.getArg(traverser));
+if (P.Type.eq == this.predicate)
+return object.containsKey(this.key.getArg(traverser));
+else {
+final K testKey = this.ke

[tinkerpop] branch tp4 updated: added P. It is much easier to work with than TP3 P. It is just an enum at the bytecode level. No more nested analysis of the P argument. Really clean.

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 34a848c  added P. It is much easier to work with than TP3 P. It is 
just an enum at the bytecode level. No more nested analysis of the P argument. 
Really clean.
34a848c is described below

commit 34a848cb317a78471c2258f2d1c89628d453463f
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 08:48:37 2019 -0600

added P. It is much easier to work with than TP3 P. It is just an enum at 
the bytecode level. No more nested analysis of the P argument. Really clean.
---
 .../tinkerpop/machine/bytecode/BytecodeUtil.java   |   2 +-
 .../tinkerpop/machine/bytecode/Compilation.java|   4 +-
 .../org/apache/tinkerpop/machine/bytecode/P.java   | 113 +
 .../machine/function/filter/IsFilter.java  |  11 +-
 .../machine/processor/ConstantProcessor.java   |   2 +-
 ...{BooleanProcessor.java => FilterProcessor.java} |   4 +-
 .../machine/processor/HasNextProcessor.java|   2 +-
 .../machine/processor/LoopsProcessor.java  |   2 +-
 ...ingletonProcessor.java => SimpleProcessor.java} |   2 +-
 .../tinkerpop/language/gremlin/Traversal.java  |  10 +-
 .../tinkerpop/language/gremlin/TraversalUtil.java  |   4 +
 .../org/apache/tinkerpop/machine/beam/Beam.java|   2 +-
 .../beam/{OutputStep.java => OutputFn.java}|   9 +-
 .../tinkerpop/machine/beam/RepeatDeadEndFn.java|   1 +
 .../machine/beam/util/PrimitiveVisitor.java|   9 +-
 .../apache/tinkerpop/machine/beam/BeamTest.java|   7 ++
 .../apache/tinkerpop/machine/pipes/PipesTest.java  |   6 ++
 java/pom.xml   |   2 +-
 18 files changed, 170 insertions(+), 22 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
index 016eb30..7a81cd1 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/BytecodeUtil.java
@@ -177,7 +177,7 @@ public final class BytecodeUtil {
 case Symbols.INJECT:
 return new InjectInitial<>(coefficient, labels, 
instruction.args());
 case Symbols.IS:
-return new IsFilter<>(coefficient, labels, 
Argument.create(instruction.args()[0]));
+return new IsFilter<>(coefficient, labels, (P.Type) 
instruction.args()[0], Argument.create(instruction.args()[1]));
 case Symbols.INCR:
 return new IncrMap<>(coefficient, labels);
 case Symbols.JOIN:
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 3c8c0a2..7872c7e 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -19,7 +19,7 @@
 package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.function.CFunction;
-import org.apache.tinkerpop.machine.processor.BooleanProcessor;
+import org.apache.tinkerpop.machine.processor.FilterProcessor;
 import org.apache.tinkerpop.machine.processor.LoopsProcessor;
 import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
@@ -160,7 +160,7 @@ public final class Compilation implements 
Serializable {
 else if (arg instanceof Integer)
 objects.add(new Compilation<>(new LoopsProcessor<>((int) 
arg)));
 else if (arg instanceof Boolean)
-objects.add(new Compilation<>(new BooleanProcessor<>((boolean) 
arg)));
+objects.add(new Compilation<>(new FilterProcessor<>((boolean) 
arg)));
 }
 return objects;
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
new file mode 100644
index 000..32dc6c7
--- /dev/null
+++ b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/P.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0

[tinkerpop] branch tp4 updated: We no longer need Selector interface for branches. The branch selector is simply a compilation. For constant or simple mappings, a SingletonProcessor is used in the Com

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new c99ff4e  We no longer need Selector interface for branches. The branch 
selector is simply a compilation. For constant or simple mappings, a 
SingletonProcessor is used in the Compilation. pom.xml work -- both Beam and 
Pipes only need gremlin/ in scope=test. This is pretty cool -- Gremlin is not 
front and center in TP4.
c99ff4e is described below

commit c99ff4e23b0ad14e50dae2a53c054107a54b675d
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 06:53:04 2019 -0600

We no longer need Selector interface for branches. The branch selector is 
simply a compilation. For constant or simple mappings, a SingletonProcessor is 
used in the Compilation. pom.xml work -- both Beam and Pipes only need gremlin/ 
in scope=test. This is pretty cool -- Gremlin is not front and center in TP4.
---
 .../tinkerpop/machine/bytecode/Compilation.java|  6 
 .../tinkerpop/machine/function/BranchFunction.java |  3 +-
 .../machine/function/branch/ChooseBranch.java  |  9 +++---
 .../machine/function/branch/UnionBranch.java   |  8 +++---
 .../machine/function/branch/selector/Selector.java | 32 --
 .../machine/processor/BooleanProcessor.java|  2 +-
 .../ConstantProcessor.java}| 19 ++---
 .../HasNextProcessor.java} | 16 +--
 .../machine/processor/LoopsProcessor.java  |  2 +-
 .../machine/processor/SingletonProcessor.java  |  8 +++---
 .../tinkerpop/machine/traverser/COTraverser.java   |  7 +
 .../machine/traverser/EmptyTraverser.java  |  7 -
 .../tinkerpop/machine/traverser/Traverser.java |  2 ++
 java/machine/beam/pom.xml  | 19 +
 .../apache/tinkerpop/machine/beam/BranchFn.java|  8 +++---
 java/machine/pipes/pom.xml |  7 +
 .../apache/tinkerpop/machine/pipes/BranchStep.java |  7 ++---
 17 files changed, 79 insertions(+), 83 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 58c59d8..3c8c0a2 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -104,6 +104,12 @@ public final class Compilation implements 
Serializable {
 return this.processor;
 }
 
+public Optional> maybeTraverser(final Traverser 
traverser) {
+this.prepareProcessor();
+this.processor.addStart(this.prepareTraverser(traverser));
+return this.processor.hasNext() ? Optional.of(this.processor.next()) : 
Optional.empty();
+}
+
 @Override
 public String toString() {
 return this.functions.toString();
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/BranchFunction.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/BranchFunction.java
index 73a7fb2..d54ba8b 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/BranchFunction.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/BranchFunction.java
@@ -19,7 +19,6 @@
 package org.apache.tinkerpop.machine.function;
 
 import org.apache.tinkerpop.machine.bytecode.Compilation;
-import org.apache.tinkerpop.machine.function.branch.selector.Selector;
 
 import java.util.List;
 import java.util.Map;
@@ -29,7 +28,7 @@ import java.util.Map;
  */
 public interface BranchFunction extends CFunction {
 
-public Selector getBranchSelector();
+public Compilation getBranchSelector();
 
 public Map>> getBranches();
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/branch/ChooseBranch.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/branch/ChooseBranch.java
index f064a81..315 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/branch/ChooseBranch.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/branch/ChooseBranch.java
@@ -21,9 +21,8 @@ package org.apache.tinkerpop.machine.function.branch;
 import org.apache.tinkerpop.machine.bytecode.Compilation;
 import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.AbstractFunction;
-import org.apache.tinkerpop.machine.function.branch.selector.HasNextSelector;
-import org.apache.tinkerpop.machine.function.branch.selector.Selector;
 import org.apache.tinkerpop.machine.function.BranchFunction;
+import org.apache.tinkerpop.machine.processor.HasNextProcessor;
 import org.apache.tinkerpop.util.StringFactory;
 
 import java.util.Collections;
@@ -37,7 +36,7 @@ import java

[tinkerpop] branch tp4 updated: We now have 'primitive processors' which are like Pipes and Beam, but are simple. For instance, repeat().times(5), the 5 is turned into a LoopsProcessor. This ensures t

2019-03-18 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new c0482f6  We now have 'primitive processors' which are like Pipes and 
Beam, but are simple. For instance, repeat().times(5), the 5 is turned into a 
LoopsProcessor. This ensures that basic operations can run efficiently and 
within the framework we have thus far of hiding everything behind a 
Compilation. We will probably end up replacing branch Selectors with such 
'primitive procssors'.
c0482f6 is described below

commit c0482f6c45ca51985f1bbb7fdf714b36d90979a5
Author: Marko A. Rodriguez 
AuthorDate: Mon Mar 18 06:25:00 2019 -0600

We now have 'primitive processors' which are like Pipes and Beam, but are 
simple. For instance, repeat().times(5), the 5 is turned into a LoopsProcessor. 
This ensures that basic operations can run efficiently and within the framework 
we have thus far of hiding everything behind a Compilation. We will probably 
end up replacing branch Selectors with such 'primitive procssors'.
---
 .../tinkerpop/machine/bytecode/Compilation.java| 15 +-
 .../tinkerpop/machine/function/map/PathMap.java|  4 +--
 .../{EmptyProcessor.java => BooleanProcessor.java} | 31 +---
 .../machine/processor/LoopsProcessor.java} | 18 ++--
 ...ocessorFactory.java => SingletonProcessor.java} | 33 --
 .../tinkerpop/machine/traverser/COPTraverser.java  |  1 +
 .../tinkerpop/machine/traverser/COTraverser.java   |  2 ++
 .../machine/traverser/EmptyTraverser.java  |  1 +
 .../tinkerpop/machine/traverser/Traverser.java |  2 +-
 .../machine/traverser/{ => path}/BasicPath.java|  2 +-
 .../machine/traverser/{ => path}/EmptyPath.java|  4 +--
 .../machine/traverser/{ => path}/Path.java |  2 +-
 .../tinkerpop/language/gremlin/Traversal.java  | 27 ++
 .../language/gremlin/TraversalSource.java  |  2 +-
 .../tinkerpop/language/gremlin/TraversalUtil.java  | 10 +++
 15 files changed, 85 insertions(+), 69 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 83b487f..58c59d8 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -19,6 +19,8 @@
 package org.apache.tinkerpop.machine.bytecode;
 
 import org.apache.tinkerpop.machine.function.CFunction;
+import org.apache.tinkerpop.machine.processor.BooleanProcessor;
+import org.apache.tinkerpop.machine.processor.LoopsProcessor;
 import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
 import org.apache.tinkerpop.machine.traverser.Traverser;
@@ -26,6 +28,7 @@ import 
org.apache.tinkerpop.machine.traverser.TraverserFactory;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
@@ -47,6 +50,12 @@ public final class Compilation implements 
Serializable {
 this.functions = BytecodeUtil.compile(bytecode);
 }
 
+public Compilation(final ProcessorFactory processorFactory) {
+this.functions = Collections.emptyList(); // TODO: somehow strings for 
primitive processors
+this.processorFactory = processorFactory;
+this.traverserFactory = null;
+}
+
 public Processor getProcessor() {
 if (null == this.processor)
 this.processor = this.processorFactory.mint(this);
@@ -140,8 +149,12 @@ public final class Compilation implements 
Serializable {
 for (final Object arg : args) {
 if (arg instanceof Bytecode)
 objects.add(new Compilation<>((Bytecode) arg));
-else
+else if (arg instanceof Character)
 objects.add(arg);
+else if (arg instanceof Integer)
+objects.add(new Compilation<>(new LoopsProcessor<>((int) 
arg)));
+else if (arg instanceof Boolean)
+objects.add(new Compilation<>(new BooleanProcessor<>((boolean) 
arg)));
 }
 return objects;
 }
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
index 39d7514..04172b1 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/function/map/PathMap.java
@@ -23,8 +23,8 @@ import 
org.apache.tinkerpop.machine.bytecode.CompilationCircle;
 import org.apache.tinke

[tinkerpop] branch tp4 updated: moved all the Gremlin stuff to a new submodule of languages -- languages/gremlin. I think we can get rid of core here soon.

2019-03-17 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new e416465  moved all the Gremlin stuff to a new submodule of languages 
-- languages/gremlin. I think we can get rid of core here soon.
e416465 is described below

commit e41646548e9f09150ec03e6ae5868dcb053e4731
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 17 15:00:16 2019 -0600

moved all the Gremlin stuff to a new submodule of languages -- 
languages/gremlin. I think we can get rid of core here soon.
---
 java/{machine/pipes => language/gremlin}/pom.xml   |  6 ++--
 .../apache/tinkerpop/language/gremlin/Gremlin.java |  0
 .../tinkerpop/language/gremlin/Traversal.java  |  0
 .../language/gremlin/TraversalSource.java  |  0
 .../tinkerpop/language/gremlin/TraversalUtil.java  |  0
 .../org/apache/tinkerpop/language/gremlin/__.java  |  0
 java/{machine/pipes => language}/pom.xml   | 37 --
 java/machine/beam/pom.xml  |  2 +-
 java/machine/pipes/pom.xml |  2 +-
 java/pom.xml   |  1 +
 10 files changed, 13 insertions(+), 35 deletions(-)

diff --git a/java/machine/pipes/pom.xml b/java/language/gremlin/pom.xml
similarity index 93%
copy from java/machine/pipes/pom.xml
copy to java/language/gremlin/pom.xml
index 3d40c2a..30ee8dc 100644
--- a/java/machine/pipes/pom.xml
+++ b/java/language/gremlin/pom.xml
@@ -17,12 +17,12 @@ limitations under the License.
  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
 4.0.0
 
-machine
+language
 org.apache.tinkerpop
 4.0.0-SNAPSHOT
 
-Apache TinkerPop (Java) :: Machine :: Pipes
-pipes
+Apache TinkerPop (Java) :: Language :: Gremlin
+gremlin
 
 
 org.apache.tinkerpop
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
similarity index 100%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
rename to 
java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Gremlin.java
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
similarity index 100%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
rename to 
java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/Traversal.java
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
similarity index 100%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
rename to 
java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalSource.java
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
similarity index 100%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
rename to 
java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/TraversalUtil.java
diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/gremlin/__.java 
b/java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
similarity index 100%
rename from 
java/core/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
rename to 
java/language/gremlin/src/main/java/org/apache/tinkerpop/language/gremlin/__.java
diff --git a/java/machine/pipes/pom.xml b/java/language/pom.xml
similarity index 51%
copy from java/machine/pipes/pom.xml
copy to java/language/pom.xml
index 3d40c2a..3aaf916 100644
--- a/java/machine/pipes/pom.xml
+++ b/java/language/pom.xml
@@ -16,38 +16,15 @@ limitations under the License.
  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
 4.0.0
+pom
 
-machine
 org.apache.tinkerpop
+tinkerpop
 4.0.0-SNAPSHOT
 
-Apache TinkerPop (Java) :: Machine :: Pipes
-pipes
-
-
-org.apache.tinkerpop
-core
-${project.version}
-
-
-
-${basedir}/target
-${project.artifactId}-${project.version}
-
-
-${

[tinkerpop] branch tp4 updated: added language/data with TMap, TVertex, TEdge, TKV, TElement interfaces. Moved Traversal stuff to languages/gremlin. This will need to be moved to a newproject called l

2019-03-17 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 01afbf7  added language/data with TMap, TVertex, TEdge, TKV, TElement 
interfaces. Moved Traversal stuff to languages/gremlin. This will need to be 
moved to a newproject called language (like we have machine).
01afbf7 is described below

commit 01afbf72754afe7433a0ca87c1834085fddf1452
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 17 14:50:01 2019 -0600

added language/data with TMap, TVertex, TEdge, TKV, TElement interfaces. 
Moved Traversal stuff to languages/gremlin. This will need to be moved to a 
newproject called language (like we have machine).
---
 .../{TraversalUtil.java => data/TEdge.java} | 19 ++-
 .../language/{Gremlin.java => data/TElement.java}   | 10 +-
 .../language/{Gremlin.java => data/TKV.java}| 10 +-
 .../language/{Gremlin.java => data/TMap.java}   | 19 ++-
 .../{TraversalUtil.java => data/TVertex.java}   | 21 +
 .../tinkerpop/language/{ => gremlin}/Gremlin.java   |  2 +-
 .../tinkerpop/language/{ => gremlin}/Traversal.java |  2 +-
 .../language/{ => gremlin}/TraversalSource.java |  2 +-
 .../language/{ => gremlin}/TraversalUtil.java   |  2 +-
 .../apache/tinkerpop/language/{ => gremlin}/__.java |  2 +-
 .../org/apache/tinkerpop/machine/beam/BeamTest.java | 14 +++---
 .../beam/functions/TraversalSourceLibrary.java  |  4 ++--
 .../apache/tinkerpop/machine/pipes/PipesTest.java   | 12 ++--
 .../pipes/function/TraversalSourceLibrary.java  |  4 ++--
 14 files changed, 69 insertions(+), 54 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/language/TraversalUtil.java 
b/java/core/src/main/java/org/apache/tinkerpop/language/data/TEdge.java
similarity index 70%
copy from 
java/core/src/main/java/org/apache/tinkerpop/language/TraversalUtil.java
copy to java/core/src/main/java/org/apache/tinkerpop/language/data/TEdge.java
index 758fb47..b5e37d2 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/language/TraversalUtil.java
+++ b/java/core/src/main/java/org/apache/tinkerpop/language/data/TEdge.java
@@ -16,21 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.language;
+package org.apache.tinkerpop.language.data;
 
-import org.apache.tinkerpop.machine.bytecode.Bytecode;
+import org.apache.tinkerpop.util.IteratorUtils;
+
+import java.util.Iterator;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class TraversalUtil {
+public interface TEdge extends TElement {
 
-private TraversalUtil() {
-// do nothing
-}
+public TVertex inVertex();
 
-public static  Bytecode getBytecode(final Traversal 
traversal) {
-return traversal.bytecode;
-}
+public TVertex outVertex();
 
+public default Iterator> bothVertices() {
+return IteratorUtils.of(this.inVertex(), this.outVertex());
+}
 }
diff --git a/java/core/src/main/java/org/apache/tinkerpop/language/Gremlin.java 
b/java/core/src/main/java/org/apache/tinkerpop/language/data/TElement.java
similarity index 83%
copy from java/core/src/main/java/org/apache/tinkerpop/language/Gremlin.java
copy to java/core/src/main/java/org/apache/tinkerpop/language/data/TElement.java
index 2f29245..6b056cc 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/language/Gremlin.java
+++ b/java/core/src/main/java/org/apache/tinkerpop/language/data/TElement.java
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.language;
+package org.apache.tinkerpop.language.data;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class Gremlin {
+public interface TElement extends TMap {
 
-public static  TraversalSource traversal() {
-return new TraversalSource<>();
-}
+public String label();
+
+public Object id();
 }
diff --git a/java/core/src/main/java/org/apache/tinkerpop/language/Gremlin.java 
b/java/core/src/main/java/org/apache/tinkerpop/language/data/TKV.java
similarity index 83%
copy from java/core/src/main/java/org/apache/tinkerpop/language/Gremlin.java
copy to java/core/src/main/java/org/apache/tinkerpop/language/data/TKV.java
index 2f29245..62bc553 100644
--- a/java/core/src/main/java/org/apache/tinkerpop/language/Gremlin.java
+++ b/java/core/src/main/java/org/apache/tinkerpop/language/data/TKV.java
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.tinkerpop.language;
+package org.apache.tinkerpop.language.data;
 
 /**
  * @author Marko A. Rodriguez (http://marko

[tinkerpop] branch tp4 updated: minor nothings....taking a break.

2019-03-17 Thread okram
This is an automated email from the ASF dual-hosted git repository.

okram pushed a commit to branch tp4
in repository https://gitbox.apache.org/repos/asf/tinkerpop.git


The following commit(s) were added to refs/heads/tp4 by this push:
 new 6ed80f2  minor nothingstaking a break.
6ed80f2 is described below

commit 6ed80f252d5328d9a34896381c8104c0bdb522bc
Author: Marko A. Rodriguez 
AuthorDate: Sun Mar 17 14:01:58 2019 -0600

minor nothingstaking a break.
---
 .../main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java  | 4 
 1 file changed, 4 deletions(-)

diff --git 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
index 02c7221..83b487f 100644
--- 
a/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
+++ 
b/java/core/src/main/java/org/apache/tinkerpop/machine/bytecode/Compilation.java
@@ -18,7 +18,6 @@
  */
 package org.apache.tinkerpop.machine.bytecode;
 
-import org.apache.tinkerpop.machine.coefficient.Coefficient;
 import org.apache.tinkerpop.machine.function.CFunction;
 import org.apache.tinkerpop.machine.processor.Processor;
 import org.apache.tinkerpop.machine.processor.ProcessorFactory;
@@ -39,15 +38,12 @@ public final class Compilation implements 
Serializable {
 private final List> functions;
 private final ProcessorFactory processorFactory;
 private final TraverserFactory traverserFactory;
-private final Coefficient unity;
 private transient Processor processor;
 
 public Compilation(final Bytecode bytecode) {
 BytecodeUtil.strategize(bytecode);
 this.processorFactory = 
BytecodeUtil.getProcessorFactory(bytecode).get();
 this.traverserFactory = 
BytecodeUtil.getTraverserFactory(bytecode).get();
-this.unity = BytecodeUtil.getCoefficient(bytecode).get().clone();
-this.unity.unity();
 this.functions = BytecodeUtil.compile(bytecode);
 }
 



  1   2   3   4   5   6   7   8   9   10   >