@SuppressWarnings("unchecked") public static Object castValue(Object inputValue, FieldType input, FieldType output) { TypeName inputType = input.getTypeName(); TypeName outputType = output.getTypeName(); 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());
private static Object fieldToAvatica(Schema.FieldType type, Object beamValue) { switch (type.getTypeName()) { case DATETIME: if (Arrays.equals(type.getMetadata(), CalciteUtils.TIMESTAMP.getMetadata())) { return ((ReadableInstant) beamValue).getMillis(); } else if (Arrays.equals(type.getMetadata(), CalciteUtils.TIME.getMetadata())) { return (int) ((ReadableInstant) beamValue).getMillis(); } else if (Arrays.equals(type.getMetadata(), CalciteUtils.DATE.getMetadata())) { return (int) (((ReadableInstant) beamValue).getMillis() / MILLIS_PER_DAY); } else { throw new IllegalArgumentException( "Unknown DateTime type " + new String(type.getMetadata(), UTF_8)); return ((List<?>) beamValue) .stream() .map(elem -> fieldToAvatica(type.getCollectionElementType(), elem)) .collect(Collectors.toList()); case MAP: Collectors.toMap( entry -> entry.getKey(), entry -> fieldToAvatica(type.getCollectionElementType(), entry.getValue()))); case ROW: default: throw new IllegalStateException( String.format("Unreachable case for Beam typename %s", type.getTypeName()));
Schema.Field.of("f_datetime", FieldType.DATETIME).withNullable(true), Schema.Field.of("f_boolean", FieldType.BOOLEAN).withNullable(true), Schema.Field.of("f_array", FieldType.array(FieldType.DATETIME)).withNullable(true), Schema.Field.of("f_map", FieldType.map(FieldType.INT32, FieldType.DOUBLE)) .withNullable(true)) .collect(toSchema());
@Test public void testExecute_createTableWithPrefixMapField() throws Exception { InMemoryMetaStore metaStore = new InMemoryMetaStore(); metaStore.registerProvider(new TextTableProvider()); BeamSqlCli cli = new BeamSqlCli().metaStore(metaStore); cli.execute( "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name', \n" + "age int COMMENT 'age', \n" + "tags MAP<VARCHAR, VARCHAR>, \n" + "nestedMap MAP<INTEGER, MAP<VARCHAR, INTEGER>> \n" + ") \n" + "TYPE 'text' \n" + "COMMENT '' LOCATION '/home/admin/orders'"); Table table = metaStore.getTables().get("person"); assertNotNull(table); assertEquals( Stream.of( Field.of("id", INTEGER).withDescription("id").withNullable(true), Field.of("name", VARCHAR).withDescription("name").withNullable(true), Field.of("age", INTEGER).withDescription("age").withNullable(true), Field.of("tags", Schema.FieldType.map(VARCHAR, VARCHAR)).withNullable(true), Field.of( "nestedMap", Schema.FieldType.map(INTEGER, Schema.FieldType.map(VARCHAR, INTEGER))) .withNullable(true)) .collect(toSchema()), table.getSchema()); }
private static long estimatedSizeBytes(FieldType typeDescriptor, Object value) { switch (typeDescriptor.getTypeName()) { case ROW: return estimatedSizeBytes((Row) value); long listSizeBytes = 0; for (Object elem : list) { listSizeBytes += estimatedSizeBytes(typeDescriptor.getCollectionElementType(), elem); for (Map.Entry<Object, Object> elem : map.entrySet()) { mapSizeBytes += typeDescriptor.getMapKeyType().getTypeName().equals(TypeName.STRING) ? ((String) elem.getKey()).length() : ESTIMATED_FIELD_SIZES.get(typeDescriptor.getMapKeyType().getTypeName()); mapSizeBytes += estimatedSizeBytes(typeDescriptor.getMapValueType(), elem.getValue()); return ESTIMATED_FIELD_SIZES.get(typeDescriptor.getTypeName());
return null; if (TypeName.ROW.equals(type.getTypeName())) { return (T) fromRow((Row) value, (Class) fieldType, setterFactory); } else if (TypeName.ARRAY.equals(type.getTypeName())) { return (T) fromListValue( type.getCollectionElementType(), (List) value, elemenentType, setterFactory); } else if (TypeName.MAP.equals(type.getTypeName())) { return (T) fromMapValue( type.getMapKeyType(), type.getMapValueType(), (Map) value, keyType,
@Test public void testEquivalent() { final Schema expectedNested1 = Schema.builder().addStringField("yard1").addInt64Field("yard2").build(); final Schema expectedSchema1 = Schema.builder() .addStringField("field1") .addInt64Field("field2") .addRowField("field3", expectedNested1) .addArrayField("field4", FieldType.row(expectedNested1)) .addMapField("field5", FieldType.STRING, FieldType.row(expectedNested1)) .build(); final Schema expectedNested2 = Schema.builder().addInt64Field("yard2").addStringField("yard1").build(); final Schema expectedSchema2 = Schema.builder() .addMapField("field5", FieldType.STRING, FieldType.row(expectedNested2)) .addArrayField("field4", FieldType.row(expectedNested2)) .addRowField("field3", expectedNested2) .addInt64Field("field2") .addStringField("field1") .build(); assertNotEquals(expectedSchema1, expectedSchema2); assertTrue(expectedSchema1.equivalent(expectedSchema2)); }
private boolean equivalent(FieldType other) { if (!other.getTypeName().equals(getTypeName())) { return false; } switch (getTypeName()) { case ROW: if (!other.getRowSchema().equivalent(getRowSchema())) { return false; } break; case ARRAY: if (!other.getCollectionElementType().equivalent(getCollectionElementType())) { return false; } break; case MAP: if (!other.getMapKeyType().equivalent(getMapKeyType()) || !other.getMapValueType().equivalent(getMapValueType())) { return false; } break; default: return other.equals(this); } return true; }
static <T> T visit(SchemaZipFold<T> zipFold, Context context, FieldType left, FieldType right) { if (left.getTypeName() != right.getTypeName()) { return zipFold.accept(context, left, right); } Context newContext = context.withParent(left.getTypeName()); switch (left.getTypeName()) { case ARRAY: return zipFold.accumulate( zipFold.accept(context, left, right), visit( zipFold, newContext, left.getCollectionElementType(), right.getCollectionElementType())); case ROW: return visitRow(zipFold, newContext, left.getRowSchema(), right.getRowSchema()); case MAP: return zipFold.accumulate( zipFold.accept(context, left, right), visit( zipFold, newContext, left.getCollectionElementType(), right.getCollectionElementType())); default: return zipFold.accept(context, left, right); } }
@Test public void testSelectAll() { PCollection<Row> input = pCollectionOf2Elements(); Schema resultType = Schema.builder() .addInt32Field("f_int") .addNullableField( "f_map", Schema.FieldType.map(Schema.FieldType.STRING, Schema.FieldType.INT32)) .build(); PCollection<Row> result = input.apply( "sqlQuery", SqlTransform.query("SELECT f_int, f_intStringMap as f_map FROM PCOLLECTION")); PAssert.that(result) .containsInAnyOrder( Row.withSchema(resultType) .addValues(1, ImmutableMap.of("key11", 11, "key22", 22)) .build(), Row.withSchema(resultType) .addValues(2, ImmutableMap.of("key33", 33, "key44", 44, "key55", 55)) .build()); pipeline.run(); }
private static List<TableFieldSchema> toTableFieldSchema(Schema schema) { List<TableFieldSchema> fields = new ArrayList<>(schema.getFieldCount()); for (Field schemaField : schema.getFields()) { FieldType type = schemaField.getType(); TableFieldSchema field = new TableFieldSchema().setName(schemaField.getName()); if (schemaField.getDescription() != null && !"".equals(schemaField.getDescription())) { field.setDescription(schemaField.getDescription()); } if (!schemaField.getNullable()) { field.setMode(Mode.REQUIRED.toString()); } if (TypeName.ARRAY == type.getTypeName()) { type = type.getCollectionElementType(); field.setMode(Mode.REPEATED.toString()); } if (TypeName.ROW == type.getTypeName()) { Schema subType = type.getRowSchema(); field.setFields(toTableFieldSchema(subType)); } field.setType(toStandardSQLTypeName(type).toString()); fields.add(field); } return fields; }
private Object verify(Object value, FieldType type, String fieldName) { if (TypeName.ARRAY.equals(type.getTypeName())) { List<Object> arrayElements = verifyArray( value, type.getCollectionElementType(), type.getCollectionElementTypeNullable(), fieldName); return arrayElements; } else if (TypeName.MAP.equals(type.getTypeName())) { Map<Object, Object> mapElements = verifyMap( value, type.getMapKeyType().getTypeName(), type.getMapValueType(), type.getMapValueTypeNullable(), fieldName); return mapElements; } else if (TypeName.ROW.equals(type.getTypeName())) { return verifyRow(value, fieldName); } else { return verifyPrimitiveType(value, type.getTypeName(), fieldName); } }
/** Get a {@link FieldType} from a {@link TypeDescriptor}. */ public static FieldType fieldTypeForJavaType(TypeDescriptor typeDescriptor) { if (typeDescriptor.isArray() || typeDescriptor.isSubtypeOf(TypeDescriptor.of(Collection.class))) { return getArrayFieldType(typeDescriptor); } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Map.class))) { return getMapFieldType(typeDescriptor); } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Row.class))) { throw new IllegalArgumentException( "Cannot automatically determine a field type from a Row class" + " as we cannot determine the schema. You should set a field type explicitly."); } else { TypeName typeName = PRIMITIVE_MAPPING.inverse().get(typeDescriptor); if (typeName == null) { throw new RuntimeException("Couldn't find field type for " + typeDescriptor); } FieldType fieldType = FieldType.of(typeName); return fieldType; } }
@Test public void testCreateMapWithMapValueWithNull() { Map<Integer, Map<Integer, String>> data = new HashMap(); Map<Integer, String> innerData = new HashMap(); innerData.put(11, null); innerData.put(12, "value3"); data.put(1, ImmutableMap.of(1, "value1")); data.put(2, ImmutableMap.of(2, "value2")); data.put(3, null); data.put(4, innerData); Schema type = Stream.of( Schema.Field.of( "map", FieldType.map( FieldType.INT32, FieldType.map(FieldType.INT32, FieldType.STRING, true), true))) .collect(toSchema()); Row row = Row.withSchema(type).addValue(data).build(); assertEquals(data, row.getMap("map")); }
@SuppressWarnings({"TypeParameterUnusedInFormals", "unchecked"}) private <T> T getValue(FieldType type, Object fieldValue, @Nullable Integer cacheKey) { if (type.getTypeName().equals(TypeName.ROW)) { return (T) new RowWithGetters(type.getRowSchema(), fieldValueGetterFactory, fieldValue); } else if (type.getTypeName().equals(TypeName.ARRAY)) { return cacheKey != null ? (T) cachedLists.computeIfAbsent( cacheKey, i -> getListValue(type.getCollectionElementType(), fieldValue)) : (T) getListValue(type.getCollectionElementType(), fieldValue); } else if (type.getTypeName().equals(TypeName.MAP)) { Map map = (Map) fieldValue; return cacheKey != null ? (T) cachedMaps.computeIfAbsent( cacheKey, i -> getMapValue(type.getMapKeyType(), type.getMapValueType(), map)) : (T) getMapValue(type.getMapKeyType(), type.getMapValueType(), map); } else { return (T) fieldValue; } }
static Schema getOutputSchema(Schema inputSchema, FieldAccessDescriptor fieldAccessDescriptor) { if (fieldAccessDescriptor.allFields()) { return inputSchema; } Schema.Builder builder = new Schema.Builder(); for (int fieldId : fieldAccessDescriptor.fieldIdsAccessed()) { builder.addField(inputSchema.getField(fieldId)); } for (Map.Entry<Integer, FieldAccessDescriptor> nested : fieldAccessDescriptor.nestedFields().entrySet()) { Field field = inputSchema.getField(nested.getKey()); FieldAccessDescriptor nestedDescriptor = nested.getValue(); FieldType nestedType = FieldType.row(getOutputSchema(field.getType().getRowSchema(), nestedDescriptor)); if (field.getNullable()) { builder.addNullableField(field.getName(), nestedType); } else { builder.addField(field.getName(), nestedType); } } return builder.build(); }
private static Schema getUnnestedSchema( Schema schema, List<String> nameComponents, SerializableFunction<List<String>, String> fn) { Schema.Builder builder = Schema.builder(); for (Field field : schema.getFields()) { nameComponents.add(field.getName()); if (field.getType().getTypeName().isCompositeType()) { Schema nestedSchema = getUnnestedSchema(field.getType().getRowSchema(), nameComponents, fn); for (Field nestedField : nestedSchema.getFields()) { builder.addField(nestedField); } } else { String name = fn.apply(nameComponents); Field newField = field.toBuilder().setName(name).build(); builder.addField(newField); } nameComponents.remove(nameComponents.size() - 1); } return builder.build(); } /** Unnest a row. */
private static Object toBeamValue(FieldType fieldType, Object jsonBQValue) { if (jsonBQValue instanceof String && JSON_VALUE_PARSERS.containsKey(fieldType.getTypeName())) { return JSON_VALUE_PARSERS.get(fieldType.getTypeName()).apply((String) jsonBQValue); } if (jsonBQValue instanceof List) { return ((List<Object>) jsonBQValue) .stream() .map(v -> ((Map<String, Object>) v).get("v")) .map(v -> toBeamValue(fieldType.getCollectionElementType(), v)) .collect(toList()); } throw new UnsupportedOperationException( "Converting BigQuery type '" + jsonBQValue.getClass() + "' to '" + fieldType + "' is not supported"); } }