From 3ff39866908dc0c76cdd5eba0423dfefa39a0fd6 Mon Sep 17 00:00:00 2001 From: Philip Fritzsche Date: Wed, 6 Nov 2019 14:08:39 +0100 Subject: [PATCH 1/7] Add initial functions for vertex retention. - add unit tests for key functions --- .../api/functions/DefaultKeyCheckable.java | 32 ++++ .../KeyFunctionWithDefaultValue.java | 18 ++- .../keyedgrouping/KeyedGrouping.java | 47 +++++- .../functions/UpdateIdField.java | 4 +- .../keys/CompositeKeyFunction.java | 2 +- ...CompositeKeyFunctionWithDefaultValues.java | 18 +++ .../LabelSpecificKeyFunction.java | 17 +- .../WithAllKeysSetToDefault.java | 73 +++++++++ ...ositeKeyFunctionWithDefaultValuesTest.java | 104 ++++++++++++ .../keys/ConstantKeyFunctionTest.java | 40 +++++ .../keys/KeyFunctionTestBase.java | 151 ++++++++++++++++++ .../keys/LabelKeyFunctionTest.java | 62 +++++++ .../keys/LabelSpecificKeyFunctionTest.java | 23 +++ .../keys/PropertyKeyFunctionTest.java | 69 ++++++++ .../WithAllKeysSetToDefaultTest.java | 82 ++++++++++ 15 files changed, 734 insertions(+), 8 deletions(-) create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java new file mode 100644 index 000000000000..c4eaa8231365 --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java @@ -0,0 +1,32 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.api.functions; + +/** + * Indicates that a key-function can check if a key is considered a default value for that function.

+ * This interface needs to be implemented on {@link KeyFunction key functions} where the default key + * is not unique, but depending on some other conditions, like the label of an element. + */ +public interface DefaultKeyCheckable { + + /** + * Check if a key is considered a default key for this function. + * + * @param key The key to check. + * @return {@code true}, if the key is effectively a default key. + */ + boolean isDefaultKey(Object key); +} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java index e3cb4275c2a2..19e802ce8ef6 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java @@ -15,6 +15,8 @@ */ package org.gradoop.flink.model.api.functions; +import java.util.Objects; + /** * A (grouping) key function with a default value. The value will be used in some cases where the key can * not be determined or when this key function is not applicable for an element.

@@ -23,7 +25,7 @@ * @param The type of the object from which the grouping key is extracted. * @param The type of the extracted key. */ -public interface KeyFunctionWithDefaultValue extends KeyFunction { +public interface KeyFunctionWithDefaultValue extends KeyFunction, DefaultKeyCheckable { /** * Get the default key value for all elements. @@ -31,4 +33,18 @@ public interface KeyFunctionWithDefaultValue extends KeyFunction { * @return The default key. */ K getDefaultKey(); + + /** + * {@inheritDoc} + *

+ * The default implementation of this method compares the key to the default key provided by this class + * using {@link Objects#deepEquals(Object, Object)}. + * + * @param key The key to check. + * @return {@code true}, if the key is a default key for this key-function. + */ + @Override + default boolean isDefaultKey(Object key) { + return Objects.deepEquals(getDefaultKey(), key); + } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java index 52196e6f62c8..6d57c8ca47ec 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java @@ -15,6 +15,7 @@ */ package org.gradoop.flink.model.impl.operators.keyedgrouping; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -25,6 +26,7 @@ import org.gradoop.flink.model.api.epgm.BaseGraph; import org.gradoop.flink.model.api.epgm.BaseGraphCollection; import org.gradoop.flink.model.api.functions.AggregateFunction; +import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.api.operators.UnaryBaseGraphToBaseGraphOperator; import org.gradoop.flink.model.impl.functions.filters.Not; @@ -37,6 +39,7 @@ import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.ReduceEdgeTuples; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.ReduceVertexTuples; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.UpdateIdField; +import org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific.WithAllKeysSetToDefault; import java.util.Collections; import java.util.List; @@ -93,6 +96,11 @@ public class KeyedGrouping< */ private boolean useGroupCombine = true; + /** + * Should vertices with all default keys be kept as is? + */ + private boolean retainUngroupedVertices = false; + /** * Instantiate this grouping function. * @@ -123,9 +131,17 @@ public KeyedGrouping(List> vertexGroupingKeys, @Override public LG execute(LG graph) { /* First we create tuple representations of each vertex. - Those tuples will then be grouped by the respective key fields (the fields containing the values + If retention of ungrouped vertices is enabled, we filter out vertices with unset keys prior to this + step. Those tuples will then be grouped by the respective key fields (the fields containing the values extracted by the key functions) and reduced to assign a super vertex and to calculate aggregates. */ - DataSet verticesWithSuperVertex = graph.getVertices() + DataSet vertices = graph.getVertices(); + DataSet ungrouped = vertices; + if (retainUngroupedVertices) { + final FilterFunction retentionSelector = new WithAllKeysSetToDefault<>(vertexGroupingKeys); + ungrouped = ungrouped.filter(new Not<>(retentionSelector)); + vertices = vertices.filter(retentionSelector); + } + DataSet verticesWithSuperVertex = vertices .map(new BuildTuplesFromVertices<>(vertexGroupingKeys, vertexAggregateFunctions)) .groupBy(getInternalVertexGroupingKeys()) .reduceGroup(new ReduceVertexTuples<>( @@ -139,11 +155,11 @@ Those tuples will then be grouped by the respective key fields (the fields conta with the mapping extracted in the previous step. Edges will then point from and to super-vertices. */ DataSet edgesWithUpdatedIds = graph.getEdges() .map(new BuildTuplesFromEdges<>(edgeGroupingKeys, edgeAggregateFunctions)) - .join(idToSuperId) + .leftOuterJoin(idToSuperId) .where(GroupingConstants.EDGE_TUPLE_SOURCEID) .equalTo(GroupingConstants.VERTEX_TUPLE_ID) .with(new UpdateIdField<>(GroupingConstants.EDGE_TUPLE_SOURCEID)) - .join(idToSuperId) + .leftOuterJoin(idToSuperId) .where(GroupingConstants.EDGE_TUPLE_TARGETID) .equalTo(GroupingConstants.VERTEX_TUPLE_ID) .with(new UpdateIdField<>(GroupingConstants.EDGE_TUPLE_TARGETID)); @@ -167,6 +183,10 @@ Those tuples will then be grouped by the respective key fields (the fields conta .map(new BuildSuperEdgeFromTuple<>(edgeGroupingKeys, edgeAggregateFunctions, graph.getFactory().getEdgeFactory())); + if (retainUngroupedVertices) { + /* We have to add the previously filtered vertices back. */ + superVertices = superVertices.union(ungrouped); + } return graph.getFactory().fromDataSets(superVertices, superEdges); } @@ -203,4 +223,23 @@ public KeyedGrouping setUseGroupCombine(boolean useGroupCombine this.useGroupCombine = useGroupCombine; return this; } + + /** + * Enable or disable vertex retention. + *

+ * Enabling this features requires that all vertex keys implement {@link DefaultKeyCheckable}. + *

+ * This is disabled per default. + * + * @param retainVertices Should vertices be retained? + * @return This operator. + * @throws IllegalArgumentException When any vertex key function is not supported for this feature. + */ + public KeyedGrouping setRetainUngroupedVertices(boolean retainVertices) { + if (retainVertices) { + WithAllKeysSetToDefault.checkKeySupport(vertexGroupingKeys); + } + this.retainUngroupedVertices = retainVertices; + return this; + } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdField.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdField.java index d4a39dfa8ab7..4dbb92c74697 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdField.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdField.java @@ -45,7 +45,9 @@ public UpdateIdField(int index) { @Override public T join(T inputTuple, Tuple2 updateValue) throws Exception { - inputTuple.setField(updateValue.f1, index); + if (updateValue != null) { + inputTuple.setField(updateValue.f1, index); + } return inputTuple; } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java index 9086e46d91b0..d893203978a9 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java @@ -34,7 +34,7 @@ public class CompositeKeyFunction implements KeyFunction { /** * A list of grouping key functions combined in this key function. */ - private final List> componentFunctions; + protected final List> componentFunctions; /** * Reduce object instantiations. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java index aba03f5c9cf7..6f4b8933e2d1 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java @@ -52,4 +52,22 @@ public CompositeKeyFunctionWithDefaultValues(List The type of the elements to group. */ -public class LabelSpecificKeyFunction implements KeyFunction { +public class LabelSpecificKeyFunction implements KeyFunction, + DefaultKeyCheckable { /** * A label used to identify the default groups. @@ -162,4 +164,17 @@ public TypeInformation getType() { } return new TupleTypeInfo<>(types); } + + @Override + public boolean isDefaultKey(Object key) { + if (!(key instanceof Tuple)) { + return false; + } + Tuple keyTuple = (Tuple) key; + if (keyTuple.getArity() != (1 + keyFunctions.size())) { + return false; + } + Integer index = keyTuple.getField(0); + return keyFunctions.get(index).isDefaultKey(keyTuple.getField(1 + index)); + } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java new file mode 100644 index 000000000000..4ac55cbb2015 --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java @@ -0,0 +1,73 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific; + +import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; +import org.gradoop.flink.model.api.functions.KeyFunction; +import org.gradoop.flink.model.impl.functions.filters.CombinableFilter; + +import java.util.List; +import java.util.Objects; + +/** + * A filter function selecting elements with all values set to a default value. + * + * @param The type of the elements to filter. + */ +public class WithAllKeysSetToDefault implements CombinableFilter { + + /** + * The keys to check on each element. + */ + private final List> keys; + + /** + * Create a new instance of this filter function. + * + * @param keys The list of key functions to check. + */ + public WithAllKeysSetToDefault(List> keys) { + checkKeySupport(keys); + this.keys = Objects.requireNonNull(keys); + } + + @Override + public boolean filter(E value) { + for (KeyFunction key : keys) { + final Object extractedKey = key.getKey(value); + if (!((DefaultKeyCheckable) key).isDefaultKey(extractedKey)) { + return false; + } + } + return true; + } + + /** + * Check if keys functions are supported with this filter function. + * + * @param keys The key functions to check. + * @param The type of the key functions. + * @throws IllegalArgumentException When any key function is not supported by this filter. + */ + public static void checkKeySupport(List> keys) { + for (KeyFunction key : keys) { + if (!(key instanceof DefaultKeyCheckable)) { + throw new IllegalArgumentException("Key function " + key + " does not implement " + + DefaultKeyCheckable.class.getName()); + } + } + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java new file mode 100644 index 000000000000..64884a1e7cca --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java @@ -0,0 +1,104 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.keys; + +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.gradoop.common.GradoopTestUtils; +import org.gradoop.common.model.api.entities.Element; +import org.gradoop.common.model.api.entities.VertexFactory; +import org.gradoop.common.model.impl.pojo.EPGMVertex; +import org.gradoop.common.model.impl.properties.PropertyValue; +import org.gradoop.flink.model.api.functions.KeyFunction; +import org.gradoop.flink.model.api.functions.KeyFunctionWithDefaultValue; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * Test for the {@link CompositeKeyFunctionWithDefaultValues}. + */ +public class CompositeKeyFunctionWithDefaultValuesTest extends KeyFunctionTestBase { + + /** + * A property key for one component function of the test function. + */ + private final String propertyKey = "key"; + + @Override + public void setUp() { + checkForKeyEquality = false; + } + + @Override + protected KeyFunction getInstance() { + return new CompositeKeyFunctionWithDefaultValues<>( + Arrays.asList(new PropertyKeyFunction<>(propertyKey), new LabelKeyFunction<>())); + } + + /** + * Test if the default key has the correct value.

+ */ + @Test + public void checkDefaultKeyValue() { + final Tuple defaultKey = ((KeyFunctionWithDefaultValue) getInstance()).getDefaultKey(); + assertEquals(2, defaultKey.getArity()); + assertArrayEquals(PropertyValue.NULL_VALUE.getRawBytes(), defaultKey.getField(0)); + assertEquals("", defaultKey.getField(1)); + } + + @Override + public void testGetKey() { + final KeyFunction keyFunction = getInstance(); + final VertexFactory vertexFactory = getConfig().getLogicalGraphFactory().getVertexFactory(); + EPGMVertex testVertex = vertexFactory.createVertex(); + Tuple key = keyFunction.getKey(testVertex); + assertEquals(2, key.getArity()); + assertArrayEquals(PropertyValue.NULL_VALUE.getRawBytes(), key.getField(0)); + assertEquals("", key.getField(1)); + final String testLabel = "testLabel"; + testVertex.setLabel(testLabel); + key = keyFunction.getKey(testVertex); + assertEquals(2, key.getArity()); + assertArrayEquals(PropertyValue.NULL_VALUE.getRawBytes(), key.getField(0)); + assertEquals(testLabel, key.getField(1)); + final PropertyValue testValue = PropertyValue.create(GradoopTestUtils.DOUBLE_VAL_5); + testVertex.setProperty(propertyKey, testValue); + key = keyFunction.getKey(testVertex); + assertEquals(2, key.getArity()); + assertArrayEquals(testValue.getRawBytes(), key.getField(0)); + assertEquals(testLabel, key.getField(1)); + } + + /** + * Test for the {@link CompositeKeyFunctionWithDefaultValues#addKeyToElement(Object, Object)} function. + */ + @Test + public void testAddKeyToElement() { + final EPGMVertex vertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); + final PropertyValue testValue = PropertyValue.create(GradoopTestUtils.BIG_DECIMAL_VAL_7); + final String testLabel = "testLabel"; + assertEquals("", vertex.getLabel()); + assertFalse(vertex.hasProperty(propertyKey)); + getInstance().addKeyToElement(vertex, Tuple2.of(testValue.getRawBytes(), testLabel)); + assertEquals(testValue, vertex.getPropertyValue(propertyKey)); + assertEquals(testLabel, vertex.getLabel()); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java new file mode 100644 index 000000000000..a5850abd8585 --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java @@ -0,0 +1,40 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.keys; + +import org.gradoop.common.model.api.entities.Element; +import org.gradoop.flink.model.api.functions.KeyFunction; + +import java.util.HashMap; +import java.util.Map; + +/** + * Test for the {@link ConstantKeyFunction}. + */ +public class ConstantKeyFunctionTest extends KeyFunctionTestBase { + + @Override + protected KeyFunction getInstance() { + return new ConstantKeyFunction<>(); + } + + @Override + protected Map getTestElements() { + Map testCases = new HashMap<>(); + testCases.put(getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(), Boolean.TRUE); + return testCases; + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java new file mode 100644 index 000000000000..c81caa5f794d --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java @@ -0,0 +1,151 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.keys; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.gradoop.flink.model.GradoopFlinkTestBase; +import org.gradoop.flink.model.api.functions.KeyFunction; +import org.gradoop.flink.model.api.functions.KeyFunctionWithDefaultValue; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeTrue; + +/** + * A base class for key-function tests. This provides some common tests that should pass by all keys to be + * used for grouping. + * + * @param The element type. + * @param The key type. + */ +public abstract class KeyFunctionTestBase extends GradoopFlinkTestBase { + + /** + * Flag indicating whether a serialized and deserialized key should be checked for equality. + * Setting this to false might be useful when testing composite keys or keys that use array types. + */ + protected boolean checkForKeyEquality; + + /** + * Get an instance of the key function to test. + * + * @return The key function. + */ + protected abstract KeyFunction getInstance(); + + /** + * Get a {@link Map} of test cases for the {@link KeyFunction#getKey(Object)} method. + * Keys of this map will be elements and values keys extracted from those elements.

+ * The default implementation of this method provides no test elements. + * + * @return A map of test cases. + */ + protected Map getTestElements() { + return Collections.emptyMap(); + } + + /** + * Setup this test. + */ + @Before + public void setUp() { + checkForKeyEquality = true; + } + + /** + * Check if the {@link KeyFunction#getType()} function returns a non-{@code null} value that is a valid + * key type. + */ + @Test + public void checkTypeInfo() { + TypeInformation type = getInstance().getType(); + assertNotNull("Type information provided by the key fuction was null.", type); + assertTrue("Type is not a valid key type.", type.isKeyType()); + assertNotEquals("Key type has no fields.", 0, type.getTotalFields()); + } + + /** + * Check if the default key value has the correct type.

+ * This test will only run if the key function has a default key. + * + * @throws IOException when serialization of a key fails. + */ + @Test + public void checkDefaultKey() throws IOException { + KeyFunction function = getInstance(); + assumeTrue(function instanceof KeyFunctionWithDefaultValue); + KeyFunctionWithDefaultValue withDefaultValue = (KeyFunctionWithDefaultValue) function; + K defaultKey = withDefaultValue.getDefaultKey(); + checkKeyType(withDefaultValue.getType(), defaultKey); + assertTrue(withDefaultValue.isDefaultKey(defaultKey)); + } + + /** + * Check if a key is of a certain type and if it is serializable as that type. + * + * @param type The type. + * @param key The key to check. + * @throws IOException when serialization of the key fails. + */ + protected void checkKeyType(TypeInformation type, K key) throws IOException { + assertTrue(type.getTypeClass().isInstance(key)); + // Check serializability + ByteArrayOutputStream byteOutput = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper output = new DataOutputViewStreamWrapper(byteOutput); + TypeSerializer serializer = type.createSerializer(getExecutionEnvironment().getConfig()); + serializer.serialize(key, output); + output.close(); + byte[] serializedData = byteOutput.toByteArray(); + DataInputViewStreamWrapper input = new DataInputViewStreamWrapper( + new ByteArrayInputStream(serializedData)); + K deserializedKey = serializer.deserialize(input); + if (checkForKeyEquality) { + assertTrue(Objects.deepEquals(key, deserializedKey)); + } + input.close(); + } + + /** + * Test the {@link KeyFunction#getKey(Object)} function using test cases provided by + * {@link #getTestElements()}. + * + * @throws IOException when serialization of a key fails + */ + @Test + public void testGetKey() throws IOException { + final KeyFunction function = getInstance(); + TypeInformation type = function.getType(); + for (Map.Entry testCase : getTestElements().entrySet()) { + final K actual = function.getKey(testCase.getKey()); + final K expected = testCase.getValue(); + checkKeyType(type, actual); + assertTrue(Objects.deepEquals(expected, actual)); + } + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java new file mode 100644 index 000000000000..a68712cb60ab --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java @@ -0,0 +1,62 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.keys; + +import org.gradoop.common.model.api.entities.Element; +import org.gradoop.common.model.api.entities.Labeled; +import org.gradoop.common.model.api.entities.VertexFactory; +import org.gradoop.common.model.impl.pojo.EPGMVertex; +import org.gradoop.flink.model.api.functions.KeyFunction; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * Test for the {@link LabelKeyFunction}. + */ +public class LabelKeyFunctionTest extends KeyFunctionTestBase { + + @Override + protected KeyFunction getInstance() { + return new LabelKeyFunction<>(); + } + + @Override + protected Map getTestElements() { + final VertexFactory vertexFactory = getConfig().getLogicalGraphFactory().getVertexFactory(); + Map testCases = new HashMap<>(); + testCases.put(vertexFactory.createVertex(), ""); + final String testLabel = "test"; + testCases.put(vertexFactory.createVertex(testLabel), testLabel); + return testCases; + } + + /** + * Test for the {@link LabelKeyFunction#addKeyToElement(Labeled, Object)} function. + */ + @Test + public void testAddKeyToElement() { + final EPGMVertex testVertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); + final String testLabel = "testLabel"; + assertNotEquals(testLabel, testVertex.getLabel()); + getInstance().addKeyToElement(testVertex, testLabel); + assertEquals(testLabel, testVertex.getLabel()); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java index 97da131997c1..0ca93ccc24f6 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.gradoop.common.GradoopTestUtils; import org.gradoop.common.model.impl.pojo.EPGMVertex; import org.gradoop.common.model.impl.properties.PropertyValue; import org.gradoop.flink.model.GradoopFlinkTestBase; @@ -44,6 +45,7 @@ import static org.gradoop.flink.model.impl.operators.keyedgrouping.GroupingKeys.property; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -237,6 +239,27 @@ public void testAddKeyToElement() { assertEquals(valueForDefault, testVertex.getPropertyValue("forDefault")); } + /** + * Test if the {@link LabelSpecificKeyFunction#isDefaultKey(Object)} method works as expected. + */ + @Test + public void testIsDefaultKey() { + // The default values are set initially, check them for each label + final EPGMVertex vertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); + for (String label : Arrays.asList("a", "b", "c", "")) { + vertex.setLabel(label); + assertTrue("Default key check fail for label: " + label, + testFunction.isDefaultKey(testFunction.getKey(vertex))); + } + // Changing the values on the vertex should only affect the result when the key function corresponding + // to the value is used. We check this by setting the property for label "c" to a non-default value. + vertex.setProperty("forC", PropertyValue.create(GradoopTestUtils.BIG_DECIMAL_VAL_7)); + vertex.setLabel("a"); + assertTrue(testFunction.isDefaultKey(testFunction.getKey(vertex))); + vertex.setLabel("c"); + assertFalse(testFunction.isDefaultKey(testFunction.getKey(vertex))); + } + /** * Check if two tuples are equal. This is necessary since {@code byte[]} does not have a valid {@code * equals} implementation. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java new file mode 100644 index 000000000000..331649844318 --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java @@ -0,0 +1,69 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.keys; + +import org.gradoop.common.GradoopTestUtils; +import org.gradoop.common.model.api.entities.Attributed; +import org.gradoop.common.model.api.entities.Element; +import org.gradoop.common.model.api.entities.VertexFactory; +import org.gradoop.common.model.impl.pojo.EPGMVertex; +import org.gradoop.common.model.impl.properties.PropertyValue; +import org.gradoop.flink.model.api.functions.KeyFunction; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.*; + +/** + * Test for the {@link PropertyKeyFunction}. + */ +public class PropertyKeyFunctionTest extends KeyFunctionTestBase { + + /** + * The key of the properties to extract. + */ + private final String key = "key"; + + @Override + protected KeyFunction getInstance() { + return new PropertyKeyFunction<>(key); + } + + @Override + protected Map getTestElements() { + final VertexFactory vertexFactory = getConfig().getLogicalGraphFactory().getVertexFactory(); + Map testCases = new HashMap<>(); + testCases.put(vertexFactory.createVertex(), PropertyValue.NULL_VALUE.getRawBytes()); + EPGMVertex testVertex = vertexFactory.createVertex(); + testVertex.setProperty(key, GradoopTestUtils.LONG_VAL_3); + testCases.put(testVertex, PropertyValue.create(GradoopTestUtils.LONG_VAL_3).getRawBytes()); + return testCases; + } + + /** + * Test for the {@link PropertyKeyFunction#addKeyToElement(Attributed, Object)} function. + */ + @Test + public void testAddKeyToElement() { + final EPGMVertex testVertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); + final PropertyValue testValue = PropertyValue.create(GradoopTestUtils.BIG_DECIMAL_VAL_7); + assertFalse(testVertex.hasProperty(key)); + getInstance().addKeyToElement(testVertex, testValue.getRawBytes()); + assertEquals(testValue, testVertex.getPropertyValue(key)); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java new file mode 100644 index 000000000000..417cf6d2a452 --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java @@ -0,0 +1,82 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific; + +import org.apache.flink.api.java.tuple.Tuple; +import org.gradoop.common.model.api.entities.Element; +import org.gradoop.flink.model.GradoopFlinkTestBase; +import org.gradoop.flink.model.api.functions.KeyFunction; +import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.CompositeKeyFunction; +import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.LabelKeyFunction; +import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.PropertyKeyFunction; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.gradoop.common.GradoopTestUtils.BIG_DECIMAL_VAL_7; +import static org.gradoop.common.GradoopTestUtils.KEY_0; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test for the {@link WithAllKeysSetToDefault} filter function. + */ +public class WithAllKeysSetToDefaultTest extends GradoopFlinkTestBase { + + /** + * A key function that can not be used with the filter function. + */ + private final KeyFunction invalidKeyFunction = new CompositeKeyFunction<>( + Collections.emptyList()); + + /** + * A key function that can be used with the filter function. + */ + private final LabelKeyFunction validKeyFunction = new LabelKeyFunction<>(); + + /** + * Another key function that can be used with the filter function. + */ + private final PropertyKeyFunction validKeyFunction2 = new PropertyKeyFunction<>(KEY_0); + + /** + * Test if the constructor throws an {@link IllegalArgumentException} when a non-supported function + * is supplied. + */ + @Test(expected = IllegalArgumentException.class) + public void testConstructorWithInvalidKey() { + new WithAllKeysSetToDefault<>(Arrays.asList(validKeyFunction, invalidKeyFunction)); + } + + /** + * Test if the filter works as expected. + */ + @Test + public void testFilter() { + final WithAllKeysSetToDefault filter = new WithAllKeysSetToDefault<>(Arrays.asList( + validKeyFunction, validKeyFunction2)); + final Element vertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); + assertTrue(filter.filter(vertex)); + vertex.setLabel("a"); + assertFalse(filter.filter(vertex)); + vertex.setLabel(""); + vertex.setProperty(KEY_0, BIG_DECIMAL_VAL_7); + assertFalse(filter.filter(vertex)); + vertex.setLabel("a"); + assertFalse(filter.filter(vertex)); + } +} From 01d912aaf07217233d497aa458d7db6ca76d5bea Mon Sep 17 00:00:00 2001 From: Philip Fritzsche Date: Wed, 18 Dec 2019 17:00:33 +0100 Subject: [PATCH 2/7] [#1433] Handle edges between retained vertices. --- .../impl/operators/grouping/Grouping.java | 9 +-- .../keyedgrouping/KeyedGrouping.java | 72 +++++++++++++++---- .../functions/BuildTuplesFromEdges.java | 28 +++++++- .../functions/BuildTuplesFromEdgesWithId.java | 51 +++++++++++++ .../functions/CreateElementMappingToSelf.java | 42 +++++++++++ .../functions/FilterEdgesToGroup.java | 33 +++++++++ .../functions/GroupingConstants.java | 14 ++++ .../functions/PickRetainedEdgeIDs.java | 37 ++++++++++ .../functions/UpdateIdFieldAndMarkTuple.java | 60 ++++++++++++++++ .../WithAllKeysSetToDefault.java | 4 +- 10 files changed, 327 insertions(+), 23 deletions(-) create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java create mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/grouping/Grouping.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/grouping/Grouping.java index 41cb05aea9e7..120a58729380 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/grouping/Grouping.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/grouping/Grouping.java @@ -779,13 +779,10 @@ GC extends BaseGraphCollection> UnaryBaseGraphToBaseGraphOperat retainVerticesWithoutGroup); break; case GROUP_WITH_KEYFUNCTIONS: - if (retainVerticesWithoutGroup) { - throw new UnsupportedOperationException("Retaining vertices without group is not yet supported" + - " with this strategy."); - } - groupingOperator = KeyedGroupingUtils.createInstance( + groupingOperator = KeyedGroupingUtils.createInstance( useVertexLabel, useEdgeLabel, vertexLabelGroups, edgeLabelGroups, - globalVertexAggregateFunctions, globalEdgeAggregateFunctions); + globalVertexAggregateFunctions, globalEdgeAggregateFunctions) + .setRetainUngroupedVertices(retainVerticesWithoutGroup); break; default: throw new IllegalArgumentException("Unsupported strategy: " + strategy); diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java index 6d57c8ca47ec..c78bd83dcef0 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java @@ -29,16 +29,23 @@ import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.api.operators.UnaryBaseGraphToBaseGraphOperator; +import org.gradoop.flink.model.impl.functions.epgm.Id; import org.gradoop.flink.model.impl.functions.filters.Not; +import org.gradoop.flink.model.impl.functions.utils.LeftSide; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.BuildSuperEdgeFromTuple; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.BuildSuperVertexFromTuple; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.BuildTuplesFromEdges; +import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.BuildTuplesFromEdgesWithId; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.BuildTuplesFromVertices; +import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.CreateElementMappingToSelf; +import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.FilterEdgesToGroup; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.FilterSuperVertices; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.GroupingConstants; +import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.PickRetainedEdgeIDs; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.ReduceEdgeTuples; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.ReduceVertexTuples; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.UpdateIdField; +import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.UpdateIdFieldAndMarkTuple; import org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific.WithAllKeysSetToDefault; import java.util.Collections; @@ -138,8 +145,8 @@ step. Those tuples will then be grouped by the respective key fields (the fields DataSet ungrouped = vertices; if (retainUngroupedVertices) { final FilterFunction retentionSelector = new WithAllKeysSetToDefault<>(vertexGroupingKeys); - ungrouped = ungrouped.filter(new Not<>(retentionSelector)); - vertices = vertices.filter(retentionSelector); + ungrouped = ungrouped.filter(retentionSelector); + vertices = vertices.filter(new Not<>(retentionSelector)); } DataSet verticesWithSuperVertex = vertices .map(new BuildTuplesFromVertices<>(vertexGroupingKeys, vertexAggregateFunctions)) @@ -150,22 +157,41 @@ step. Those tuples will then be grouped by the respective key fields (the fields DataSet> idToSuperId = verticesWithSuperVertex .filter(new Not<>(new FilterSuperVertices<>())) .project(GroupingConstants.VERTEX_TUPLE_ID, GroupingConstants.VERTEX_TUPLE_SUPERID); + if (retainUngroupedVertices) { + /* Retained vertices will be mapped to themselves, instead of a super-vertex. */ + idToSuperId = idToSuperId.union(ungrouped.map(new CreateElementMappingToSelf<>())); + } + final int edgeOffset = retainUngroupedVertices ? + GroupingConstants.EDGE_RETENTION_OFFSET : GroupingConstants.EDGE_DEFAULT_OFFSET; /* Create tuple representations of each edge and update the source- and target-ids of those tuples with - with the mapping extracted in the previous step. Edges will then point from and to super-vertices. */ + with the mapping extracted in the previous step. Edges will then point from and to super-vertices. + When retention of ungrouped vertices is enabled, we keep track of edge IDs to pick those that point + to and from retained vertices later. The ID is stored at the beginning of the tuple, we therefore + add some additional offset for these operations. */ DataSet edgesWithUpdatedIds = graph.getEdges() - .map(new BuildTuplesFromEdges<>(edgeGroupingKeys, edgeAggregateFunctions)) - .leftOuterJoin(idToSuperId) - .where(GroupingConstants.EDGE_TUPLE_SOURCEID) + .map(retainUngroupedVertices ? + new BuildTuplesFromEdgesWithId<>(edgeGroupingKeys, edgeAggregateFunctions) : + new BuildTuplesFromEdges<>(edgeGroupingKeys, edgeAggregateFunctions)) + .join(idToSuperId) + .where(GroupingConstants.EDGE_TUPLE_SOURCEID + edgeOffset) .equalTo(GroupingConstants.VERTEX_TUPLE_ID) - .with(new UpdateIdField<>(GroupingConstants.EDGE_TUPLE_SOURCEID)) - .leftOuterJoin(idToSuperId) - .where(GroupingConstants.EDGE_TUPLE_TARGETID) + .with(retainUngroupedVertices ? + new UpdateIdFieldAndMarkTuple<>(GroupingConstants.EDGE_TUPLE_SOURCEID) : + new UpdateIdField<>(GroupingConstants.EDGE_TUPLE_SOURCEID)) + .join(idToSuperId) + .where(GroupingConstants.EDGE_TUPLE_TARGETID + edgeOffset) .equalTo(GroupingConstants.VERTEX_TUPLE_ID) - .with(new UpdateIdField<>(GroupingConstants.EDGE_TUPLE_TARGETID)); + .with(retainUngroupedVertices ? + new UpdateIdFieldAndMarkTuple<>(GroupingConstants.EDGE_TUPLE_TARGETID) : + new UpdateIdField<>(GroupingConstants.EDGE_TUPLE_TARGETID)); - /* Group the edge-tuples by the key fields and vertex IDs and reduce them to single elements. */ - DataSet superEdgeTuples = edgesWithUpdatedIds + /* Group the edge-tuples by the key fields and vertex IDs and reduce them to single elements. + When retention of ungrouped vertices is enabled, we have to filter out edges marked for retention + before the grouping step and then project to remove the additional ID field. */ + DataSet superEdgeTuples = retainUngroupedVertices ? edgesWithUpdatedIds + .filter(new FilterEdgesToGroup<>()) + .project(getInternalEdgeProjectionIndices()) : edgesWithUpdatedIds .groupBy(getInternalEdgeGroupingKeys()) .reduceGroup(new ReduceEdgeTuples<>( GroupingConstants.EDGE_TUPLE_RESERVED + edgeGroupingKeys.size(), edgeAggregateFunctions)) @@ -186,7 +212,15 @@ step. Those tuples will then be grouped by the respective key fields (the fields if (retainUngroupedVertices) { /* We have to add the previously filtered vertices back. */ superVertices = superVertices.union(ungrouped); + /* We have to select the retained edges and add them back. */ + DataSet retainedEdgeIds = edgesWithUpdatedIds.flatMap(new PickRetainedEdgeIDs<>()); + DataSet retainedEdges = graph.getEdges().join(retainedEdgeIds) + .where(new Id<>()) + .equalTo("*") + .with(new LeftSide<>()); + superEdges = superEdges.union(retainedEdges); } + return graph.getFactory().fromDataSets(superVertices, superEdges); } @@ -210,6 +244,20 @@ private int[] getInternalVertexGroupingKeys() { GroupingConstants.VERTEX_TUPLE_RESERVED + vertexGroupingKeys.size()).toArray(); } + /** + * Get the indices to which edge tuples should be projected to remove the additional and at this stage + * no longer required {@link GroupingConstants#EDGE_TUPLE_ID} field. This will effectively return all + * the indices of all fields, except for that ID field.

+ * This is only needed when {@link #retainUngroupedVertices} is enabled. + * + * @return The edge tuple indices. + */ + private int[] getInternalEdgeProjectionIndices() { + return IntStream.range(GroupingConstants.EDGE_RETENTION_OFFSET, GroupingConstants.EDGE_RETENTION_OFFSET + + GroupingConstants.EDGE_TUPLE_RESERVED + edgeGroupingKeys.size() + edgeAggregateFunctions.size()) + .toArray(); + } + /** * Enable or disable an optional combine step before the reduce step. * Note that this currently only affects the edge reduce step. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdges.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdges.java index b6b8a9dfa199..da2a9917eb98 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdges.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdges.java @@ -30,6 +30,28 @@ */ public class BuildTuplesFromEdges extends BuildTuplesFromElements { + /** + * An additional edge offset. All tuple accesses will be shifted by this value. + */ + private final int offset; + + /** + * Initialize this function, setting the grouping keys and aggregate functions.

+ * This constructor will consider additional reserved fields in the edge tuple. + * + * @param keys The grouping keys. + * @param aggregateFunctions The aggregate functions used to determine the aggregate property + * @param additionalOffset An additional number of fields to be reserved in edge tuples. + */ + protected BuildTuplesFromEdges(List> keys, List aggregateFunctions, + int additionalOffset) { + super(GroupingConstants.EDGE_TUPLE_RESERVED + additionalOffset, keys, aggregateFunctions); + if (additionalOffset < 0) { + throw new IllegalArgumentException("Additional offset can not be negative: " + additionalOffset); + } + this.offset = additionalOffset; + } + /** * Initialize this function, setting the grouping keys and aggregate functions. * @@ -37,14 +59,14 @@ public class BuildTuplesFromEdges extends BuildTuplesFromElement * @param aggregateFunctions The aggregate functions used to determine the aggregate property */ public BuildTuplesFromEdges(List> keys, List aggregateFunctions) { - super(GroupingConstants.EDGE_TUPLE_RESERVED, keys, aggregateFunctions); + this(keys, aggregateFunctions, 0); } @Override public Tuple map(E element) throws Exception { final Tuple result = super.map(element); - result.setField(element.getSourceId(), GroupingConstants.EDGE_TUPLE_SOURCEID); - result.setField(element.getTargetId(), GroupingConstants.EDGE_TUPLE_TARGETID); + result.setField(element.getSourceId(), GroupingConstants.EDGE_TUPLE_SOURCEID + offset); + result.setField(element.getTargetId(), GroupingConstants.EDGE_TUPLE_TARGETID + offset); return result; } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java new file mode 100644 index 000000000000..51627584ee49 --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java @@ -0,0 +1,51 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.java.tuple.Tuple; +import org.gradoop.common.model.api.entities.Edge; +import org.gradoop.flink.model.api.functions.AggregateFunction; +import org.gradoop.flink.model.api.functions.KeyFunction; + +import java.util.List; + +/** + * Build a tuple-based representation of edges for grouping with an additional source ID field at position + * {@value GroupingConstants#EDGE_TUPLE_ID}. All other fields will be shifted by + * {@value GroupingConstants#EDGE_RETENTION_OFFSET}. + * + * @param The edge type. + */ +public class BuildTuplesFromEdgesWithId extends BuildTuplesFromEdges { + + /** + * Initialize this function, setting the grouping keys and aggregate functions. + * + * @param keys The edge grouping keys. + * @param aggregateFunctions The edge aggregate functions. + */ + public BuildTuplesFromEdgesWithId(List> keys, + List aggregateFunctions) { + super(keys, aggregateFunctions, GroupingConstants.EDGE_RETENTION_OFFSET); + } + + @Override + public Tuple map(E element) throws Exception { + final Tuple tuple = super.map(element); + tuple.setField(element.getId(), GroupingConstants.EDGE_TUPLE_ID); + return tuple; + } +} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java new file mode 100644 index 000000000000..60e1135ff5d6 --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java @@ -0,0 +1,42 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.gradoop.common.model.api.entities.Identifiable; +import org.gradoop.common.model.impl.id.GradoopId; + +/** + * Create a mapping (in the form of a {@link Tuple2}) from the ID of an element to itself. + * + * @param The element type. + */ +public class CreateElementMappingToSelf + implements MapFunction> { + + /** + * Reduce object instantiations. + */ + private final Tuple2 reuse = new Tuple2<>(); + + @Override + public Tuple2 map(E element) { + reuse.f0 = element.getId(); + reuse.f1 = element.getId(); + return reuse; + } +} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java new file mode 100644 index 000000000000..6e48c047d064 --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java @@ -0,0 +1,33 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.java.tuple.Tuple; +import org.gradoop.common.model.impl.id.GradoopId; + +/** + * A filter function accepting all edges that were not marked for retention. + * + * @param The edge tuple type. + */ +public class FilterEdgesToGroup implements FilterFunction { + + @Override + public boolean filter(T tuple) { + return tuple.getField(GroupingConstants.EDGE_TUPLE_ID).equals(GradoopId.NULL_VALUE); + } +} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/GroupingConstants.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/GroupingConstants.java index a4f3b99b0900..b90988f91f6a 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/GroupingConstants.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/GroupingConstants.java @@ -43,4 +43,18 @@ public abstract class GroupingConstants { * The number of reserved fields in the tuple-representation of an edge. */ public static final int EDGE_TUPLE_RESERVED = 2; + /** + * The number of additionally reserved fields in the tuple-representation of an edge. + */ + public static final int EDGE_DEFAULT_OFFSET = 0; + /** + * The number of additionally reserved fields in the tuple-representation of an edge, when retention of + * ungrouped vertices is enabled. + */ + public static final int EDGE_RETENTION_OFFSET = 1; + /** + * The index of the ID in the tuple-representation of an edge. This will only be available + * if retention of ungrouped vertices is enabled. + */ + public static final int EDGE_TUPLE_ID = 0; } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java new file mode 100644 index 000000000000..58e29281c13b --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java @@ -0,0 +1,37 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.util.Collector; +import org.gradoop.common.model.impl.id.GradoopId; + +/** + * Picks the ID from a tuple if that ID is not {@link GradoopId#NULL_VALUE}. + * + * @param The input tuple type. + */ +public class PickRetainedEdgeIDs implements FlatMapFunction { + + @Override + public void flatMap(T tuple, Collector collector) { + final GradoopId id = tuple.getField(GroupingConstants.EDGE_TUPLE_ID); + if (!id.equals(GradoopId.NULL_VALUE)) { + collector.collect(id); + } + } +} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java new file mode 100644 index 000000000000..fb2ce5aec68f --- /dev/null +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java @@ -0,0 +1,60 @@ +/* + * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.gradoop.common.model.impl.id.GradoopId; + +/** + * Updates the an ID field of an edge tuple to the ID of the corresponding super vertex.

+ * This function is used when retention of ungrouped vertices is enabled. In this case edge tuples have an + * additional ID field. This field will initially be equal to the ID of the edge. When the ID field is + * updated by this function, that field will be set to {@link GradoopId#NULL_VALUE} instead. + * + * @param The edge tuple type. + */ +public class UpdateIdFieldAndMarkTuple + implements JoinFunction, T> { + + /** + * The index of the field to update. + */ + private final int index; + + /** + * Create a new instance of this update function. + * + * @param index The index of the field to update (without offset). + */ + public UpdateIdFieldAndMarkTuple(int index) { + if (index < 0) { + throw new IllegalArgumentException("Index can not be negative."); + } + this.index = index + GroupingConstants.EDGE_RETENTION_OFFSET; + } + + @Override + public T join(T edgeTuple, Tuple2 mapping) { + if (!mapping.f0.equals(mapping.f1)) { + // Mark the tuple and update the field, if the mapping would actually change it. + edgeTuple.setField(GradoopId.NULL_VALUE, GroupingConstants.EDGE_TUPLE_ID); + edgeTuple.setField(mapping.f1, index); + } + return edgeTuple; + } +} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java index 4ac55cbb2015..01539e668aa5 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java @@ -65,8 +65,8 @@ public boolean filter(E value) { public static void checkKeySupport(List> keys) { for (KeyFunction key : keys) { if (!(key instanceof DefaultKeyCheckable)) { - throw new IllegalArgumentException("Key function " + key + " does not implement " + - DefaultKeyCheckable.class.getName()); + throw new IllegalArgumentException("Key function " + key.getClass().getName() + + " does not implement " + DefaultKeyCheckable.class.getName()); } } } From 7ba54c0071a521c144f09ed293a340a103048dc3 Mon Sep 17 00:00:00 2001 From: Philip Fritzsche Date: Mon, 20 Jan 2020 16:01:36 +0100 Subject: [PATCH 3/7] [#1433] Fix vertex retention implementation and add tests. --- .../flink/io/impl/gdl/GDLConsoleOutput.java | 2 +- .../api/functions/DefaultKeyCheckable.java | 2 +- .../keyedgrouping/KeyedGrouping.java | 4 +- .../functions/BuildTuplesFromEdgesWithId.java | 2 +- .../functions/BuildTuplesFromElements.java | 2 +- .../functions/CreateElementMappingToSelf.java | 2 +- .../functions/FilterEdgesToGroup.java | 2 +- .../functions/FilterSuperVertices.java | 2 - .../functions/PickRetainedEdgeIDs.java | 2 +- .../functions/UpdateIdFieldAndMarkTuple.java | 4 +- .../WithAllKeysSetToDefault.java | 2 +- .../KeyedGroupingVertexRetentionTest.java | 247 ++++++++++++++++++ .../BuildTuplesFromEdgesWithIdTest.java | 60 +++++ .../functions/PickRetainedEdgeIDsTest.java | 51 ++++ .../UpdateIdFieldAndMarkTupleTest.java | 61 +++++ ...ositeKeyFunctionWithDefaultValuesTest.java | 19 +- .../keys/ConstantKeyFunctionTest.java | 2 +- .../keys/KeyFunctionTestBase.java | 2 +- .../keys/LabelKeyFunctionTest.java | 2 +- .../keys/PropertyKeyFunctionTest.java | 5 +- .../WithAllKeysSetToDefaultTest.java | 2 +- 21 files changed, 456 insertions(+), 21 deletions(-) create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java create mode 100644 gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/io/impl/gdl/GDLConsoleOutput.java b/gradoop-flink/src/main/java/org/gradoop/flink/io/impl/gdl/GDLConsoleOutput.java index 70a4ea13cc02..d15e14728bfc 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/io/impl/gdl/GDLConsoleOutput.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/io/impl/gdl/GDLConsoleOutput.java @@ -16,8 +16,8 @@ package org.gradoop.flink.io.impl.gdl; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.gradoop.common.model.api.entities.GraphHead; import org.gradoop.common.model.api.entities.Edge; +import org.gradoop.common.model.api.entities.GraphHead; import org.gradoop.common.model.api.entities.Vertex; import org.gradoop.flink.model.api.epgm.BaseGraph; import org.gradoop.flink.model.api.epgm.BaseGraphCollection; diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java index c4eaa8231365..6bd287fb3d5c 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java index bc30c2cb7c07..78d07941f38b 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java @@ -189,9 +189,9 @@ step. Those tuples will then be grouped by the respective key fields (the fields /* Group the edge-tuples by the key fields and vertex IDs and reduce them to single elements. When retention of ungrouped vertices is enabled, we have to filter out edges marked for retention before the grouping step and then project to remove the additional ID field. */ - DataSet superEdgeTuples = retainUngroupedVertices ? edgesWithUpdatedIds + DataSet superEdgeTuples = (retainUngroupedVertices ? edgesWithUpdatedIds .filter(new FilterEdgesToGroup<>()) - .project(getInternalEdgeProjectionIndices()) : edgesWithUpdatedIds + .project(getInternalEdgeProjectionIndices()) : edgesWithUpdatedIds) .groupBy(getInternalEdgeGroupingKeys()) .reduceGroup(new ReduceEdgeTuples<>( GroupingConstants.EDGE_TUPLE_RESERVED + edgeGroupingKeys.size(), edgeAggregateFunctions)) diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java index 51627584ee49..d6b7ccd8e578 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java index e5bec526e704..deb3f58194d6 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java @@ -105,7 +105,7 @@ public BuildTuplesFromElements(int tupleDataOffset, List> keys reuseTuple = Tuple.newInstance(tupleSize); // Fill first fields with default ID values. for (int i = 0; i < tupleDataOffset; i++) { - reuseTuple.setField(GradoopId.NULL_VALUE, i); + reuseTuple.setField(GradoopId.NULL_VALUE.copy(), i); } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java index 60e1135ff5d6..aa8fc28f0e16 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java index 6e48c047d064..2f6194d07334 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterSuperVertices.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterSuperVertices.java index 13577352d205..67195cbd6b30 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterSuperVertices.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterSuperVertices.java @@ -16,7 +16,6 @@ package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.java.functions.FunctionAnnotation; import org.apache.flink.api.java.tuple.Tuple; import org.gradoop.common.model.impl.id.GradoopId; @@ -26,7 +25,6 @@ * * @param The type of the vertex-tuples. */ -@FunctionAnnotation.ReadFields({"f0", "f1"}) public class FilterSuperVertices implements FilterFunction { @Override diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java index 58e29281c13b..0390c1b1cab6 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java index fb2ce5aec68f..8eaf22e9f500 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -52,7 +52,7 @@ public UpdateIdFieldAndMarkTuple(int index) { public T join(T edgeTuple, Tuple2 mapping) { if (!mapping.f0.equals(mapping.f1)) { // Mark the tuple and update the field, if the mapping would actually change it. - edgeTuple.setField(GradoopId.NULL_VALUE, GroupingConstants.EDGE_TUPLE_ID); + GradoopId.NULL_VALUE.copyTo(edgeTuple.getField(GroupingConstants.EDGE_TUPLE_ID)); edgeTuple.setField(mapping.f1, index); } return edgeTuple; diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java index 01539e668aa5..a8ee7e436bc1 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java new file mode 100644 index 000000000000..5fc7e8c88694 --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java @@ -0,0 +1,247 @@ +/* + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping; + +import org.gradoop.common.model.impl.pojo.EPGMEdge; +import org.gradoop.common.model.impl.pojo.EPGMGraphHead; +import org.gradoop.common.model.impl.pojo.EPGMVertex; +import org.gradoop.flink.model.impl.epgm.GraphCollection; +import org.gradoop.flink.model.impl.epgm.LogicalGraph; +import org.gradoop.flink.model.impl.operators.aggregation.functions.count.Count; +import org.gradoop.flink.model.impl.operators.grouping.Grouping; +import org.gradoop.flink.model.impl.operators.grouping.GroupingStrategy; +import org.gradoop.flink.model.impl.operators.grouping.VertexRetentionTestBase; +import org.gradoop.flink.util.FlinkAsciiGraphLoader; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; +import static org.gradoop.flink.model.impl.operators.keyedgrouping.GroupingKeys.property; + +/** + * Tests if {@link KeyedGrouping#setRetainUngroupedVertices(boolean)} works as expected. + */ +public class KeyedGroupingVertexRetentionTest extends VertexRetentionTestBase { + + @Override + protected GroupingStrategy getStrategy() { + return GroupingStrategy.GROUP_WITH_KEYFUNCTIONS; + } + + /** + * This test is overwritten here, since it does not work in the same way for {@link KeyedGrouping}. + */ + @Override + public void testRetainVerticesFlag() { + } + + /** + * Test grouping on label and two properties.

+ * The expected result of this test is different, since vertices with one property or a label are grouped + * together according to that property or label. + * + * @throws Exception When the execution in Flink fails. + */ + @Override + public void testGroupByLabelAndProperties() throws Exception { + String asciiInput = "input[" + + "(v1 {a : 1})" + + "(v2 {a : 1, b : 2})" + + "(v4:B {a : 1})" + + "(v5:B {a : 1, b : 2})" + + "]"; + + FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); + + loader.appendToDatabaseFromString( + "expected[" + + "(v01 {a : 1, b: NULL, count: 1L})" + + "(v02 {a : 1, b : 2, count : 1L})" + + "(v04:B {a : 1, b: NULL, count : 1L})" + + "(v05:B {a : 1, b : 2, count : 1L})" + + "]"); + + final LogicalGraph input = loader.getLogicalGraphByVariable("input"); + + LogicalGraph output = new Grouping.GroupingBuilder() + .setStrategy(getStrategy()) + .retainVerticesWithoutGroup() + .useVertexLabel(true) + .addVertexGroupingKeys(Arrays.asList("a", "b")) + .addVertexAggregateFunction(new Count()) + .build() + .execute(input); + + collectAndAssertTrue( + output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); + } + + /** + * Test grouping on labels and one property with retention enabled.

+ * The expected result is different, since the two vertices with label "B" and no property "a" are + * grouped together. + * + * @throws Exception when the execution in Flink fails. + */ + @Override + public void testGroupByLabelAndProperty() throws Exception { + String asciiInput = "input[" + + "(v0 {})" + + "(v1 {a : 1})" + + "(v2 {b : 2})" + + "(v3:B {})" + + "(v4:B {a : 1})" + + "(v5:B {b : 2})" + + "]"; + + FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); + + loader.appendToDatabaseFromString( + "expected[" + + "(v00 {})" + + "(v01 {a : 1, count : 1L})" + + "(v02 {b : 2})" + + "(v03:B {a: NULL, count: 2L})" + + "(v04:B {a : 1, count : 1L})" + + "]"); + + final LogicalGraph input = loader.getLogicalGraphByVariable("input"); + + LogicalGraph output = new Grouping.GroupingBuilder() + .setStrategy(getStrategy()) + .retainVerticesWithoutGroup() + .useVertexLabel(true) + .addVertexGroupingKey("a") + .addVertexAggregateFunction(new Count()) + .build() + .execute(input); + + collectAndAssertTrue( + output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); + } + + /** + * Test grouping using two properties.

+ * The expected result is different here, since vertices with only one of two properties set are also + * grouped together. + * + * @throws Exception when the execution in Flink fails. + */ + @Override + public void testGroupByProperties() throws Exception { + String asciiInput = "input[" + + "(v1 {a : 1})" + + "(v2 {a : 1, b : 2})" + + "(v4:B {a : 1})" + + "(v5:B {a : 1, b : 2})" + + "]"; + + FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); + + loader.appendToDatabaseFromString( + "expected[" + + "(v01 {a : 1, b: NULL, count : 2L})" + + "(v0205 {a : 1, b : 2, count : 2L})" + + "]"); + + final LogicalGraph input = loader.getLogicalGraphByVariable("input"); + + LogicalGraph output = new Grouping.GroupingBuilder() + .setStrategy(getStrategy()) + .retainVerticesWithoutGroup() + .useVertexLabel(false) + .addVertexGroupingKeys(Arrays.asList("a", "b")) + .addVertexAggregateFunction(new Count()) + .build() + .execute(input); + + collectAndAssertTrue( + output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); + } + + /** + * Test label specific grouping with vertex retention enabled. + * + * @throws Exception when the execution in Flink fails. + */ + @Override + public void testLabelSpecificGrouping() throws Exception { + String asciiInput = "input[" + + "(v0 {})" + + "(v1 {a : 1})" + + "(v2 {a : 1, b : 2})" + + "(v3:B {})" + + "(v4:B {a : 1})" + + "(v5:B {a : 1, b : 2})" + + "(v6:A {})" + + "(v7:A {a : 1})" + + "(v8:A {a : 1, b : 2})" + + "]"; + + FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); + + loader.appendToDatabaseFromString( + "expected[" + + "(v00 {})" + + "(v01 {a : 1})" + + "(v02 {a : 1, b : 2})" + + "(v03:B {})" + + "(v04:B {a : 1})" + + "(v05:B {a : 1, b : 2})" + + "(v06:A)" + + "(v07:A {a : 1, b : NULL, count : 1L})" + + "(v08:A {a : 1, b : 2, count: 1L})" + + "]"); + + final LogicalGraph input = loader.getLogicalGraphByVariable("input"); + + LogicalGraph output = new Grouping.GroupingBuilder() + .setStrategy(getStrategy()) + .retainVerticesWithoutGroup() + .useVertexLabel(false) + .addVertexLabelGroup("A", "A", Arrays.asList("a", "b"), + Collections.singletonList(new Count())) + .build() + .execute(input); + + collectAndAssertTrue( + output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); + } + + /** + * Test if edges are properly updates when the source or target vertex was retained. + * + * @throws Exception when the execution in Flink fails. + */ + @Test + public void testEdgeUpdateWithRetainedSourceOrTarget() throws Exception { + FlinkAsciiGraphLoader loader = getLoaderFromString("input[" + + "(retained:Retained {otherprop: 1L})-[e:edge]->(notretained:NotRetained {prop: 1L, otherprop: 1L})" + + "-[:otherEdge {otherprop: 1L}]->(:NotRetained2 {prop:1L, otherprop: 2L})-[e2:edge2]->" + + "(retained2:RetainedTarget {otherprop: 2L})" + + "] expected [" + + "(retained)-->(resv {prop: 1L})-->(resv)-->(retained2)" + + "]"); + LogicalGraph res = loader.getLogicalGraphByVariable("input").callForGraph( + new KeyedGrouping( + singletonList(property("prop")), emptyList(), emptyList(), emptyList()) + .setRetainUngroupedVertices(true)); + collectAndAssertTrue(res.equalsByData(loader.getLogicalGraphByVariable("expected"))); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java new file mode 100644 index 000000000000..4d1ebc0d001b --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java @@ -0,0 +1,60 @@ +/* + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.java.tuple.Tuple; +import org.gradoop.common.model.api.entities.Edge; +import org.gradoop.common.model.impl.id.GradoopId; +import org.gradoop.common.model.impl.properties.PropertyValue; +import org.gradoop.flink.model.GradoopFlinkTestBase; +import org.gradoop.flink.model.impl.operators.aggregation.functions.max.MaxProperty; +import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.LabelKeyFunction; +import org.junit.Test; + +import static java.util.Collections.singletonList; +import static org.junit.Assert.assertEquals; + +/** + * Test for the {@link BuildTuplesFromEdgesWithId} function. + */ +public class BuildTuplesFromEdgesWithIdTest extends GradoopFlinkTestBase { + + /** + * Test the {@link BuildTuplesFromEdgesWithId#map(Edge)} functionality. + * + * @throws Exception when the function throws an exception + */ + @Test + public void testMap() throws Exception { + GradoopId source = GradoopId.get(); + GradoopId target = GradoopId.get(); + Edge testEdge = getConfig().getLogicalGraphFactory().getEdgeFactory().createEdge(source, target); + String testLabel = "a"; + String testAggKey = "key"; + PropertyValue testAggValue = PropertyValue.create(17L); + testEdge.setLabel(testLabel); + testEdge.setProperty(testAggKey, testAggValue); + BuildTuplesFromEdgesWithId function = new BuildTuplesFromEdgesWithId<>( + singletonList(new LabelKeyFunction<>()), singletonList(new MaxProperty(testAggKey))); + Tuple result = function.map(testEdge); + assertEquals("Invalid result tuple size", 5, result.getArity()); + assertEquals(testEdge.getId(), result.getField(0)); + assertEquals(source, result.getField(1)); + assertEquals(target, result.getField(2)); + assertEquals(testLabel, result.getField(3)); + assertEquals(testAggValue, result.getField(4)); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java new file mode 100644 index 000000000000..ec29b13c87c7 --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java @@ -0,0 +1,51 @@ +/* + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.common.functions.util.ListCollector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.util.Collector; +import org.gradoop.common.model.impl.id.GradoopId; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Test for the {@link PickRetainedEdgeIDs} function. + */ +public class PickRetainedEdgeIDsTest { + + /** + * Test the {@link PickRetainedEdgeIDs#flatMap(Tuple, Collector)} functionality. + */ + @Test + public void testFlatMap() { + PickRetainedEdgeIDs> toTest = new PickRetainedEdgeIDs<>(); + List result = new ArrayList<>(); + Collector collector = new ListCollector(result); + GradoopId someId = GradoopId.get(); + GradoopId otherId = GradoopId.get(); + toTest.flatMap(Tuple1.of(someId), collector); + toTest.flatMap(Tuple1.of(GradoopId.NULL_VALUE), collector); + toTest.flatMap(Tuple1.of(otherId), collector); + collector.close(); + assertArrayEquals(new GradoopId[] {someId, otherId}, result.toArray()); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java new file mode 100644 index 000000000000..b4ed71ed7de8 --- /dev/null +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java @@ -0,0 +1,61 @@ +/* + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.gradoop.flink.model.impl.operators.keyedgrouping.functions; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.gradoop.common.model.impl.id.GradoopId; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * Test for the {@link UpdateIdFieldAndMarkTuple} function. + */ +public class UpdateIdFieldAndMarkTupleTest { + + /** + * Test the join function in cases where the update should be performed. + */ + @Test + public void testWithUpdate() { + GradoopId id = GradoopId.get(); + GradoopId foreignKey = GradoopId.get(); + GradoopId newForeignKey = GradoopId.get(); + Tuple2 input = Tuple2.of(id, foreignKey); + Tuple2 mapping = Tuple2.of(foreignKey, newForeignKey); + assertNotEquals(GradoopId.NULL_VALUE, input.f0); + Tuple2 result = + new UpdateIdFieldAndMarkTuple>(0).join(input, mapping); + assertEquals(GradoopId.NULL_VALUE, result.f0); + assertEquals(newForeignKey, result.f1); + } + + /** + * Test the join function in cases where the update should not be performed. + */ + @Test + public void testWithNoUpdate() { + GradoopId id = GradoopId.get(); + GradoopId foreignKey = GradoopId.get(); + Tuple2 input = Tuple2.of(id, foreignKey); + Tuple2 mapping = Tuple2.of(foreignKey, foreignKey); + Tuple2 result = new UpdateIdFieldAndMarkTuple>(0) + .join(input, mapping); + assertEquals(id, result.f0); + assertEquals(foreignKey, result.f1); + } +} diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java index 64884a1e7cca..41c3578cc3c9 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,11 +17,13 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.gradoop.common.GradoopTestUtils; import org.gradoop.common.model.api.entities.Element; import org.gradoop.common.model.api.entities.VertexFactory; import org.gradoop.common.model.impl.pojo.EPGMVertex; import org.gradoop.common.model.impl.properties.PropertyValue; +import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.api.functions.KeyFunctionWithDefaultValue; import org.junit.Test; @@ -31,6 +33,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; /** * Test for the {@link CompositeKeyFunctionWithDefaultValues}. @@ -101,4 +104,18 @@ public void testAddKeyToElement() { assertEquals(testValue, vertex.getPropertyValue(propertyKey)); assertEquals(testLabel, vertex.getLabel()); } + + /** + * Test for the {@link CompositeKeyFunctionWithDefaultValues#isDefaultKey(Object)} function. + */ + @Test + public void testIsDefaultKey() { + KeyFunction keyFunction = getInstance(); + assertTrue(keyFunction instanceof DefaultKeyCheckable); + DefaultKeyCheckable toTest = (DefaultKeyCheckable) keyFunction; + assertFalse("wrong type", toTest.isDefaultKey(1L)); + assertFalse("wrong arity", toTest.isDefaultKey( + Tuple3.of(PropertyValue.NULL_VALUE.getRawBytes(), "", ""))); + assertTrue(toTest.isDefaultKey(Tuple2.of(PropertyValue.NULL_VALUE.getRawBytes(), ""))); + } } diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java index a5850abd8585..82f47661c136 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java index c81caa5f794d..a1b1e3572fbf 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java index a68712cb60ab..1c29f5b72b34 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java index 331649844318..a8a02884662a 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,7 +27,8 @@ import java.util.HashMap; import java.util.Map; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; /** * Test for the {@link PropertyKeyFunction}. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java index 417cf6d2a452..ca6af6548781 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2019 Leipzig University (Database Research Group) + * Copyright © 2014 - 2020 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From 10f36249529ede00a826603087205477dca4ca96 Mon Sep 17 00:00:00 2001 From: timo95 Date: Mon, 29 Mar 2021 16:20:18 +0200 Subject: [PATCH 4/7] [#1433] Update file headers --- .../gradoop/flink/model/api/functions/DefaultKeyCheckable.java | 2 +- .../keyedgrouping/functions/BuildTuplesFromEdgesWithId.java | 2 +- .../keyedgrouping/functions/CreateElementMappingToSelf.java | 2 +- .../operators/keyedgrouping/functions/FilterEdgesToGroup.java | 2 +- .../operators/keyedgrouping/functions/PickRetainedEdgeIDs.java | 2 +- .../keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java | 2 +- .../keyedgrouping/labelspecific/WithAllKeysSetToDefault.java | 2 +- .../keyedgrouping/KeyedGroupingVertexRetentionTest.java | 2 +- .../keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java | 2 +- .../keyedgrouping/functions/PickRetainedEdgeIDsTest.java | 2 +- .../keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java | 2 +- .../keys/CompositeKeyFunctionWithDefaultValuesTest.java | 2 +- .../operators/keyedgrouping/keys/ConstantKeyFunctionTest.java | 2 +- .../impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java | 2 +- .../impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java | 2 +- .../operators/keyedgrouping/keys/PropertyKeyFunctionTest.java | 2 +- .../labelspecific/WithAllKeysSetToDefaultTest.java | 2 +- 17 files changed, 17 insertions(+), 17 deletions(-) diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java index 6bd287fb3d5c..1669b30f7b46 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java index d6b7ccd8e578..d06ff56a03e9 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithId.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java index aa8fc28f0e16..355b88970ae1 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/CreateElementMappingToSelf.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java index 2f6194d07334..699f2da241c4 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/FilterEdgesToGroup.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java index 0390c1b1cab6..8c8dbdca2c71 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDs.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java index 8eaf22e9f500..84b0c32384a8 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTuple.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java index a8ee7e436bc1..c7181ca259aa 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java index 5fc7e8c88694..4781d1c1c0a9 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java index 4d1ebc0d001b..68f611d78ddb 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromEdgesWithIdTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java index ec29b13c87c7..28ba3fdf08e0 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/PickRetainedEdgeIDsTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java index b4ed71ed7de8..65825df06b03 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/UpdateIdFieldAndMarkTupleTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java index 41c3578cc3c9..7b8bd47c3a63 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java index 82f47661c136..d84659bc2a6f 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/ConstantKeyFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java index a1b1e3572fbf..3dbba0804855 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java index 1c29f5b72b34..f790b981c349 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelKeyFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java index a8a02884662a..67a7bc5b1b94 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/PropertyKeyFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java index ca6af6548781..ad8f028c5cc6 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java @@ -1,5 +1,5 @@ /* - * Copyright © 2014 - 2020 Leipzig University (Database Research Group) + * Copyright © 2014 - 2021 Leipzig University (Database Research Group) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From 7d67074bad812222448f6ed2869d66d63d0ddfc2 Mon Sep 17 00:00:00 2001 From: timo95 Date: Thu, 8 Apr 2021 12:06:32 +0200 Subject: [PATCH 5/7] Add test, fix bug, add more doc, make non defaultable key functions compatible. --- .../api/functions/DefaultKeyCheckable.java | 32 ------------- .../model/api/functions/KeyFunction.java | 12 +++++ .../KeyFunctionWithDefaultValue.java | 8 ++-- .../keyedgrouping/KeyedGrouping.java | 18 +++---- .../keys/CompositeKeyFunction.java | 5 ++ ...CompositeKeyFunctionWithDefaultValues.java | 18 ------- .../keys/PropertyKeyFunction.java | 5 ++ .../LabelSpecificKeyFunction.java | 48 ++++++++++--------- ...oDefault.java => WithAllKeysRetained.java} | 30 ++---------- .../grouping/VertexRetentionTestBase.java | 33 +++++++++++++ ...ositeKeyFunctionWithDefaultValuesTest.java | 17 ------- .../keys/KeyFunctionTestBase.java | 18 ------- .../keys/LabelSpecificKeyFunctionTest.java | 11 +++-- ...Test.java => WithAllKeysRetainedTest.java} | 25 ++-------- 14 files changed, 107 insertions(+), 173 deletions(-) delete mode 100644 gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java rename gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/{WithAllKeysSetToDefault.java => WithAllKeysRetained.java} (56%) rename gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/{WithAllKeysSetToDefaultTest.java => WithAllKeysRetainedTest.java} (68%) diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java deleted file mode 100644 index 1669b30f7b46..000000000000 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/DefaultKeyCheckable.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Copyright © 2014 - 2021 Leipzig University (Database Research Group) - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.gradoop.flink.model.api.functions; - -/** - * Indicates that a key-function can check if a key is considered a default value for that function.

- * This interface needs to be implemented on {@link KeyFunction key functions} where the default key - * is not unique, but depending on some other conditions, like the label of an element. - */ -public interface DefaultKeyCheckable { - - /** - * Check if a key is considered a default key for this function. - * - * @param key The key to check. - * @return {@code true}, if the key is effectively a default key. - */ - boolean isDefaultKey(Object key); -} diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunction.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunction.java index 99a931536e8e..1080cfc844d1 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunction.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunction.java @@ -54,4 +54,16 @@ default void addKeyToElement(E element, Object key) { * @return The key type. */ TypeInformation getType(); + + /** + * The element will not be grouped, if this returns true for all key functions and + * {@link org.gradoop.flink.model.impl.operators.keyedgrouping.KeyedGrouping#setRetainUngroupedVertices(boolean)} + * is set to true. + * + * @param element element to test + * @return true, if element should not be grouped + */ + default boolean retainElement(E element) { + return false; + } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java index 0b19b594775d..3306d60e72ec 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/api/functions/KeyFunctionWithDefaultValue.java @@ -25,7 +25,7 @@ * @param The type of the object from which the grouping key is extracted. * @param The type of the extracted key. */ -public interface KeyFunctionWithDefaultValue extends KeyFunction, DefaultKeyCheckable { +public interface KeyFunctionWithDefaultValue extends KeyFunction { /** * Get the default key value for all elements. @@ -40,11 +40,11 @@ public interface KeyFunctionWithDefaultValue extends KeyFunction, De * The default implementation of this method compares the key to the default key provided by this class * using {@link Objects#deepEquals(Object, Object)}. * - * @param key The key to check. + * @param element The key to check. * @return {@code true}, if the key is a default key for this key-function. */ @Override - default boolean isDefaultKey(Object key) { - return Objects.deepEquals(getDefaultKey(), key); + default boolean retainElement(E element) { + return Objects.deepEquals(getDefaultKey(), getKey(element)); } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java index 09e83ca0af8e..96198051c48a 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java @@ -26,7 +26,6 @@ import org.gradoop.flink.model.api.epgm.BaseGraph; import org.gradoop.flink.model.api.epgm.BaseGraphCollection; import org.gradoop.flink.model.api.functions.AggregateFunction; -import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.api.operators.UnaryBaseGraphToBaseGraphOperator; import org.gradoop.flink.model.impl.functions.epgm.Id; @@ -46,7 +45,7 @@ import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.ReduceVertexTuples; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.UpdateIdField; import org.gradoop.flink.model.impl.operators.keyedgrouping.functions.UpdateIdFieldAndMarkTuple; -import org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific.WithAllKeysSetToDefault; +import org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific.WithAllKeysRetained; import java.util.Collections; import java.util.List; @@ -144,7 +143,7 @@ step. Those tuples will then be grouped by the respective key fields (the fields DataSet vertices = graph.getVertices(); DataSet ungrouped = vertices; if (retainUngroupedVertices) { - final FilterFunction retentionSelector = new WithAllKeysSetToDefault<>(vertexGroupingKeys); + final FilterFunction retentionSelector = new WithAllKeysRetained<>(vertexGroupingKeys); ungrouped = ungrouped.filter(retentionSelector); vertices = vertices.filter(new Not<>(retentionSelector)); } @@ -153,6 +152,7 @@ step. Those tuples will then be grouped by the respective key fields (the fields .groupBy(getInternalVertexGroupingKeys()) .reduceGroup(new ReduceVertexTuples<>( GroupingConstants.VERTEX_TUPLE_RESERVED + vertexGroupingKeys.size(), vertexAggregateFunctions)); + /* Extract a mapping from vertex-ID to super-vertex-ID from the result of the vertex-reduce step. */ DataSet> idToSuperId = verticesWithSuperVertex .filter(new Not<>(new FilterSuperVertices<>())) @@ -274,19 +274,21 @@ public KeyedGrouping setUseGroupCombine(boolean useGroupCombine /** * Enable or disable vertex retention. + * Vertices will be retained, if all key functions return true for + * {@link KeyFunction#retainElement(Object)}. + * For example {@code KeyFunctionWithDefaultValue} returns true, + * if the extracted key equals the default key. *

- * Enabling this features requires that all vertex keys implement {@link DefaultKeyCheckable}. + * Retained vertices will not be grouped and returned without modifications. + * Edges between retained vertices will not be grouped and returned without modifications. + * Edges between retained vertices and grouped vertices will be grouped. *

* This is disabled per default. * * @param retainVertices Should vertices be retained? * @return This operator. - * @throws IllegalArgumentException When any vertex key function is not supported for this feature. */ public KeyedGrouping setRetainUngroupedVertices(boolean retainVertices) { - if (retainVertices) { - WithAllKeysSetToDefault.checkKeySupport(vertexGroupingKeys); - } this.retainUngroupedVertices = retainVertices; return this; } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java index dfacb33ff088..8b2947814b56 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunction.java @@ -81,4 +81,9 @@ public TypeInformation getType() { } return new TupleTypeInfo<>(types); } + + @Override + public boolean retainElement(T element) { + return componentFunctions.stream().allMatch(k -> k.retainElement(element)); + } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java index 86b0f019ae1b..bbc9fc943773 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValues.java @@ -52,22 +52,4 @@ public CompositeKeyFunctionWithDefaultValues(List getType() { public byte[] getDefaultKey() { return PropertyValue.NULL_VALUE.getRawBytes(); } + + @Override + public boolean retainElement(T element) { + return !element.hasProperty(propertyKey); + } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/LabelSpecificKeyFunction.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/LabelSpecificKeyFunction.java index d553c8d0a1ab..5f104778ebfe 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/LabelSpecificKeyFunction.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/LabelSpecificKeyFunction.java @@ -20,7 +20,6 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.gradoop.common.model.api.entities.Element; -import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.api.functions.KeyFunctionWithDefaultValue; import org.gradoop.flink.model.impl.operators.grouping.Grouping; @@ -31,15 +30,24 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; /** * A grouping key function that extracts grouping keys only for specific labels. + * It wraps a map of labels and key functions and returns a tuple of keys as key. + *

+ * The first position of the tuple represents the label of the element (stored as index). + * Each other position corresponds to a label. + *

+ * The position corresponding to the element label contains a combined key for that label. + * Every other position contains default values. + * If the element label is not represented in the tuple, a key function representing + * the default label group is used. * * @param The type of the elements to group. */ -public class LabelSpecificKeyFunction implements KeyFunction, - DefaultKeyCheckable { +public class LabelSpecificKeyFunction implements KeyFunction { /** * A label used to identify the default groups. @@ -51,7 +59,9 @@ public class LabelSpecificKeyFunction implements KeyFunction< public static final String DEFAULT_GROUP_LABEL = Grouping.DEFAULT_VERTEX_LABEL_GROUP; /** - * A map assigning an internally used number to each label. + * A map assigning an index to each label. + * It encodes the position of labels in {@code targetLabels} + * and of corresponding key functions in {@code keyFunctions}. */ private final Map labelToIndex; @@ -93,15 +103,18 @@ public LabelSpecificKeyFunction(Map Tuple.MAX_ARITY) { - throw new IllegalArgumentException("Too many labels. Tuple arity exceeded: " + (totalLabels + 1) + - " (max.: " + Tuple.MAX_ARITY + ")"); + throw new IllegalArgumentException( + String.format("Too many labels. Tuple arity exceeded: %d (max.: %d)", + totalLabels + 1, Tuple.MAX_ARITY)); } + int labelNr = 1; labelToIndex = new HashMap<>(); // The list needs to be filled initially, the set(int,Object) function will fail otherwise. keyFunctions = new ArrayList<>(Collections.nCopies(totalLabels, null)); targetLabels = new String[totalLabels]; - for (Map.Entry>> labelToKeys : labelsWithKeys.entrySet()) { + + for (Entry>> labelToKeys : labelsWithKeys.entrySet()) { final String key = labelToKeys.getKey(); final List> keysForLabel = labelToKeys.getValue(); if (key.equals(defaultGroupLabel)) { @@ -116,8 +129,9 @@ public LabelSpecificKeyFunction(Map(keysForLabel)); labelNr++; } + if (labelToSuperLabel != null) { - for (Map.Entry labelUpdateEntry : labelToSuperLabel.entrySet()) { + for (Entry labelUpdateEntry : labelToSuperLabel.entrySet()) { Integer index = labelToIndex.get(labelUpdateEntry.getKey()); if (index == null) { continue; @@ -130,13 +144,10 @@ public LabelSpecificKeyFunction(Map getType() { } @Override - public boolean isDefaultKey(Object key) { - if (!(key instanceof Tuple)) { - return false; - } - Tuple keyTuple = (Tuple) key; - if (keyTuple.getArity() != (1 + keyFunctions.size())) { - return false; - } - Integer index = keyTuple.getField(0); - return keyFunctions.get(index).isDefaultKey(keyTuple.getField(1 + index)); + public boolean retainElement(T element) { + Integer index = labelToIndex.getOrDefault(element.getLabel(), 0); + return keyFunctions.get(index).retainElement(element); } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java similarity index 56% rename from gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java rename to gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java index c7181ca259aa..57b401b012c1 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefault.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java @@ -15,7 +15,6 @@ */ package org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific; -import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.impl.functions.filters.CombinableFilter; @@ -27,7 +26,7 @@ * * @param The type of the elements to filter. */ -public class WithAllKeysSetToDefault implements CombinableFilter { +public class WithAllKeysRetained implements CombinableFilter { /** * The keys to check on each element. @@ -39,35 +38,12 @@ public class WithAllKeysSetToDefault implements CombinableFilter { * * @param keys The list of key functions to check. */ - public WithAllKeysSetToDefault(List> keys) { - checkKeySupport(keys); + public WithAllKeysRetained(List> keys) { this.keys = Objects.requireNonNull(keys); } @Override public boolean filter(E value) { - for (KeyFunction key : keys) { - final Object extractedKey = key.getKey(value); - if (!((DefaultKeyCheckable) key).isDefaultKey(extractedKey)) { - return false; - } - } - return true; - } - - /** - * Check if keys functions are supported with this filter function. - * - * @param keys The key functions to check. - * @param The type of the key functions. - * @throws IllegalArgumentException When any key function is not supported by this filter. - */ - public static void checkKeySupport(List> keys) { - for (KeyFunction key : keys) { - if (!(key instanceof DefaultKeyCheckable)) { - throw new IllegalArgumentException("Key function " + key.getClass().getName() + - " does not implement " + DefaultKeyCheckable.class.getName()); - } - } + return keys.stream().allMatch(k -> k.retainElement(value)); } } diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java index ec950d0e46fd..3bfafc6f9b44 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java @@ -116,6 +116,39 @@ public void testRetentionSingleProperty() throws Exception { output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); } + /** + * Tests correct retention of a vertex with a null property. + * + * @throws Exception if collecting result values fails + */ + @Test + public void testRetentionNullProperty() throws Exception { + String asciiInput = "input[" + + "(v0 {a: NULL})" + // group + "]"; + + FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); + + loader.appendToDatabaseFromString( + "expected[" + + "(v00 {a: NULL, count: 1L})" + + "]"); + + final LogicalGraph input = loader.getLogicalGraphByVariable("input"); + + LogicalGraph output = new Grouping.GroupingBuilder() + .setStrategy(getStrategy()) + .retainVerticesWithoutGroup() + .useVertexLabel(true) + .addVertexGroupingKey("a") + .addVertexAggregateFunction(new Count()) + .build() + .execute(input); + + collectAndAssertTrue( + output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); + } + /** * Tests function {@link Grouping#groupInternal(BaseGraph)}. * Tests correct retention of a vertex with multiple properties. diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java index 7b8bd47c3a63..90ebac2d815c 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/CompositeKeyFunctionWithDefaultValuesTest.java @@ -17,13 +17,11 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.gradoop.common.GradoopTestUtils; import org.gradoop.common.model.api.entities.Element; import org.gradoop.common.model.api.entities.VertexFactory; import org.gradoop.common.model.impl.pojo.EPGMVertex; import org.gradoop.common.model.impl.properties.PropertyValue; -import org.gradoop.flink.model.api.functions.DefaultKeyCheckable; import org.gradoop.flink.model.api.functions.KeyFunction; import org.gradoop.flink.model.api.functions.KeyFunctionWithDefaultValue; import org.junit.Test; @@ -33,7 +31,6 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; /** * Test for the {@link CompositeKeyFunctionWithDefaultValues}. @@ -104,18 +101,4 @@ public void testAddKeyToElement() { assertEquals(testValue, vertex.getPropertyValue(propertyKey)); assertEquals(testLabel, vertex.getLabel()); } - - /** - * Test for the {@link CompositeKeyFunctionWithDefaultValues#isDefaultKey(Object)} function. - */ - @Test - public void testIsDefaultKey() { - KeyFunction keyFunction = getInstance(); - assertTrue(keyFunction instanceof DefaultKeyCheckable); - DefaultKeyCheckable toTest = (DefaultKeyCheckable) keyFunction; - assertFalse("wrong type", toTest.isDefaultKey(1L)); - assertFalse("wrong arity", toTest.isDefaultKey( - Tuple3.of(PropertyValue.NULL_VALUE.getRawBytes(), "", ""))); - assertTrue(toTest.isDefaultKey(Tuple2.of(PropertyValue.NULL_VALUE.getRawBytes(), ""))); - } } diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java index 3dbba0804855..5814dc8ad12e 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/KeyFunctionTestBase.java @@ -21,7 +21,6 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.gradoop.flink.model.GradoopFlinkTestBase; import org.gradoop.flink.model.api.functions.KeyFunction; -import org.gradoop.flink.model.api.functions.KeyFunctionWithDefaultValue; import org.junit.Before; import org.junit.Test; @@ -35,7 +34,6 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assume.assumeTrue; /** * A base class for key-function tests. This provides some common tests that should pass by all keys to be @@ -90,22 +88,6 @@ public void checkTypeInfo() { assertNotEquals("Key type has no fields.", 0, type.getTotalFields()); } - /** - * Check if the default key value has the correct type.

- * This test will only run if the key function has a default key. - * - * @throws IOException when serialization of a key fails. - */ - @Test - public void checkDefaultKey() throws IOException { - KeyFunction function = getInstance(); - assumeTrue(function instanceof KeyFunctionWithDefaultValue); - KeyFunctionWithDefaultValue withDefaultValue = (KeyFunctionWithDefaultValue) function; - K defaultKey = withDefaultValue.getDefaultKey(); - checkKeyType(withDefaultValue.getType(), defaultKey); - assertTrue(withDefaultValue.isDefaultKey(defaultKey)); - } - /** * Check if a key is of a certain type and if it is serializable as that type. * diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java index 32fba1f27f25..c5ea07475b1b 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.gradoop.common.GradoopTestUtils; +import org.gradoop.common.model.api.entities.Element; import org.gradoop.common.model.impl.pojo.EPGMVertex; import org.gradoop.common.model.impl.properties.PropertyValue; import org.gradoop.flink.model.GradoopFlinkTestBase; @@ -240,24 +241,24 @@ public void testAddKeyToElement() { } /** - * Test if the {@link LabelSpecificKeyFunction#isDefaultKey(Object)} method works as expected. + * Test if the {@link LabelSpecificKeyFunction#retainElement(Element)} method works as expected. */ @Test - public void testIsDefaultKey() { + public void testElementRetention() { // The default values are set initially, check them for each label final EPGMVertex vertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); for (String label : Arrays.asList("a", "b", "c", "")) { vertex.setLabel(label); assertTrue("Default key check fail for label: " + label, - testFunction.isDefaultKey(testFunction.getKey(vertex))); + testFunction.retainElement(vertex)); } // Changing the values on the vertex should only affect the result when the key function corresponding // to the value is used. We check this by setting the property for label "c" to a non-default value. vertex.setProperty("forC", PropertyValue.create(GradoopTestUtils.BIG_DECIMAL_VAL_7)); vertex.setLabel("a"); - assertTrue(testFunction.isDefaultKey(testFunction.getKey(vertex))); + assertTrue(testFunction.retainElement(vertex)); vertex.setLabel("c"); - assertFalse(testFunction.isDefaultKey(testFunction.getKey(vertex))); + assertFalse(testFunction.retainElement(vertex)); } /** diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetainedTest.java similarity index 68% rename from gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java rename to gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetainedTest.java index ad8f028c5cc6..7daf32705e5b 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysSetToDefaultTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetainedTest.java @@ -15,17 +15,13 @@ */ package org.gradoop.flink.model.impl.operators.keyedgrouping.labelspecific; -import org.apache.flink.api.java.tuple.Tuple; import org.gradoop.common.model.api.entities.Element; import org.gradoop.flink.model.GradoopFlinkTestBase; -import org.gradoop.flink.model.api.functions.KeyFunction; -import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.CompositeKeyFunction; import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.LabelKeyFunction; import org.gradoop.flink.model.impl.operators.keyedgrouping.keys.PropertyKeyFunction; import org.junit.Test; import java.util.Arrays; -import java.util.Collections; import static org.gradoop.common.GradoopTestUtils.BIG_DECIMAL_VAL_7; import static org.gradoop.common.GradoopTestUtils.KEY_0; @@ -33,15 +29,9 @@ import static org.junit.Assert.assertTrue; /** - * Test for the {@link WithAllKeysSetToDefault} filter function. + * Test for the {@link WithAllKeysRetained} filter function. */ -public class WithAllKeysSetToDefaultTest extends GradoopFlinkTestBase { - - /** - * A key function that can not be used with the filter function. - */ - private final KeyFunction invalidKeyFunction = new CompositeKeyFunction<>( - Collections.emptyList()); +public class WithAllKeysRetainedTest extends GradoopFlinkTestBase { /** * A key function that can be used with the filter function. @@ -53,21 +43,12 @@ public class WithAllKeysSetToDefaultTest extends GradoopFlinkTestBase { */ private final PropertyKeyFunction validKeyFunction2 = new PropertyKeyFunction<>(KEY_0); - /** - * Test if the constructor throws an {@link IllegalArgumentException} when a non-supported function - * is supplied. - */ - @Test(expected = IllegalArgumentException.class) - public void testConstructorWithInvalidKey() { - new WithAllKeysSetToDefault<>(Arrays.asList(validKeyFunction, invalidKeyFunction)); - } - /** * Test if the filter works as expected. */ @Test public void testFilter() { - final WithAllKeysSetToDefault filter = new WithAllKeysSetToDefault<>(Arrays.asList( + final WithAllKeysRetained filter = new WithAllKeysRetained<>(Arrays.asList( validKeyFunction, validKeyFunction2)); final Element vertex = getConfig().getLogicalGraphFactory().getVertexFactory().createVertex(); assertTrue(filter.filter(vertex)); From a338b3f053487c5a9158a6148a175b0a3cc773a9 Mon Sep 17 00:00:00 2001 From: timo95 Date: Thu, 8 Apr 2021 14:01:28 +0200 Subject: [PATCH 6/7] Update test, update javadoc. --- .../keyedgrouping/KeyedGrouping.java | 8 +++-- .../functions/BuildTuplesFromElements.java | 1 + .../labelspecific/WithAllKeysRetained.java | 2 +- .../grouping/VertexRetentionTestBase.java | 31 +++++++++++++++++++ .../KeyedGroupingVertexRetentionTest.java | 30 +++--------------- .../keys/LabelSpecificKeyFunctionTest.java | 2 +- 6 files changed, 43 insertions(+), 31 deletions(-) diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java index 96198051c48a..ff0e0a74d43d 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGrouping.java @@ -103,7 +103,7 @@ public class KeyedGrouping< private boolean useGroupCombine = true; /** - * Should vertices with all default keys be kept as is? + * Should vertices without groups be retained in the result as is. */ private boolean retainUngroupedVertices = false; @@ -137,8 +137,8 @@ public KeyedGrouping(List> vertexGroupingKeys, @Override public LG execute(LG graph) { /* First we create tuple representations of each vertex. - If retention of ungrouped vertices is enabled, we filter out vertices with unset keys prior to this - step. Those tuples will then be grouped by the respective key fields (the fields containing the values + If retention of ungrouped vertices is enabled, we filter out retained vertices prior to this step. + Those tuples will then be grouped by the respective key fields (the fields containing the values extracted by the key functions) and reduced to assign a super vertex and to calculate aggregates. */ DataSet vertices = graph.getVertices(); DataSet ungrouped = vertices; @@ -147,6 +147,8 @@ step. Those tuples will then be grouped by the respective key fields (the fields ungrouped = ungrouped.filter(retentionSelector); vertices = vertices.filter(new Not<>(retentionSelector)); } + + /* Group vertices and create super-vertices. Do not yet remove the input vertices. */ DataSet verticesWithSuperVertex = vertices .map(new BuildTuplesFromVertices<>(vertexGroupingKeys, vertexAggregateFunctions)) .groupBy(getInternalVertexGroupingKeys()) diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java index b3e30b49dcfb..2b32b86332e5 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/functions/BuildTuplesFromElements.java @@ -105,6 +105,7 @@ public BuildTuplesFromElements(int tupleDataOffset, List> keys reuseTuple = Tuple.newInstance(tupleSize); // Fill first fields with default ID values. for (int i = 0; i < tupleDataOffset; i++) { + // The copy is needed to protect NULL_VALUE from changes. See #1466 reuseTuple.setField(GradoopId.NULL_VALUE.copy(), i); } } diff --git a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java index 57b401b012c1..4eff31feb5e8 100644 --- a/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java +++ b/gradoop-flink/src/main/java/org/gradoop/flink/model/impl/operators/keyedgrouping/labelspecific/WithAllKeysRetained.java @@ -22,7 +22,7 @@ import java.util.Objects; /** - * A filter function selecting elements with all values set to a default value. + * A filter function selecting elements where all keys vote to retain it. * * @param The type of the elements to filter. */ diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java index 3bfafc6f9b44..9ffad282a760 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java @@ -931,6 +931,7 @@ public void testLabelSpecificGroupingNoVerticesMatch() throws Exception { * - a vertex with a matching label, no properties: retain * - a vertex with a matching label, one matching property: retain * - two vertices with matching labels, two matching properties: group + * - two vertices with matching labels, two matching null properties: group * * @throws Exception if collecting result values fails */ @@ -946,6 +947,7 @@ public void testLabelSpecificGrouping() throws Exception { "(v6:A {})" + "(v7:A {a : 1})" + "(v8:A {a : 1, b : 2})" + + "(v9:A {a : NULL, b : NULL})" + "]"; FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); @@ -961,6 +963,7 @@ public void testLabelSpecificGrouping() throws Exception { "(v06:A {})" + "(v07:A {a : 1})" + "(v08:A {a : 1, b : 2, count: 1L})" + + "(v09:A {a : NULL, b : NULL, count: 1L})" + "]"); final LogicalGraph input = loader.getLogicalGraphByVariable("input"); @@ -971,6 +974,8 @@ public void testLabelSpecificGrouping() throws Exception { .useVertexLabel(false) .addVertexLabelGroup("A", "A", Arrays.asList("a", "b"), Collections.singletonList(new Count())) + .addVertexLabelGroup("C", "C", Collections.singletonList("a"), + Collections.singletonList(new Count())) .build() .execute(input); @@ -1082,4 +1087,30 @@ public void testLabelSpecificGroupingNoGlobalPropertyGrouping() throws Exception collectAndAssertTrue( output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); } + + /** + * Test if edges are properly updated when the source or target vertex was retained. + * + * @throws Exception when the execution in Flink fails. + */ + @Test + public void testEdgeUpdateWithRetainedSourceOrTarget() throws Exception { + FlinkAsciiGraphLoader loader = getLoaderFromString("input[" + + "(retained:Retained {otherprop: 1L})-[e:edge]->(notretained:NotRetained {prop: 1L, otherprop: 1L})" + + "-[:otherEdge {otherprop: 1L}]->(:NotRetained2 {prop:1L, otherprop: 2L})-[e2:edge2]->" + + "(retained2:RetainedTarget {otherprop: 2L})" + + "] expected [" + + "(retained)-->(resv {prop: 1L})-->(resv)-->(retained2)" + + "]"); + LogicalGraph input = loader.getLogicalGraphByVariable("input"); + LogicalGraph output = new Grouping.GroupingBuilder() + .setStrategy(getStrategy()) + .retainVerticesWithoutGroup() + .useVertexLabel(false) + .addVertexGroupingKey("prop") + .build() + .execute(input); + + collectAndAssertTrue(output.equalsByData(loader.getLogicalGraphByVariable("expected"))); + } } diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java index 4781d1c1c0a9..2e9bc0be807c 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/KeyedGroupingVertexRetentionTest.java @@ -25,15 +25,10 @@ import org.gradoop.flink.model.impl.operators.grouping.GroupingStrategy; import org.gradoop.flink.model.impl.operators.grouping.VertexRetentionTestBase; import org.gradoop.flink.util.FlinkAsciiGraphLoader; -import org.junit.Test; import java.util.Arrays; import java.util.Collections; -import static java.util.Collections.emptyList; -import static java.util.Collections.singletonList; -import static org.gradoop.flink.model.impl.operators.keyedgrouping.GroupingKeys.property; - /** * Tests if {@link KeyedGrouping#setRetainUngroupedVertices(boolean)} works as expected. */ @@ -177,6 +172,8 @@ public void testGroupByProperties() throws Exception { /** * Test label specific grouping with vertex retention enabled. + * The expected result is different here (v7), since vertices with only one of two properties set + * are also grouped together. * * @throws Exception when the execution in Flink fails. */ @@ -192,6 +189,7 @@ public void testLabelSpecificGrouping() throws Exception { "(v6:A {})" + "(v7:A {a : 1})" + "(v8:A {a : 1, b : 2})" + + "(v9:A {a : NULL, b : NULL})" + "]"; FlinkAsciiGraphLoader loader = getLoaderFromString(asciiInput); @@ -207,6 +205,7 @@ public void testLabelSpecificGrouping() throws Exception { "(v06:A)" + "(v07:A {a : 1, b : NULL, count : 1L})" + "(v08:A {a : 1, b : 2, count: 1L})" + + "(v09:A {a : NULL, b : NULL, count: 1L})" + "]"); final LogicalGraph input = loader.getLogicalGraphByVariable("input"); @@ -223,25 +222,4 @@ public void testLabelSpecificGrouping() throws Exception { collectAndAssertTrue( output.equalsByElementData(loader.getLogicalGraphByVariable("expected"))); } - - /** - * Test if edges are properly updates when the source or target vertex was retained. - * - * @throws Exception when the execution in Flink fails. - */ - @Test - public void testEdgeUpdateWithRetainedSourceOrTarget() throws Exception { - FlinkAsciiGraphLoader loader = getLoaderFromString("input[" + - "(retained:Retained {otherprop: 1L})-[e:edge]->(notretained:NotRetained {prop: 1L, otherprop: 1L})" + - "-[:otherEdge {otherprop: 1L}]->(:NotRetained2 {prop:1L, otherprop: 2L})-[e2:edge2]->" + - "(retained2:RetainedTarget {otherprop: 2L})" + - "] expected [" + - "(retained)-->(resv {prop: 1L})-->(resv)-->(retained2)" + - "]"); - LogicalGraph res = loader.getLogicalGraphByVariable("input").callForGraph( - new KeyedGrouping( - singletonList(property("prop")), emptyList(), emptyList(), emptyList()) - .setRetainUngroupedVertices(true)); - collectAndAssertTrue(res.equalsByData(loader.getLogicalGraphByVariable("expected"))); - } } diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java index c5ea07475b1b..8e12f6ce8f08 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/keyedgrouping/keys/LabelSpecificKeyFunctionTest.java @@ -253,7 +253,7 @@ public void testElementRetention() { testFunction.retainElement(vertex)); } // Changing the values on the vertex should only affect the result when the key function corresponding - // to the value is used. We check this by setting the property for label "c" to a non-default value. + // to the value is used. We check this by setting the property for label "c" to any value. vertex.setProperty("forC", PropertyValue.create(GradoopTestUtils.BIG_DECIMAL_VAL_7)); vertex.setLabel("a"); assertTrue(testFunction.retainElement(vertex)); From 05518f793fd21d8bd5b062229f13d330637c5c29 Mon Sep 17 00:00:00 2001 From: timo95 Date: Thu, 8 Apr 2021 14:16:59 +0200 Subject: [PATCH 7/7] Remove unused line --- .../model/impl/operators/grouping/VertexRetentionTestBase.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java index 9ffad282a760..4c6b762445e4 100644 --- a/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java +++ b/gradoop-flink/src/test/java/org/gradoop/flink/model/impl/operators/grouping/VertexRetentionTestBase.java @@ -974,8 +974,6 @@ public void testLabelSpecificGrouping() throws Exception { .useVertexLabel(false) .addVertexLabelGroup("A", "A", Arrays.asList("a", "b"), Collections.singletonList(new Count())) - .addVertexLabelGroup("C", "C", Collections.singletonList("a"), - Collections.singletonList(new Count())) .build() .execute(input);