Skip to content
Open
Show file tree
Hide file tree
Changes from 2 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
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 0
"modification": 1
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 2
"modification": 3
}
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,13 @@ message LogicalTypes {
// A variable-length string with its maximum length as the argument.
VAR_CHAR = 7 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:logical_type:var_char:v1"];

// A URN for Date type
// - Representation type: INT64
// - A date without a timezone, represented by the number of days
// since the epoch.
DATE = 8 [(org.apache.beam.model.pipeline.v1.beam_urn) =
Copy link
Contributor

Choose a reason for hiding this comment

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

Consider adding TIME as well?

"beam:logical_type:date:v1"];
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,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.Date;
import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
import org.apache.beam.sdk.schemas.logicaltypes.FixedString;
Expand Down Expand Up @@ -113,6 +114,7 @@ private static String getLogicalTypeUrn(String identifier) {
.put(VariableBytes.IDENTIFIER, VariableBytes.class)
.put(FixedString.IDENTIFIER, FixedString.class)
.put(VariableString.IDENTIFIER, VariableString.class)
.put(Date.IDENTIFIER, Date.class)
.build();

public static SchemaApi.Schema schemaToProto(Schema schema, boolean serializeLogicalType) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,10 @@
package org.apache.beam.sdk.schemas.logicaltypes;

import java.time.LocalDate;
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.checkerframework.checker.nullness.qual.Nullable;

/**
* A date without a time-zone.
Expand All @@ -30,23 +33,20 @@
* incrementing count of days where day 0 is 1970-01-01 (ISO).
*/
public class Date implements Schema.LogicalType<LocalDate, Long> {
public static final String IDENTIFIER = "beam:logical_type:date:v1";
public static final String IDENTIFIER =
SchemaApi.LogicalTypes.Enum.DATE
.getValueDescriptor()
.getOptions()
.getExtension(RunnerApi.beamUrn);

@Override
public String getIdentifier() {
return IDENTIFIER;
}

// unused
@Override
public Schema.FieldType getArgumentType() {
return Schema.FieldType.STRING;
}

// unused
@Override
public String getArgument() {
return "";
public Schema.@Nullable FieldType getArgumentType() {
return null;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@

import java.math.BigDecimal;
import java.nio.charset.StandardCharsets;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.util.ArrayList;
import java.util.HashMap;
Expand Down Expand Up @@ -142,6 +143,7 @@ public static Iterable<Schema> data() {
Field.of("decimal", FieldType.DECIMAL), Field.of("datetime", FieldType.DATETIME)))
.add(Schema.of(Field.of("fixed_bytes", FieldType.logicalType(FixedBytes.of(24)))))
.add(Schema.of(Field.of("micros_instant", FieldType.logicalType(new MicrosInstant()))))
.add(Schema.of(Field.of("date", FieldType.logicalType(SqlTypes.DATE))))
.add(Schema.of(Field.of("python_callable", FieldType.logicalType(new PythonCallable()))))
.add(
Schema.of(
Expand Down Expand Up @@ -388,6 +390,7 @@ public static Iterable<Row> data() {
.add(simpleRow(FieldType.DECIMAL, BigDecimal.valueOf(100000)))
.add(simpleRow(FieldType.logicalType(new PortableNullArgLogicalType()), "str"))
.add(simpleRow(FieldType.logicalType(new DateTime()), LocalDateTime.of(2000, 1, 3, 3, 1)))
.add(simpleRow(FieldType.logicalType(SqlTypes.DATE), LocalDate.of(2000, 1, 3)))
.add(simpleNullRow(FieldType.STRING))
.add(simpleNullRow(FieldType.INT32))
.add(simpleNullRow(FieldType.map(FieldType.STRING, FieldType.INT32)))
Expand Down
1 change: 1 addition & 0 deletions sdks/python/apache_beam/portability/common_urns.py
Original file line number Diff line number Diff line change
Expand Up @@ -92,3 +92,4 @@
var_bytes = LogicalTypes.Enum.VAR_BYTES
fixed_char = LogicalTypes.Enum.FIXED_CHAR
var_char = LogicalTypes.Enum.VAR_CHAR
date = LogicalTypes.Enum.DATE
14 changes: 8 additions & 6 deletions sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#

import datetime
import os
import unittest
import uuid
Expand All @@ -28,10 +28,10 @@


@pytest.mark.uses_io_java_expansion_service
@unittest.skipUnless(
os.environ.get('EXPANSION_JARS'),
"EXPANSION_JARS environment var is not provided, "
"indicating that jars have not been built")
# @unittest.skipUnless(
# os.environ.get('EXPANSION_JARS'),
# "EXPANSION_JARS environment var is not provided, "
# "indicating that jars have not been built")
class ManagedIcebergIT(unittest.TestCase):
WAREHOUSE = "gs://temp-storage-for-end-to-end-tests/xlang-python-using-java"

Expand All @@ -49,7 +49,8 @@ def _create_row(self, num: int):
bytes_=bytes(num),
bool_=(num % 2 == 0),
float_=(num + float(num) / 100),
arr_=[num, num, num])
arr_=[num, num, num],
date_=datetime.date.today() - datetime.timedelta(days=num))

def test_write_read_pipeline(self):
iceberg_config = {
Expand All @@ -58,6 +59,7 @@ def test_write_read_pipeline(self):
"catalog_properties": {
"type": "hadoop",
"warehouse": self.WAREHOUSE,
"io-impl": "org.apache.iceberg.gcp.gcs.GCSFileIO"
}
}

Expand Down
28 changes: 28 additions & 0 deletions sdks/python/apache_beam/typehints/schemas.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
bytes <-----> BYTES
ByteString ------> BYTES
Timestamp <-----> LogicalType(urn="beam:logical_type:micros_instant:v1")
datetime.date <---> LogicalType(urn="beam:logical_type:date:v1")
Copy link
Contributor

Choose a reason for hiding this comment

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

Most Java SDK's IO uses joda data and time, and are serialized as joda timestamp. Then JDBCDateType and JDBCTimeType were created to accomadate them.

WIth the introducation of "ture" portable data/time logical types, we need to make sure it doesn't break existing Python JdbcIO. We can run https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml against this PR to test

Decimal <-----> LogicalType(urn="beam:logical_type:fixed_decimal:v1")
Mapping <-----> MapType
Sequence <-----> ArrayType
Expand Down Expand Up @@ -991,6 +992,33 @@ def to_language_type(self, value):
return Timestamp(seconds=int(value.seconds), micros=int(value.micros))


# @LogicalType._register_internal
# class Date(NoArgumentLogicalType[datetime.date, np.int64]):
# """Date logical type that handles ``datetime.date``, days since epoch."""
# EPOCH = datetime.date(1970, 1, 1)
#
# @classmethod
# def urn(cls):
# return common_urns.date.urn
#
# @classmethod
# def representation_type(cls):
# # type: () -> type
# return np.int64
#
# @classmethod
# def language_type(cls):
# return datetime.date
#
# def to_representation_type(self, value):
# # type: (datetime.date) -> np.int64
# return (value - self.EPOCH).days
#
# def to_language_type(self, value):
# # type: (np.int64) -> datetime.date
# return self.EPOCH + datetime.timedelta(days=value)


@LogicalType._register_internal
class PythonCallable(NoArgumentLogicalType[PythonCallableWithSource, str]):
"""A logical type for PythonCallableSource objects."""
Expand Down
2 changes: 2 additions & 0 deletions sdks/python/apache_beam/typehints/schemas_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
# pytype: skip-file

import dataclasses
import datetime
import itertools
import pickle
import unittest
Expand Down Expand Up @@ -105,6 +106,7 @@ class ComplexSchema(NamedTuple):
optional_array: Optional[Sequence[np.float32]]
array_optional: Sequence[Optional[bool]]
timestamp: Timestamp
date: datetime.date


def get_test_beam_fieldtype_protos():
Expand Down
Loading