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

[Coral-Schema] Add DATE type support in RelDataTypeToAvroType #484

Merged
merged 3 commits into from
Jan 11, 2024
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
@@ -1,5 +1,5 @@
/**
* Copyright 2019-2023 LinkedIn Corporation. All rights reserved.
* Copyright 2019-2024 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
Expand Down Expand Up @@ -90,6 +90,7 @@ private static Schema relDataTypeToAvroType(RelDataType relDataType, String reco
}

private static Schema basicSqlTypeToAvroType(BasicSqlType relDataType) {
Schema schema;
switch (relDataType.getSqlTypeName()) {
case BOOLEAN:
return Schema.create(Schema.Type.BOOLEAN);
Expand All @@ -110,9 +111,14 @@ private static Schema basicSqlTypeToAvroType(BasicSqlType relDataType) {
case NULL:
return Schema.create(Schema.Type.NULL);
case TIMESTAMP:
Schema schema = Schema.create(Schema.Type.LONG);
schema = Schema.create(Schema.Type.LONG);
schema.addProp("logicalType", "timestamp-millis");
return schema;
case DATE:
ljfgem marked this conversation as resolved.
Show resolved Hide resolved
// In Avro, "date" type is represented as {"type": "int", "logicalType": "date"}.
schema = Schema.create(Schema.Type.INT);
Copy link
Contributor

Choose a reason for hiding this comment

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

Since there's no Date type schema in avro https://github.com/apache/avro/blob/branch-1.10/lang/java/avro/src/main/java/org/apache/avro/Schema.java#L125, how did we choose 'int' here?

What is the default spark derived schema for 'Date' type field?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

so spark derived date type is cast-able to coral derived 'int' type?

Copy link
Collaborator Author

@ljfgem ljfgem Jan 10, 2024

Choose a reason for hiding this comment

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

No, Spark AvroDeserializer would return date type directly for this schema, and spark.table.printSchema also returns date type, Spark won't cast to int type.

schema.addProp("logicalType", "date");
return schema;
case DECIMAL:
JsonNodeFactory factory = JsonNodeFactory.instance;
Schema decimalSchema = Schema.create(Schema.Type.BYTES);
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/**
* Copyright 2020-2023 LinkedIn Corporation. All rights reserved.
* Copyright 2020-2024 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
Expand Down Expand Up @@ -78,4 +78,17 @@ public void testTimestampTypeField() {
Assert.assertEquals(actualAvroType.toString(true),
TestUtils.loadSchema("rel2avro-testTimestampTypeField-expected.avsc"));
}

@Test
public void testDateTypeField() {
String viewSql = "CREATE VIEW v AS SELECT * FROM basedatetypefield";

TestUtils.executeCreateViewQuery("default", "v", viewSql);
RelNode relNode = hiveToRelConverter.convertView("default", "v");
Schema actualAvroType =
RelDataTypeToAvroType.relDataTypeToAvroTypeNonNullable(relNode.getRowType(), "dateTypeField");

Assert.assertEquals(actualAvroType.toString(true),
TestUtils.loadSchema("rel2avro-testDateTypeField-expected.avsc"));
}
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/**
* Copyright 2019-2023 LinkedIn Corporation. All rights reserved.
* Copyright 2019-2024 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
Expand Down Expand Up @@ -96,6 +96,7 @@ private static void initializeTables() {
String baseNestedComplexSchema = loadSchema("base-nested-complex.avsc");
String baseNullTypeFieldSchema = loadSchema("base-null-type-field.avsc");
String baseTimestampTypeFieldSchema = loadSchema("base-timestamp-type-field.avsc");
String baseDateTypeFieldSchema = loadSchema("base-date-type-field.avsc");
String baseComplexUnionTypeSchema = loadSchema("base-complex-union-type.avsc");
String baseNestedUnionSchema = loadSchema("base-nested-union.avsc");
String baseComplexLowercase = loadSchema("base-complex-lowercase.avsc");
Expand All @@ -113,6 +114,7 @@ private static void initializeTables() {
executeCreateTableQuery("default", "basenullability", baseNullabilitySchema);
executeCreateTableQuery("default", "basenulltypefield", baseNullTypeFieldSchema);
executeCreateTableQuery("default", "basetimestamptypefield", baseTimestampTypeFieldSchema);
executeCreateTableQuery("default", "basedatetypefield", baseDateTypeFieldSchema);
executeCreateTableQuery("default", "basecomplexuniontype", baseComplexUnionTypeSchema);
executeCreateTableQuery("default", "basenestedunion", baseNestedUnionSchema);
executeCreateTableQuery("default", "basecomplexlowercase", baseComplexLowercase);
Expand Down
12 changes: 12 additions & 0 deletions coral-schema/src/test/resources/base-date-type-field.avsc
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
{
"type" : "record",
"name" : "basedatetypefield",
"namespace" : "coral.schema.avro.base.date.type.field",
"fields" : [ {
"name" : "Date_Field",
"type" : [ "null", {
"type" : "int",
"logicalType" : "date"
} ]
Comment on lines +7 to +10
Copy link
Contributor

Choose a reason for hiding this comment

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

is this the hive schema representation of 'Date' data type column? who injects the 'logicalType'? why is this not just

{
          "name" : "Date_Field",
          "type" : [ "null", "date" ]
        },

Copy link
Collaborator Author

@ljfgem ljfgem Jan 10, 2024

Choose a reason for hiding this comment

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

This is Avro schema representing date type, "type" : [ "null", "date" ] is not valid in Avro.

} ]
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
{
"type" : "record",
"name" : "dateTypeField",
"namespace" : "rel_avro",
"fields" : [ {
"name" : "date_field",
"type" : [ "null", {
"type" : "int",
"logicalType" : "date"
} ],
"default" : null
} ]
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/**
* Copyright 2017-2023 LinkedIn Corporation. All rights reserved.
* Copyright 2017-2024 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
Expand Down
Loading