Skip to content
Open
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
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 15
"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": 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": 1
}
5 changes: 5 additions & 0 deletions sdks/python/apache_beam/io/jdbc.py
Original file line number Diff line number Diff line change
Expand Up @@ -360,6 +360,11 @@ 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(https://github.com/apache/beam/issues/28359):
# switch JdbcIO to return portable Date type
LogicalType.register_logical_type(JdbcDateType)

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
23 changes: 16 additions & 7 deletions sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
# limitations under the License.
#

import datetime
import os
import unittest
import uuid
Expand All @@ -33,13 +34,13 @@
"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"
WAREHOUSE = "gs://temp-storage-for-end-to-end-tests"

def setUp(self):
self.test_pipeline = TestPipeline(is_integration_test=True)
self.args = self.test_pipeline.get_full_options_as_args()
self.args.extend([
'--experiments=enable_managed_transforms',
# '--experiments=enable_managed_transforms',
])

def _create_row(self, num: int):
Expand All @@ -49,16 +50,24 @@ 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):
biglake_catalog_props = {
'type': 'rest',
'uri': 'https://biglake.googleapis.com/iceberg/v1/restcatalog',
'warehouse': self.WAREHOUSE,
'header.x-goog-user-project': 'apache-beam-testing',
'rest.auth.type': 'google',
'io-impl': 'org.apache.iceberg.gcp.gcs.GCSFileIO',
'header.X-Iceberg-Access-Delegation': 'vended-credentials',
'rest-metrics-reporting-enabled': 'false'
}
iceberg_config = {
"table": "test_iceberg_write_read.test_" + uuid.uuid4().hex,
"catalog_name": "default",
"catalog_properties": {
"type": "hadoop",
"warehouse": self.WAREHOUSE,
}
"catalog_properties": biglake_catalog_props
}

rows = [self._create_row(i) for i in range(100)]
Expand Down
29 changes: 28 additions & 1 deletion 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 @@ -1004,6 +1005,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 Expand Up @@ -1244,7 +1272,6 @@ def argument(self):
# TODO: A temporary fix for missing jdbc logical types.
# See the discussion in https://github.com/apache/beam/issues/35738 for
# more detail.
@LogicalType._register_internal
class JdbcDateType(LogicalType[datetime.date, MillisInstant, str]):
"""
For internal use only; no backwards-compatibility guarantees.
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