Skip to content

Commit

Permalink
Support DECIMAL logical type in python sdk
Browse files Browse the repository at this point in the history
* migrate FixedPrecisionNumeric jdbclogicaltype to portable
  logical type

* Support DECIMAL type in python sdk xlang jdbc transform

* Support standard logical type with argument in java sdk

* proto support logical type's argument type in java sdk.
  Support of logical type's argument value is still pending

* Implement BigIntegerCoder and DecimalCoder in python sdk
  • Loading branch information
Abacn committed Sep 13, 2022
1 parent e2e7265 commit 5158b36
Show file tree
Hide file tree
Showing 13 changed files with 449 additions and 113 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.io.IOException;
import java.io.PipedInputStream;
import java.io.PipedOutputStream;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
Expand All @@ -43,6 +44,7 @@
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.Schema.LogicalType;
import org.apache.beam.sdk.schemas.Schema.TypeName;
import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant;
import org.apache.beam.sdk.schemas.logicaltypes.PythonCallable;
import org.apache.beam.sdk.schemas.logicaltypes.SchemaLogicalType;
Expand All @@ -65,7 +67,7 @@
})
public class SchemaTranslation {

private static final String URN_BEAM_LOGICAL_DECIMAL = "beam:logical_type:decimal:v1";
private static final String URN_BEAM_LOGICAL_DECIMAL = FixedPrecisionNumeric.BASE_IDENTIFIER;
private static final String URN_BEAM_LOGICAL_JAVASDK = "beam:logical_type:javasdk:v1";
private static final String URN_BEAM_LOGICAL_MILLIS_INSTANT =
SchemaApi.LogicalTypes.Enum.MILLIS_INSTANT
Expand All @@ -79,6 +81,7 @@ public class SchemaTranslation {
private static final ImmutableMap<String, Class<? extends LogicalType<?, ?>>>
STANDARD_LOGICAL_TYPES =
ImmutableMap.<String, Class<? extends LogicalType<?, ?>>>builder()
.put(FixedPrecisionNumeric.IDENTIFIER, FixedPrecisionNumeric.class)
.put(MicrosInstant.IDENTIFIER, MicrosInstant.class)
.put(SchemaLogicalType.IDENTIFIER, SchemaLogicalType.class)
.put(PythonCallable.IDENTIFIER, PythonCallable.class)
Expand Down Expand Up @@ -147,17 +150,10 @@ private static SchemaApi.FieldType fieldTypeToProto(
case LOGICAL_TYPE:
LogicalType logicalType = fieldType.getLogicalType();
SchemaApi.LogicalType.Builder logicalTypeBuilder;
if (STANDARD_LOGICAL_TYPES.containsKey(logicalType.getIdentifier())) {
Preconditions.checkArgument(
logicalType.getArgumentType() == null,
"Logical type '%s' cannot be used as a logical type, it has a non-null argument type.",
logicalType.getIdentifier());
logicalTypeBuilder =
SchemaApi.LogicalType.newBuilder()
.setRepresentation(
fieldTypeToProto(logicalType.getBaseType(), serializeLogicalType))
.setUrn(logicalType.getIdentifier());
} else if (logicalType instanceof UnknownLogicalType) {
String identifier = logicalType.getIdentifier();
boolean isStandard = STANDARD_LOGICAL_TYPES.containsKey(identifier);

if (!isStandard && logicalType instanceof UnknownLogicalType) {
logicalTypeBuilder =
SchemaApi.LogicalType.newBuilder()
.setUrn(logicalType.getIdentifier())
Expand All @@ -173,14 +169,16 @@ private static SchemaApi.FieldType fieldTypeToProto(
fieldValueToProto(logicalType.getArgumentType(), logicalType.getArgument()));
}
} else {
// TODO(https://github.com/apache/beam/issues/19715): "javasdk" types should only
// be a last resort. Types defined in Beam should have their own URN, and there
// should be a mechanism for users to register their own types by URN.
String urn =
identifier.startsWith("beam:logical_type:") ? identifier : URN_BEAM_LOGICAL_JAVASDK;
logicalTypeBuilder =
SchemaApi.LogicalType.newBuilder()
.setRepresentation(
fieldTypeToProto(logicalType.getBaseType(), serializeLogicalType))
// TODO(https://github.com/apache/beam/issues/19715): "javasdk" types should only
// be a last resort. Types defined in Beam should have their own URN, and there
// should be a mechanism for users to register their own types by URN.
.setUrn(URN_BEAM_LOGICAL_JAVASDK);
.setUrn(urn);
if (logicalType.getArgumentType() != null) {
logicalTypeBuilder =
logicalTypeBuilder
Expand All @@ -190,7 +188,8 @@ private static SchemaApi.FieldType fieldTypeToProto(
fieldValueToProto(
logicalType.getArgumentType(), logicalType.getArgument()));
}
if (serializeLogicalType) {
// No need to embed serialized bytes to payload for standard (known) logical type
if (!isStandard && serializeLogicalType) {
logicalTypeBuilder =
logicalTypeBuilder.setPayload(
ByteString.copyFrom(SerializableUtils.serializeToByteArray(logicalType)));
Expand All @@ -208,6 +207,8 @@ private static SchemaApi.FieldType fieldTypeToProto(
.build());
break;
case DECIMAL:
// DECIMAL without precision specified. Used as the representation type of
// FixedPrecisionNumeric logical type.
builder.setLogicalType(
SchemaApi.LogicalType.newBuilder()
.setUrn(URN_BEAM_LOGICAL_DECIMAL)
Expand Down Expand Up @@ -337,28 +338,74 @@ private static FieldType fieldTypeFromProtoWithoutNullable(SchemaApi.FieldType p
fieldTypeFromProto(protoFieldType.getMapType().getKeyType()),
fieldTypeFromProto(protoFieldType.getMapType().getValueType()));
case LOGICAL_TYPE:
String urn = protoFieldType.getLogicalType().getUrn();
SchemaApi.LogicalType logicalType = protoFieldType.getLogicalType();
String urn = logicalType.getUrn();
Class<? extends LogicalType<?, ?>> logicalTypeClass = STANDARD_LOGICAL_TYPES.get(urn);
if (logicalTypeClass != null) {
try {
return FieldType.logicalType(logicalTypeClass.getConstructor().newInstance());
} catch (NoSuchMethodException e) {
throw new RuntimeException(
String.format(
"Standard logical type '%s' does not have a zero-argument constructor.", urn),
e);
} catch (IllegalAccessException e) {
throw new RuntimeException(
String.format(
"Standard logical type '%s' has a zero-argument constructor, but it is not accessible.",
urn),
e);
} catch (ReflectiveOperationException e) {
throw new RuntimeException(
String.format(
"Error instantiating logical type '%s' with zero-argument constructor.", urn),
e);
boolean hasArgument = logicalType.hasArgument();
if (hasArgument) {
// Logical type with argument. Construct from compatible of() method with single
// argument type is either a primitive, List, Map, or Row.
FieldType fieldType = fieldTypeFromProto(logicalType.getArgumentType());
Object fieldValue =
Objects.requireNonNull(fieldValueFromProto(fieldType, logicalType.getArgument()));
Class clazz = fieldValue.getClass();
if (fieldValue instanceof List) {
// argument is ArrayValue or iterableValue
clazz = List.class;
}
if (fieldValue instanceof Map) {
// argument is Map
clazz = Map.class;
} else if (fieldValue instanceof Row) {
// argument is Row
clazz = Row.class;
}
String objectName = clazz.getName();
try {
return FieldType.logicalType(
logicalTypeClass.cast(
logicalTypeClass.getMethod("of", clazz).invoke(null, fieldValue)));
} catch (NoSuchMethodException e) {
throw new RuntimeException(
String.format(
"Standard logical type '%s' does not have a static of('%s') method.",
urn, objectName),
e);
} catch (IllegalAccessException e) {
throw new RuntimeException(
String.format(
"Standard logical type '%s' has an of('%s') method, but it is not accessible.",
urn, objectName),
e);
} catch (InvocationTargetException e) {
throw new RuntimeException(
String.format(
"Error instantiating logical type '%s' with of('%s') method.",
urn, objectName),
e);
}
} else {
// Logical type without argument. Construct from constructor without parameter
try {
return FieldType.logicalType(logicalTypeClass.getConstructor().newInstance());
} catch (NoSuchMethodException e) {
throw new RuntimeException(
String.format(
"Standard logical type '%s' does not have a zero-argument constructor.", urn),
e);
} catch (IllegalAccessException e) {
throw new RuntimeException(
String.format(
"Standard logical type '%s' has a zero-argument constructor, but it is not accessible.",
urn),
e);
} catch (ReflectiveOperationException e) {
throw new RuntimeException(
String.format(
"Error instantiating logical type '%s' with zero-argument constructor.", urn),
e);
}
}
}
// Special-case for DATETIME and DECIMAL which are logical types in portable representation,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.apache.beam.sdk.schemas.logicaltypes;

import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;

import java.math.BigDecimal;
import java.math.MathContext;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.values.Row;

/** Fixed precision numeric types used to represent jdbc NUMERIC and DECIMAL types. */
public class FixedPrecisionNumeric extends PassThroughLogicalType<BigDecimal> {
public static final String IDENTIFIER = "beam:logical_type:fixed_decimal:v1";

// TODO(https://github.com/apache/beam/issues/19817) implement beam:logical_type:decimal:v1 as
// CoderLogicalType (once CoderLogicalType is implemented).
/**
* Identifier of the unspecified precision numeric type. It corresponds to Java SDK's {@link
* FieldType#DECIMAL}. It is the underlying representation type of FixedPrecisionNumeric logical
* type in order to be compatible with existing Java field types.
*/
public static final String BASE_IDENTIFIER = "beam:logical_type:decimal:v1";

private final int precision;
private final int scale;

/**
* Create a FixedPrecisionNumeric instance with specified precision and scale. ``precision=-1``
* indicates unspecified precision.
*/
public static FixedPrecisionNumeric of(int precision, int scale) {
Schema schema = Schema.builder().addInt32Field("precision").addInt32Field("scale").build();
return new FixedPrecisionNumeric(schema, precision, scale);
}

/** Create a FixedPrecisionNumeric instance with specified scale and unspecified precision. */
public static FixedPrecisionNumeric of(int scale) {
return of(-1, scale);
}

/** Create a FixedPrecisionNumeric instance with specified argument row. */
public static FixedPrecisionNumeric of(Row row) {
final Integer precision = row.getInt32("precision");
final Integer scale = row.getInt32("scale");
checkArgument(
precision != null && scale != null,
"precision and scale cannot be null for FixedPrecisionNumeric arguments.");
// firstNonNull is used to cast precision and scale to @NonNull input
return of(firstNonNull(precision, -1), firstNonNull(scale, 0));
}

private FixedPrecisionNumeric(Schema schema, int precision, int scale) {
super(
IDENTIFIER,
FieldType.row(schema),
Row.withSchema(schema).addValues(precision, scale).build(),
FieldType.DECIMAL);
this.precision = precision;
this.scale = scale;
}

@Override
public BigDecimal toInputType(BigDecimal base) {
checkArgument(
base == null
|| (base.precision() <= precision && base.scale() <= scale)
// for cases when received values can be safely coerced to the schema
|| base.round(new MathContext(precision)).compareTo(base) == 0,
"Expected BigDecimal base to be null or have precision <= %s (was %s), scale <= %s (was %s)",
precision,
(base == null) ? null : base.precision(),
scale,
(base == null) ? null : base.scale());
return base;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.beam.sdk.io.jdbc.JdbcIO.ReadWithPartitions;
import org.apache.beam.sdk.io.jdbc.JdbcIO.RowMapper;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.KV;
Expand Down Expand Up @@ -191,6 +192,8 @@ static JdbcIO.PreparedStatementSetCaller getPreparedStatementSetCaller(

String logicalTypeName = fieldType.getLogicalType().getIdentifier();

// Special case of Timestamp and Numeric which are logical types in Portable framework
// but has their own fieldType in Java.
if (logicalTypeName.equals(MicrosInstant.IDENTIFIER)) {
// Process timestamp of MicrosInstant kind, which should only be passed from other type
// systems such as SQL and other Beam SDKs.
Expand All @@ -200,6 +203,10 @@ static JdbcIO.PreparedStatementSetCaller getPreparedStatementSetCaller(
element.getLogicalTypeValue(fieldWithIndex.getIndex(), java.time.Instant.class);
ps.setTimestamp(i + 1, value == null ? null : new Timestamp(value.toEpochMilli()));
};
} else if (logicalTypeName.equals(FixedPrecisionNumeric.IDENTIFIER)) {
return (element, ps, i, fieldWithIndex) -> {
ps.setBigDecimal(i + 1, element.getDecimal(fieldWithIndex.getIndex()));
};
}

JDBCType jdbcType = JDBCType.valueOf(logicalTypeName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;

import java.math.BigDecimal;
import java.math.MathContext;
import java.sql.JDBCType;
import java.time.Instant;
import java.util.Arrays;
Expand All @@ -30,9 +28,9 @@
import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
import org.apache.beam.sdk.schemas.logicaltypes.UuidLogicalType;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.checkerframework.checker.nullness.qual.Nullable;

Expand Down Expand Up @@ -100,8 +98,7 @@ static Schema.FieldType variableLengthBytes(JDBCType jdbcType, int length) {

@VisibleForTesting
static Schema.FieldType numeric(int precision, int scale) {
return Schema.FieldType.logicalType(
FixedPrecisionNumeric.of(JDBCType.NUMERIC.getName(), precision, scale));
return Schema.FieldType.logicalType(FixedPrecisionNumeric.of(precision, scale));
}

/** Base class for JDBC logical types. */
Expand Down Expand Up @@ -251,41 +248,4 @@ public byte[] toInputType(byte[] base) {
return base;
}
}

/** Fixed precision numeric types such as NUMERIC. */
static final class FixedPrecisionNumeric extends JdbcLogicalType<BigDecimal> {
private final int precision;
private final int scale;

static FixedPrecisionNumeric of(String identifier, int precision, int scale) {
Schema schema = Schema.builder().addInt32Field("precision").addInt32Field("scale").build();
return new FixedPrecisionNumeric(schema, identifier, precision, scale);
}

private FixedPrecisionNumeric(
Schema argumentSchema, String identifier, int precision, int scale) {
super(
identifier,
FieldType.row(argumentSchema),
Schema.FieldType.DECIMAL,
Row.withSchema(argumentSchema).addValues(precision, scale).build());
this.precision = precision;
this.scale = scale;
}

@Override
public BigDecimal toInputType(BigDecimal base) {
checkArgument(
base == null
|| (base.precision() <= precision && base.scale() <= scale)
// for cases when received values can be safely coerced to the schema
|| base.round(new MathContext(precision)).compareTo(base) == 0,
"Expected BigDecimal base to be null or have precision <= %s (was %s), scale <= %s (was %s)",
precision,
(base == null) ? null : base.precision(),
scale,
(base == null) ? null : base.scale());
return base;
}
}
}
Loading

0 comments on commit 5158b36

Please sign in to comment.