Skip to content
Open
Show file tree
Hide file tree
Changes from 9 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": 16
"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": 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) =
"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
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.sql.SQLException;
import java.sql.Time;
import java.sql.Timestamp;
import java.time.LocalDate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Calendar;
Expand Down Expand Up @@ -293,6 +294,13 @@ static JdbcIO.PreparedStatementSetCaller getPreparedStatementSetCaller(
return (element, ps, i, fieldWithIndex) -> {
ps.setBigDecimal(i + 1, element.getDecimal(fieldWithIndex.getIndex()));
};
} else if (logicalTypeName.equals(
org.apache.beam.sdk.schemas.logicaltypes.Date.IDENTIFIER)) {
return (element, ps, i, fieldWithIndex) -> {
LocalDate value =
element.getLogicalTypeValue(fieldWithIndex.getIndex(), LocalDate.class);
ps.setDate(i + 1, value == null ? null : Date.valueOf(value));
};
} else if (logicalTypeName.equals("DATE")) {
return (element, ps, i, fieldWithIndex) -> {
ReadableDateTime value = element.getDateTime(fieldWithIndex.getIndex());
Expand Down
4 changes: 4 additions & 0 deletions sdks/python/apache_beam/io/jdbc.py
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,7 @@ class ReadFromJdbc(ExternalTransform):
Experimental; no backwards compatibility guarantees.
"""


URN = 'beam:transform:org.apache.beam:schemaio_jdbc_read:v1'

def __init__(
Expand Down Expand Up @@ -360,6 +361,9 @@ def __init__(
of the output PCollection elements. This bypasses automatic
schema inference during pipeline construction.
"""
# override new portable Date type with the current Jdbc type
# TODO: switch JdbcIO to return portable Date type
LogicalType.register_logical_type(JdbcDateType)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

If we need it again, we may want to put it into @contextlib.contextmanager

def enforce_millis_instant_for_timestamp():

In fact we can add Date to the same try block here

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Some jdbc tests fail if I move into that contextmanager. It seems like we need to register this type regardless if schema is passed or not:

if schema is not None:
with enforce_millis_instant_for_timestamp():

classpath = classpath or DEFAULT_JDBC_CLASSPATH

dataSchema = None
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
6 changes: 4 additions & 2 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 Down Expand Up @@ -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")
Decimal <-----> LogicalType(urn="beam:logical_type:fixed_decimal:v1")
Mapping <-----> MapType
Sequence <-----> ArrayType
Expand Down Expand Up @@ -1317,3 +1318,30 @@ def argument(self):
@classmethod
def _from_typing(cls, typ):
return cls()


@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)
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