return castRow((Row) inputValue, input.getRowSchema(), output.getRowSchema()); castValue(elem, input.getCollectionElementType(), output.getCollectionElementType())); castValue(entry.getKey(), input.getMapKeyType(), output.getMapKeyType()); Object outputValue = castValue(entry.getValue(), input.getMapValueType(), output.getMapValueType()); return castNumber((Number) inputValue, inputType, outputType); } else { throw new IllegalArgumentException("input should be array, map, numeric or row");
@Override public PCollection<Row> expand(PCollection<T> input) { Schema inputSchema = input.getSchema(); verifyCompatibility(inputSchema); return input .apply( ParDo.of( new DoFn<T, Row>() { // TODO: This should be the same as resolved so that Beam knows which fields // are being accessed. Currently Beam only supports wildcard descriptors. // Once BEAM-4457 is fixed, fix this. @FieldAccess("filterFields") final FieldAccessDescriptor fieldAccessDescriptor = FieldAccessDescriptor.withAllFields(); @ProcessElement public void process( @FieldAccess("filterFields") Row input, OutputReceiver<Row> r) { Row output = castRow(input, inputSchema, outputSchema()); r.output(output); } })) .setRowSchema(outputSchema()); }
@Override public List<CompatibilityError> accept(Context context, FieldType input, FieldType output) { TypeName inputType = input.getTypeName(); TypeName outputType = output.getTypeName(); boolean supertype = outputType.isSupertypeOf(inputType); if (isIntegral(inputType) && isDecimal(outputType)) { return Collections.emptyList(); } else if (!supertype) { return Collections.singletonList( CompatibilityError.create( context.path(), "Can't cast '" + inputType + "' to '" + outputType + "'")); } return Collections.emptyList(); } }
@ProcessElement public void process( @FieldAccess("filterFields") Row input, OutputReceiver<Row> r) { Row output = castRow(input, inputSchema, outputSchema()); r.output(output); } }))
public void verifyCompatibility(Schema inputSchema) { List<CompatibilityError> errors = validator().apply(inputSchema, outputSchema()); if (!errors.isEmpty()) { String reason = errors .stream() .map(x -> Joiner.on('.').join(x.path()) + ": " + x.message()) .collect(Collectors.joining("\n\t")); throw new IllegalArgumentException( "Cast isn't compatible using " + validator() + ":\n\t" + reason); } }
@Test(expected = IllegalArgumentException.class) @Category(NeedsRunner.class) public void testComplexCastFail() throws Exception { Schema inputSchema = pipeline.getSchemaRegistry().getSchema(All1.class); Schema outputSchema = pipeline.getSchemaRegistry().getSchema(All2.class); Cast.widening(outputSchema).verifyCompatibility(inputSchema); }
public void testCasting(TypeName inputType, TypeName outputType) { Object output = Cast.castValue(NUMERICS.get(inputType), FieldType.of(inputType), FieldType.of(outputType)); assertEquals(NUMERICS.get(outputType), output); }
@Test(expected = IllegalArgumentException.class) @Category(NeedsRunner.class) public void testTypeNarrowFail() throws Exception { // narrowing is the opposite of widening Schema inputSchema = pipeline.getSchemaRegistry().getSchema(TypeWiden2.class); Schema outputSchema = pipeline.getSchemaRegistry().getSchema(TypeWiden1.class); Cast.narrowing(outputSchema).verifyCompatibility(inputSchema); }
@Test(expected = IllegalArgumentException.class) public void testIgnoreNullFail() { Schema inputSchema = Schema.of(Schema.Field.nullable("f0", Schema.FieldType.INT32)); Schema outputSchema = Schema.of(Schema.Field.of("f0", Schema.FieldType.INT32)); Cast.castRow(Row.withSchema(inputSchema).addValue(null).build(), inputSchema, outputSchema); }
@Test @Category(NeedsRunner.class) public void testComplexCast() throws Exception { Schema outputSchema = pipeline.getSchemaRegistry().getSchema(All2.class); PCollection<All2> pojos = pipeline .apply(Create.of(new All1())) .apply(Cast.narrowing(outputSchema)) .apply(Convert.to(All2.class)); PAssert.that(pojos).containsInAnyOrder(new All2()); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testTypeWiden() throws Exception { Schema outputSchema = pipeline.getSchemaRegistry().getSchema(TypeWiden2.class); PCollection<TypeWiden2> pojos = pipeline .apply(Create.of(new TypeWiden1())) .apply(Cast.widening(outputSchema)) .apply(Convert.to(TypeWiden2.class)); PAssert.that(pojos).containsInAnyOrder(new TypeWiden2()); pipeline.run(); }
@Test(expected = IllegalArgumentException.class) @Category(NeedsRunner.class) public void testWeakedNullableFail() throws Exception { Schema inputSchema = pipeline.getSchemaRegistry().getSchema(Nullable1.class); Schema outputSchema = pipeline.getSchemaRegistry().getSchema(Nullable2.class); Cast.widening(outputSchema).verifyCompatibility(inputSchema); }
@Test public void testCastArray() { Object output = Cast.castValue( Arrays.asList((short) 1, (short) 2, (short) 3), Schema.FieldType.array(Schema.FieldType.INT16), Schema.FieldType.array(Schema.FieldType.INT32)); assertEquals(Arrays.asList(1, 2, 3), output); }
@Test @Category(NeedsRunner.class) public void testTypeNarrow() throws Exception { // narrowing is the opposite of widening Schema outputSchema = pipeline.getSchemaRegistry().getSchema(TypeWiden1.class); PCollection<TypeWiden1> pojos = pipeline .apply(Create.of(new TypeWiden2())) .apply(Cast.narrowing(outputSchema)) .apply(Convert.to(TypeWiden1.class)); PAssert.that(pojos).containsInAnyOrder(new TypeWiden1()); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testProjection() throws Exception { Schema outputSchema = pipeline.getSchemaRegistry().getSchema(Projection2.class); PCollection<Projection2> pojos = pipeline .apply(Create.of(new Projection1())) .apply(Cast.widening(outputSchema)) .apply(Convert.to(Projection2.class)); PAssert.that(pojos).containsInAnyOrder(new Projection2()); pipeline.run(); }
@Test(expected = IllegalArgumentException.class) @Category(NeedsRunner.class) public void testIgnoreNullableFail() throws Exception { // ignoring nullable is opposite of weakening Schema inputSchema = pipeline.getSchemaRegistry().getSchema(Nullable2.class); Schema outputSchema = pipeline.getSchemaRegistry().getSchema(Nullable1.class); Cast.widening(outputSchema).verifyCompatibility(inputSchema); }
public static Row castRow(Row input, Schema inputSchema, Schema outputSchema) { if (input == null) { return null; } Row.Builder output = Row.withSchema(outputSchema); for (int i = 0; i < outputSchema.getFieldCount(); i++) { Schema.Field outputField = outputSchema.getField(i); int fromFieldIdx = inputSchema.indexOf(outputField.getName()); Schema.Field inputField = inputSchema.getField(fromFieldIdx); Object inputValue = input.getValue(fromFieldIdx); Object outputValue = castValue(inputValue, inputField.getType(), outputField.getType()); output.addValue(outputValue); } return output.build(); }
@Override public List<CompatibilityError> accept(Context context, FieldType input, FieldType output) { TypeName inputType = input.getTypeName(); TypeName outputType = output.getTypeName(); boolean supertype = outputType.isSupertypeOf(inputType); boolean subtype = outputType.isSubtypeOf(inputType); if (isDecimal(inputType) && isIntegral(outputType)) { return Collections.emptyList(); } else if (!supertype && !subtype) { return Collections.singletonList( CompatibilityError.create( context.path(), "Can't cast '" + inputType + "' to '" + outputType + "'")); } return Collections.emptyList(); } }
@Test @Category(NeedsRunner.class) public void testWeakedNullable() throws Exception { Schema outputSchema = pipeline.getSchemaRegistry().getSchema(Nullable2.class); PCollection<Nullable2> pojos = pipeline .apply(Create.of(new Nullable1())) .apply(Cast.narrowing(outputSchema)) .apply(Convert.to(Nullable2.class)); PAssert.that(pojos).containsInAnyOrder(new Nullable2()); pipeline.run(); }
@Test public void testCastMap() { Object output = Cast.castValue( ImmutableMap.of((short) 1, 1, (short) 2, 2, (short) 3, 3), Schema.FieldType.map(Schema.FieldType.INT16, Schema.FieldType.INT32), Schema.FieldType.map(Schema.FieldType.INT32, Schema.FieldType.INT64)); assertEquals(ImmutableMap.of(1, 1L, 2, 2L, 3, 3L), output); }