|
22 | 22 | import java.lang.reflect.Type; |
23 | 23 | import java.util.Date; |
24 | 24 | import java.util.Map; |
| 25 | +import java.util.WeakHashMap; |
25 | 26 | import java.util.stream.IntStream; |
26 | 27 | import org.apache.beam.sdk.schemas.Schema; |
27 | 28 | import org.apache.beam.sdk.schemas.Schema.FieldType; |
@@ -169,6 +170,11 @@ public static boolean isStringType(FieldType fieldType) { |
169 | 170 | FieldType.DATETIME, SqlTypeName.TIMESTAMP, |
170 | 171 | FieldType.STRING, SqlTypeName.VARCHAR); |
171 | 172 |
|
| 173 | + // Use a weak hash map to preserve logical type in output schema for a full |
| 174 | + // Beam FieldType->Calcite Type->Beam FieldType trip |
| 175 | + private static final Map<RelDataType, FieldType> LOGICAL_TYPE_REL_DATA_MAPPING = |
| 176 | + new WeakHashMap<>(); |
| 177 | + |
172 | 178 | /** Generate {@link Schema} from {@code RelDataType} which is used to create table. */ |
173 | 179 | public static Schema toSchema(RelDataType tableInfo) { |
174 | 180 | return tableInfo.getFieldList().stream().map(CalciteUtils::toField).collect(Schema.toSchema()); |
@@ -254,6 +260,9 @@ public static Schema.Field toField(String name, RelDataType calciteType) { |
254 | 260 | } |
255 | 261 |
|
256 | 262 | public static FieldType toFieldType(RelDataType calciteType) { |
| 263 | + if (LOGICAL_TYPE_REL_DATA_MAPPING.containsKey(calciteType)) { |
| 264 | + return LOGICAL_TYPE_REL_DATA_MAPPING.get(calciteType); |
| 265 | + } |
257 | 266 | switch (calciteType.getSqlTypeName()) { |
258 | 267 | case ARRAY: |
259 | 268 | case MULTISET: |
@@ -317,10 +326,27 @@ public static RelDataType toRelDataType(RelDataTypeFactory dataTypeFactory, Fiel |
317 | 326 | return toCalciteRowType(schema, dataTypeFactory); |
318 | 327 | case LOGICAL_TYPE: |
319 | 328 | Schema.LogicalType<?, ?> logicalType = fieldType.getLogicalType(); |
| 329 | + RelDataType relDataType; |
320 | 330 | if (logicalType instanceof PassThroughLogicalType) { |
321 | | - return toRelDataType(dataTypeFactory, logicalType.getBaseType()); |
| 331 | + relDataType = |
| 332 | + toRelDataType( |
| 333 | + dataTypeFactory, logicalType.getBaseType().withNullable(fieldType.getNullable())); |
| 334 | + } else { |
| 335 | + relDataType = dataTypeFactory.createSqlType(toSqlTypeName(fieldType)); |
322 | 336 | } |
323 | | - return dataTypeFactory.createSqlType(toSqlTypeName(fieldType)); |
| 337 | + // For backward-compatibility, exclude logical types registered in |
| 338 | + // CALCITE_TO_BEAM_TYPE_MAPPING, |
| 339 | + // e.g., primitive types, date time types, etc. |
| 340 | + SqlTypeName typeName = relDataType.getSqlTypeName(); |
| 341 | + if (typeName != null && !CALCITE_TO_BEAM_TYPE_MAPPING.containsKey(typeName)) { |
| 342 | + // register both nullable and non-nullable variants |
| 343 | + boolean flipNullable = !relDataType.isNullable(); |
| 344 | + LOGICAL_TYPE_REL_DATA_MAPPING.put(relDataType, fieldType); |
| 345 | + LOGICAL_TYPE_REL_DATA_MAPPING.put( |
| 346 | + dataTypeFactory.createTypeWithNullability(relDataType, flipNullable), |
| 347 | + fieldType.withNullable(flipNullable)); |
| 348 | + } |
| 349 | + return relDataType; |
324 | 350 | default: |
325 | 351 | return dataTypeFactory.createSqlType(toSqlTypeName(fieldType)); |
326 | 352 | } |
|
0 commit comments