/** * Adds, to the existing information, a field that is forwarded directly * from the source record(s) to the destination record(s). * * @param sourceField the position in the source record(s) * @param targetField the position in the destination record(s) */ public void addForwardedField(int sourceField, int targetField) { if(isTargetFieldPresent(targetField)) { throw new InvalidSemanticAnnotationException("Target field "+targetField+" was added twice."); } FieldSet targetFields = fieldMapping.get(sourceField); if (targetFields != null) { fieldMapping.put(sourceField, targetFields.addField(targetField)); } else { fieldMapping.put(sourceField, new FieldSet(targetField)); } }
if (targetField == null || targetField.size() == 0) { if(targetField.size() > 1) { LOG.warn("Found that a field is forwarded to more than one target field in " + "semantic forwarded field information. Will only use the field with the lowest index."); newOrdering.appendOrdering(targetField.toArray()[0], this.ordering.getType(i), this.ordering.getOrder(i)); if (targetField == null || targetField.size() == 0) { newPartitioningFields = null; break; } else { if(targetField.size() > 1) { LOG.warn("Found that a field is forwarded to more than one target field in " + "semantic forwarded field information. Will only use the field with the lowest index."); newPartitioningFields = newPartitioningFields.addField(targetField.toArray()[0]); Set<FieldSet> newUniqueFieldCombinations = new HashSet<FieldSet>(); for (FieldSet fieldCombo : this.uniqueFieldCombinations) { FieldSet newFieldCombo = new FieldSet(); for (Integer sourceField : fieldCombo) { FieldSet targetField = props.getForwardingTargetFields(input, sourceField); if (targetField == null || targetField.size() == 0) { newFieldCombo = null; break; } else {
public PartitionDescriptor(PartitionMethod pMethod, FieldSet pKeys, Ordering ordering, Partitioner<?> customPartitioner, DataDistribution distribution) { super(pKeys); Preconditions.checkArgument(pMethod != PartitionMethod.RANGE || pKeys.equals(new FieldSet(ordering.getFieldPositions())), "Partition keys must match the given ordering."); this.pMethod = pMethod; this.customPartitioner = customPartitioner; this.distribution = distribution; this.ordering = ordering; }
@Override public FieldSet getForwardingTargetFields(int input, int sourceField) { if(input != 0) { throw new IndexOutOfBoundsException(); } return new FieldSet(sourceField); }
@Test public void testImmutability() { FieldSet s1 = new FieldSet(); FieldSet s2 = new FieldSet(5); FieldSet s3 = new FieldSet(Integer.valueOf(7)); FieldSet s4 = new FieldSet(5, 4, 7, 6); s1.addFields(s2).addFields(s3); s2.addFields(s4); s4.addFields(s1); s1.addField(Integer.valueOf(14)); s2.addFields(78, 13, 66, 3); assertEquals(0, s1.size()); assertEquals(1, s2.size()); assertEquals(1, s3.size()); assertEquals(4, s4.size()); }
@Test public void testFieldSetAdds() { check(new FieldSet().addField(1).addField(2), 1, 2); check(FieldSet.EMPTY_SET.addField(3).addField(2), 3, 2); check(new FieldSet(13).addFields(new FieldSet(17, 31, 42)), 17, 13, 42, 31); check(new FieldSet(14).addFields(new FieldSet(17)), 17, 14); check(new FieldSet(3).addFields(2, 8, 5, 7), 3, 2, 8, 5, 7); check(new FieldSet().addFields(new FieldSet())); check(new FieldSet().addFields(new FieldSet(3, 4)), 4, 3); check(new FieldSet(5, 1).addFields(new FieldSet()), 5, 1); }
@Test public void testGetReadSet() { SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); sp.addReadFields(new FieldSet(0, 1)); assertEquals(2, sp.getReadFields(0).size()); assertTrue(sp.getReadFields(0).contains(0)); assertTrue(sp.getReadFields(0).contains(1)); sp.addReadFields(new FieldSet(3)); assertEquals(3, sp.getReadFields(0).size()); assertTrue(sp.getReadFields(0).contains(0)); assertTrue(sp.getReadFields(0).contains(1)); assertTrue(sp.getReadFields(0).contains(3)); }
private static void check(FieldSet set, int... elements) { if (elements == null) { assertEquals(0, set.size()); return; assertEquals(elements.length, set.size()); set.contains(i); int[] arr = set.toArray(); Arrays.sort(arr); assertTrue(Arrays.equals(arr, elements)); int[] fromIter = new int[set.size()]; Iterator<Integer> iter = set.iterator();
@Test public void testHashPartitioningPushedDown() { try { RequestedGlobalProperties req = new RequestedGlobalProperties(); req.setHashPartitioned(new FieldSet(3, 1)); RequestedGlobalProperties preserved = req.filterBySemanticProperties(getAllPreservingSemProps(), 0); assertEquals(PartitioningProperty.HASH_PARTITIONED, preserved.getPartitioning()); assertTrue(preserved.getPartitionedFields().isValidSubset(new FieldSet(1, 3))); RequestedGlobalProperties nonPreserved = req.filterBySemanticProperties(getNonePreservingSemProps(), 0); assertTrue(nonPreserved == null || nonPreserved.isTrivial()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
public static FieldList createOrderedFromSet(FieldSet set) { if (set instanceof FieldList) { return (FieldList) set; } else { final int[] cols = set.toArray(); Arrays.sort(cols); return new FieldList(cols); } }
private boolean isTargetFieldPresent(int targetField) { for(FieldSet targetFields : fieldMapping.values()) { if(targetFields.contains(targetField)) { return true; } } return false; }
@Override public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { if (gProps.getUniqueFieldCombination() != null && gProps.getUniqueFieldCombination().size() > 0 && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { gProps.setAnyPartitioning(gProps.getUniqueFieldCombination().iterator().next().toFieldList()); } gProps.clearUniqueFieldCombinations(); return gProps; }
@Override public int hashCode() { final int prime = 31; int result = 1; result = prime * result + (this.ordering == null ? 0 : this.ordering.hashCode()); result = prime * result + (this.groupedFields == null ? 0 : this.groupedFields.hashCode()); return result; }
public boolean isPartitionedOnFields(FieldSet fields) { if (this.partitioning.isPartitionedOnKey() && fields.isValidSubset(this.partitioningFields)) { return true; } else if (this.uniqueFieldCombinations != null) { for (FieldSet set : this.uniqueFieldCombinations) { if (fields.isValidSubset(set)) { return true; } } return false; } else { return false; } }
@Override public boolean equals(Object obj) { if (obj instanceof RequestedLocalProperties) { final RequestedLocalProperties other = (RequestedLocalProperties) obj; return (this.ordering == other.ordering || (this.ordering != null && this.ordering.equals(other.ordering))) && (this.groupedFields == other.groupedFields || (this.groupedFields != null && this.groupedFields.equals(other.groupedFields))); } else { return false; } }
@Test public void checkSinglePartitionedGroupedSource6() { ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); env.setParallelism(DEFAULT_PARALLELISM); DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType); data.getSplitDataProperties() .splitsPartitionedBy("f1.intField") .splitsGroupedBy("f0; f1.intField"); data.output(new DiscardingOutputFormat<Tuple3<Long, SomePojo, String>>()); Plan plan = env.createProgramPlan(); // submit the plan to the compiler OptimizedPlan oPlan = compileNoStats(plan); // check the optimized Plan SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); GlobalProperties gprops = sourceNode.getGlobalProperties(); LocalProperties lprops = sourceNode.getLocalProperties(); Assert.assertTrue((new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(2))); Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); Assert.assertTrue(new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(0,2))); Assert.assertTrue(lprops.getOrdering() == null); }
public FieldSet addField(Integer fieldID) { if (fieldID == null) { throw new IllegalArgumentException("Field ID must not be null."); } return new FieldSet(this, fieldID); }