You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2019/05/16 09:05:49 UTC
[flink] branch master updated: [FLINK-12254][table-common] Add a
class to data type converter
This is an automated email from the ASF dual-hosted git repository.
twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 37edea4 [FLINK-12254][table-common] Add a class to data type converter
37edea4 is described below
commit 37edea4df230268447d5960a3cdf99e77dc9629f
Author: Timo Walther <tw...@apache.org>
AuthorDate: Wed May 15 11:25:33 2019 +0200
[FLINK-12254][table-common] Add a class to data type converter
---
.../table/types/utils/ClassDataTypeConverter.java | 99 ++++++++++++++++++++
.../table/types/ClassDataTypeConverterTest.java | 103 +++++++++++++++++++++
2 files changed, 202 insertions(+)
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java
new file mode 100644
index 0000000..599c8b2
--- /dev/null
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java
@@ -0,0 +1,99 @@
+/*
+ * 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.flink.table.types.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Class-based data type extractor that supports extraction of clearly identifiable data types for
+ * input and output conversion.
+ */
+@Internal
+public final class ClassDataTypeConverter {
+
+ private static final Map<String, DataType> defaultDataTypes = new HashMap<>();
+ static {
+ // NOTE: this list explicitly excludes data types that need further parameters
+ // exclusions: DECIMAL, INTERVAL YEAR TO MONTH, MAP, MULTISET, ROW, NULL, ANY
+ addDefaultDataType(String.class, DataTypes.STRING());
+ addDefaultDataType(Boolean.class, DataTypes.BOOLEAN());
+ addDefaultDataType(boolean.class, DataTypes.BOOLEAN());
+ addDefaultDataType(Byte.class, DataTypes.TINYINT());
+ addDefaultDataType(byte.class, DataTypes.TINYINT());
+ addDefaultDataType(Short.class, DataTypes.SMALLINT());
+ addDefaultDataType(short.class, DataTypes.SMALLINT());
+ addDefaultDataType(Integer.class, DataTypes.INT());
+ addDefaultDataType(int.class, DataTypes.INT());
+ addDefaultDataType(Long.class, DataTypes.BIGINT());
+ addDefaultDataType(long.class, DataTypes.BIGINT());
+ addDefaultDataType(Float.class, DataTypes.FLOAT());
+ addDefaultDataType(float.class, DataTypes.FLOAT());
+ addDefaultDataType(Double.class, DataTypes.DOUBLE());
+ addDefaultDataType(double.class, DataTypes.DOUBLE());
+ addDefaultDataType(java.sql.Date.class, DataTypes.DATE());
+ addDefaultDataType(java.time.LocalDate.class, DataTypes.DATE());
+ addDefaultDataType(java.sql.Time.class, DataTypes.TIME(9));
+ addDefaultDataType(java.time.LocalTime.class, DataTypes.TIME(9));
+ addDefaultDataType(java.sql.Timestamp.class, DataTypes.TIMESTAMP(9));
+ addDefaultDataType(java.time.LocalDateTime.class, DataTypes.TIMESTAMP(9));
+ addDefaultDataType(java.time.OffsetDateTime.class, DataTypes.TIMESTAMP_WITH_TIME_ZONE(9));
+ addDefaultDataType(java.time.Instant.class, DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(9));
+ addDefaultDataType(java.time.Duration.class, DataTypes.INTERVAL(DataTypes.SECOND(9)));
+ }
+
+ private static void addDefaultDataType(Class<?> clazz, DataType rootType) {
+ final DataType dataType;
+ if (clazz.isPrimitive()) {
+ dataType = rootType.notNull();
+ } else {
+ dataType = rootType.nullable();
+ }
+ defaultDataTypes.put(clazz.getName(), dataType.bridgedTo(clazz));
+ }
+
+ /**
+ * Returns the clearly identifiable data type if possible. For example, {@link Long} can be
+ * expressed as {@link DataTypes#BIGINT()}. However, for example, {@link Row} cannot be extracted
+ * as information about the fields is missing. Or {@link BigDecimal} needs to be mapped from a
+ * variable precision/scale to constant ones.
+ */
+ public static Optional<DataType> extractDataType(Class<?> clazz) {
+ // byte arrays have higher priority than regular arrays
+ if (clazz.equals(byte[].class)) {
+ return Optional.of(DataTypes.BYTES().nullable().bridgedTo(byte[].class));
+ }
+ if (clazz.isArray()) {
+ return extractDataType(clazz.getComponentType())
+ .map(element -> DataTypes.ARRAY(element).nullable().bridgedTo(clazz));
+ }
+ return Optional.ofNullable(defaultDataTypes.get(clazz.getName()));
+ }
+
+ private ClassDataTypeConverter() {
+ // no instantiation
+ }
+}
diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java
new file mode 100644
index 0000000..d889dec
--- /dev/null
+++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.utils.ClassDataTypeConverter;
+import org.apache.flink.types.Row;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link ClassDataTypeConverter}.
+ */
+@RunWith(Parameterized.class)
+public class ClassDataTypeConverterTest {
+
+ @Parameterized.Parameters(name = "[{index}] class: {0} type: {1}")
+ public static List<Object[]> testData() {
+ return Arrays.asList(
+ new Object[][]{
+
+ {long.class, DataTypes.BIGINT().notNull().bridgedTo(long.class)},
+
+ {byte[].class, DataTypes.BYTES().nullable().bridgedTo(byte[].class)},
+
+ {Long.class, DataTypes.BIGINT().nullable().bridgedTo(Long.class)},
+
+ {BigDecimal.class, null},
+
+ {
+ byte[][].class,
+ DataTypes.ARRAY(DataTypes.BYTES().nullable().bridgedTo(byte[].class))
+ .nullable()
+ .bridgedTo(byte[][].class)
+ },
+
+ {
+ Byte[].class,
+ DataTypes.ARRAY(DataTypes.TINYINT().nullable().bridgedTo(Byte.class))
+ .nullable()
+ .bridgedTo(Byte[].class)
+ },
+
+ {
+ Byte[][].class,
+ DataTypes.ARRAY(
+ DataTypes.ARRAY(DataTypes.TINYINT().nullable().bridgedTo(Byte.class))
+ .nullable()
+ .bridgedTo(Byte[].class))
+ .nullable()
+ .bridgedTo(Byte[][].class)
+ },
+
+ {
+ Integer[].class,
+ DataTypes.ARRAY(DataTypes.INT().nullable().bridgedTo(Integer.class))
+ .nullable()
+ .bridgedTo(Integer[].class)
+ },
+
+ {Row.class, null}
+ }
+ );
+ }
+
+ @Parameterized.Parameter
+ public Class<?> clazz;
+
+ @Parameterized.Parameter(1)
+ public @Nullable DataType dataType;
+
+ @Test
+ public void testClassToDataTypeConversion() {
+ assertEquals(Optional.ofNullable(dataType), ClassDataTypeConverter.extractDataType(clazz));
+ }
+}