Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support DECIMAL logical type in python SDK #23014

Merged
merged 5 commits into from
Oct 3, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,18 @@ examples:

---

coder:
urn: "beam:coder:row:v1"
# f_float: float32, f_decimal: logical(decimal)
payload: "\n\r\n\x07f_float\x1a\x02\x10\x05\n1\n\tf_decimal\x1a$:\"\n\x1cbeam:logical_type:decimal:v1\x1a\x02\x10\t\x12$800c44ae-a1b7-4def-bbf6-6217cca89ec4"
examples:
"\x02\x00\x00\x00\x00\x00\x01\x01\x00": {f_float: "0.0", f_decimal: "0.0"}
"\x02\x00?\x80\x00\x00\x01\x01\n": {f_float: "1.0", f_decimal: "1.0"}
"\x02\x00@I\x0eV\x04\x02z\xb7": {f_float: "3.1415", f_decimal: "3.1415"}
"\x02\x00\xc2\xc8>\xfa\x03\x03\xfex\xe5": {f_float: "-100.123", f_decimal: "-100.123"}

---

coder:
urn: "beam:coder:sharded_key:v1"
components: [{urn: "beam:coder:string_utf8:v1"}]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,14 @@ message LogicalTypes {
// corresponds to chronological order.
MILLIS_INSTANT = 2 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:logical_type:millis_instant:v1"];

// A URN for Decimal type
// - Representation type: BYTES
// - A decimal number with variable scale. Its BYTES
// representation consists of an integer (INT32) scale followed by a
// two's complement encoded big integer.
DECIMAL = 3 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:logical_type:decimal:v1"];
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.math.BigDecimal;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
Expand Down Expand Up @@ -425,6 +426,8 @@ private static Object parseField(Object value, Schema.FieldType fieldType) {
case DATETIME:
// convert shifted millis to epoch millis as in InstantCoder
return new Instant((Long) value + -9223372036854775808L);
case DECIMAL:
return new BigDecimal((String) value);
case BYTES:
// extract String as byte[]
return ((String) value).getBytes(StandardCharsets.ISO_8859_1);
Expand Down Expand Up @@ -468,7 +471,7 @@ private static Object parseField(Object value, Schema.FieldType fieldType) {
return fieldType
.getLogicalType()
.toInputType(parseField(value, fieldType.getLogicalType().getBaseType()));
default: // DECIMAL
default:
throw new IllegalArgumentException("Unsupported type name: " + fieldType.getTypeName());
}
}
Expand Down
1 change: 1 addition & 0 deletions sdks/go/test/regression/coders/fromyaml/fromyaml.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ var filteredCases = []struct{ filter, reason string }{
{"logical", "BEAM-9615: Support logical types"},
{"30ea5a25-dcd8-4cdb-abeb-5332d15ab4b9", "https://github.com/apache/beam/issues/21206: Support encoding position."},
{"80be749a-5700-4ede-89d8-dd9a4433a3f8", "https://github.com/apache/beam/issues/19817: Support millis_instant."},
{"800c44ae-a1b7-4def-bbf6-6217cca89ec4", "https://github.com/apache/beam/issues/19817: Support decimal."},
}

// Coder is a representation a serialized beam coder.
Expand Down
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;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add this in schema.proto and reference that here? Then we can document the format in schema.proto as well

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added "beam:logical_type:decimal:v1" into schema.proto

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,116 @@
/*
* 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.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.SchemaApi;
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> {
// TODO(https://github.com/apache/beam/issues/23373) promote this URN to schema.proto once logical
// types with arguments are fully supported and the implementation of this logical type can thus
// be considered standardized.
public static final String IDENTIFIER = "beam:logical_type:fixed_decimal:v1";
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since this is being used cross-language can we define the URN in schema.proto and document it there as well?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I considered to put it in schema.proto as beam:logical_type:decimal:v1 but the problem is that I do not know how to generate a test case of it in standard_coders.yaml yet because the logical types with argument is currently not fully supported. I would like to leave it now and make it into schema.proto until we are confident that this implementation is standardized(i.e. stable). Open to opinions for sure.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, that's fine with me.


// TODO(https://github.com/apache/beam/issues/23374) implement beam:logical_type:decimal:v1 as
// CoderLogicalType (once CoderLogicalType is implemented).
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is CoderLogicalType? I don't a reference to it in that issue

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah the link in TODO is not accurate. This is a suggestion I got from @reuvenlax. The reference was here: #7865 (comment). Should we create an issue for it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

entered #23374

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you!

/**
* 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;

private static final Schema SCHEMA;

static {
BASE_IDENTIFIER =
SchemaApi.LogicalTypes.Enum.DECIMAL
.getValueDescriptor()
.getOptions()
.getExtension(RunnerApi.beamUrn);
SCHEMA = Schema.builder().addInt32Field("precision").addInt32Field("scale").build();
}

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) {
return new FixedPrecisionNumeric(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) {
checkArgument(
row.getSchema().assignableTo(SCHEMA),
"Row has an incompatible schema to construct the logical type object: %s",
row.getSchema());

final Integer precision = row.getInt32("precision");
TheNeuralBit marked this conversation as resolved.
Show resolved Hide resolved
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(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) {
if (precision != -1) {
// check value not causing overflow when precision is fixed.
checkArgument(
base == null
|| (base.precision() <= precision && base.scale() <= scale)
|| 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;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The logic in this class is a little bit complicated, what do you think about adding a unit test just for this file?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This file is just moving org.apache.beam.sdk.io.jdbc.LogicalTypes.FixedPrecisionNumeric class to a separate file under schemas/logicaltypes. Yes will do add unit tests.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah sorry about that, thank you

}
}
Loading