You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2020/05/15 01:18:08 UTC
[flink] branch master updated: [FLINK-17629][json] Implement format
factory for JSON serialization and deserialization schema
This is an automated email from the ASF dual-hosted git repository.
jark 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 2f76042 [FLINK-17629][json] Implement format factory for JSON serialization and deserialization schema
2f76042 is described below
commit 2f76042f3cc80bf5468fe383b0c57887840efffb
Author: Danny Chan <yu...@gmail.com>
AuthorDate: Fri May 15 09:17:23 2020 +0800
[FLINK-17629][json] Implement format factory for JSON serialization and deserialization schema
This closes #12140
---
flink-formats/flink-json/pom.xml | 9 ++
.../flink/formats/json/JsonFormatFactory.java | 159 +++++++++++++++++++
.../json/JsonRowDataDeserializationSchema.java | 20 +++
.../json/JsonRowDataSerializationSchema.java | 22 +++
.../org.apache.flink.table.factories.Factory | 16 ++
.../flink/formats/json/JsonFormatFactoryTest.java | 175 +++++++++++++++++++++
6 files changed, 401 insertions(+)
diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml
index 19d5045..0d31525 100644
--- a/flink-formats/flink-json/pom.xml
+++ b/flink-formats/flink-json/pom.xml
@@ -100,6 +100,15 @@ under the License.
<artifactId>scala-compiler</artifactId>
<scope>test</scope>
</dependency>
+
+ <!-- JSON RowData format factory testing -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-core</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
</dependencies>
<profiles>
diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java
new file mode 100644
index 0000000..07e6d2d
--- /dev/null
+++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java
@@ -0,0 +1,159 @@
+/*
+ * 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.formats.json;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.ScanFormat;
+import org.apache.flink.table.connector.format.SinkFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Table format factory for providing configured instances of JSON to RowData
+ * {@link SerializationSchema} and {@link DeserializationSchema}.
+ */
+public class JsonFormatFactory implements
+ DeserializationFormatFactory,
+ SerializationFormatFactory {
+
+ public static final String IDENTIFIER = "json";
+
+ // ------------------------------------------------------------------------
+ // Options
+ // ------------------------------------------------------------------------
+
+ private static final ConfigOption<Boolean> FAIL_ON_MISSING_FIELD = ConfigOptions
+ .key("fail-on-missing-field")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default");
+
+ private static final ConfigOption<Boolean> IGNORE_PARSE_ERRORS = ConfigOptions
+ .key("ignore-parse-errors")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n"
+ + "fields are set to null in case of errors, false by default");
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public ScanFormat<DeserializationSchema<RowData>> createScanFormat(
+ DynamicTableFactory.Context context,
+ ReadableConfig formatOptions) {
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+ validateFormatOptions(formatOptions);
+
+ final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD);
+ final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
+
+ return new ScanFormat<DeserializationSchema<RowData>>() {
+ @Override
+ public DeserializationSchema<RowData> createScanFormat(
+ ScanTableSource.Context scanContext,
+ DataType producedDataType) {
+ final RowType rowType = (RowType) producedDataType.getLogicalType();
+ final TypeInformation<RowData> rowDataTypeInfo =
+ (TypeInformation<RowData>) scanContext.createTypeInformation(producedDataType);
+ return new JsonRowDataDeserializationSchema(
+ rowType,
+ rowDataTypeInfo,
+ failOnMissingField,
+ ignoreParseErrors);
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.insertOnly();
+ }
+ };
+ }
+
+ @Override
+ public SinkFormat<SerializationSchema<RowData>> createSinkFormat(
+ DynamicTableFactory.Context context,
+ ReadableConfig formatOptions) {
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+
+ return new SinkFormat<SerializationSchema<RowData>>() {
+ @Override
+ public SerializationSchema<RowData> createSinkFormat(
+ DynamicTableSink.Context context,
+ DataType consumedDataType) {
+ final RowType rowType = (RowType) consumedDataType.getLogicalType();
+ return new JsonRowDataSerializationSchema(rowType);
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.insertOnly();
+ }
+ };
+ }
+
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ Set<ConfigOption<?>> options = new HashSet<>();
+ options.add(FAIL_ON_MISSING_FIELD);
+ options.add(IGNORE_PARSE_ERRORS);
+ return options;
+ }
+
+ // ------------------------------------------------------------------------
+ // Validation
+ // ------------------------------------------------------------------------
+
+ private void validateFormatOptions(ReadableConfig tableOptions) {
+ boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD);
+ boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS);
+ if (ignoreParseErrors && failOnMissingField) {
+ throw new ValidationException(FAIL_ON_MISSING_FIELD.key()
+ + " and "
+ + IGNORE_PARSE_ERRORS.key()
+ + " shouldn't both be true.");
+ }
+ }
+}
diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
index 974b8dd..87dee7f 100644
--- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
+++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
@@ -56,6 +56,7 @@ import java.time.temporal.TemporalQueries;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
+import java.util.Objects;
import static java.lang.String.format;
import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE;
@@ -130,6 +131,25 @@ public class JsonRowDataDeserializationSchema implements DeserializationSchema<R
return resultTypeInfo;
}
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o;
+ return failOnMissingField == that.failOnMissingField &&
+ ignoreParseErrors == that.ignoreParseErrors &&
+ resultTypeInfo.equals(that.resultTypeInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo);
+ }
+
// -------------------------------------------------------------------------------------
// Runtime Converters
// -------------------------------------------------------------------------------------
diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java
index 7c17738..263e282 100644
--- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java
+++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java
@@ -42,6 +42,7 @@ import java.math.BigDecimal;
import java.time.LocalDate;
import java.time.LocalTime;
import java.util.Arrays;
+import java.util.Objects;
import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE;
import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT;
@@ -59,6 +60,9 @@ import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT;
public class JsonRowDataSerializationSchema implements SerializationSchema<RowData> {
private static final long serialVersionUID = 1L;
+ /** RowType to generate the runtime converter. */
+ private final RowType rowType;
+
/** The converter that converts internal data formats to JsonNode. */
private final SerializationRuntimeConverter runtimeConverter;
@@ -69,6 +73,7 @@ public class JsonRowDataSerializationSchema implements SerializationSchema<RowDa
private transient ObjectNode node;
public JsonRowDataSerializationSchema(RowType rowType) {
+ this.rowType = rowType;
this.runtimeConverter = createConverter(rowType);
}
@@ -87,6 +92,23 @@ public class JsonRowDataSerializationSchema implements SerializationSchema<RowDa
}
}
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ JsonRowDataSerializationSchema that = (JsonRowDataSerializationSchema) o;
+ return rowType.equals(that.rowType);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(rowType);
+ }
+
// --------------------------------------------------------------------------------
// Runtime Converters
// --------------------------------------------------------------------------------
diff --git a/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 0000000..3a243d0
--- /dev/null
+++ b/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.flink.formats.json.JsonFormatFactory
diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java
new file mode 100644
index 0000000..7638378
--- /dev/null
+++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java
@@ -0,0 +1,175 @@
+/*
+ * 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.formats.json;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TestDynamicTableFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+
+import static org.apache.flink.util.CoreMatchers.containsCause;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the {@link JsonFormatFactory}.
+ */
+public class JsonFormatFactoryTest extends TestLogger {
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ private static final TableSchema SCHEMA = TableSchema.builder()
+ .field("field1", DataTypes.BOOLEAN())
+ .field("field2", DataTypes.INT())
+ .build();
+
+ private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType();
+
+ @Test
+ public void testSeDeSchema() {
+ final Map<String, String> tableOptions = getAllOptions();
+
+ testSchemaSerializationSchema(tableOptions);
+
+ testSchemaDeserializationSchema(tableOptions);
+ }
+
+ @Test
+ public void testFailOnMissingField() {
+ final Map<String, String> tableOptions = getModifyOptions(
+ options -> options.put("json.fail-on-missing-field", "true"));
+
+ thrown.expect(ValidationException.class);
+ thrown.expect(containsCause(new ValidationException("fail-on-missing-field and ignore-parse-errors shouldn't both be true.")));
+ testSchemaDeserializationSchema(tableOptions);
+ }
+
+ @Test
+ public void testInvalidOptionForIgnoreParseErrors() {
+ final Map<String, String> tableOptions = getModifyOptions(
+ options -> options.put("json.ignore-parse-errors", "abc"));
+
+ thrown.expect(ValidationException.class);
+ thrown.expect(containsCause(new IllegalArgumentException("Unrecognized option for boolean: abc. Expected either true or false(case insensitive)")));
+ testSchemaDeserializationSchema(tableOptions);
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private void testSchemaDeserializationSchema(Map<String, String> options) {
+ final JsonRowDataDeserializationSchema expectedDeser =
+ new JsonRowDataDeserializationSchema(
+ ROW_TYPE,
+ new RowDataTypeInfo(ROW_TYPE),
+ false,
+ true);
+
+ final DynamicTableSource actualSource = createTableSource(options);
+ assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock;
+ TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock =
+ (TestDynamicTableFactory.DynamicTableSourceMock) actualSource;
+
+ DeserializationSchema<RowData> actualDeser = scanSourceMock.sourceValueFormat
+ .createScanFormat(
+ ScanRuntimeProviderContext.INSTANCE,
+ SCHEMA.toRowDataType());
+
+ assertEquals(expectedDeser, actualDeser);
+ }
+
+ private void testSchemaSerializationSchema(Map<String, String> options) {
+ final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema(ROW_TYPE);
+
+ final DynamicTableSink actualSink = createTableSink(options);
+ assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock;
+ TestDynamicTableFactory.DynamicTableSinkMock sinkMock =
+ (TestDynamicTableFactory.DynamicTableSinkMock) actualSink;
+
+ SerializationSchema<RowData> actualSer = sinkMock.sinkValueFormat
+ .createSinkFormat(
+ new SinkRuntimeProviderContext(false),
+ SCHEMA.toRowDataType());
+
+ assertEquals(expectedSer, actualSer);
+ }
+
+ /**
+ * Returns the full options modified by the given consumer {@code optionModifier}.
+ *
+ * @param optionModifier Consumer to modify the options
+ */
+ private Map<String, String> getModifyOptions(Consumer<Map<String, String>> optionModifier) {
+ Map<String, String> options = getAllOptions();
+ optionModifier.accept(options);
+ return options;
+ }
+
+ private Map<String, String> getAllOptions() {
+ final Map<String, String> options = new HashMap<>();
+ options.put("connector", TestDynamicTableFactory.IDENTIFIER);
+ options.put("target", "MyTarget");
+ options.put("buffer-size", "1000");
+
+ options.put("format", JsonFormatFactory.IDENTIFIER);
+ options.put("json.fail-on-missing-field", "false");
+ options.put("json.ignore-parse-errors", "true");
+ return options;
+ }
+
+ private static DynamicTableSource createTableSource(Map<String, String> options) {
+ return FactoryUtil.createTableSource(
+ null,
+ ObjectIdentifier.of("default", "default", "t1"),
+ new CatalogTableImpl(SCHEMA, options, "Mock scan table"),
+ new Configuration(),
+ JsonFormatFactoryTest.class.getClassLoader());
+ }
+
+ private static DynamicTableSink createTableSink(Map<String, String> options) {
+ return FactoryUtil.createTableSink(
+ null,
+ ObjectIdentifier.of("default", "default", "t1"),
+ new CatalogTableImpl(SCHEMA, options, "Mock sink table"),
+ new Configuration(),
+ JsonFormatFactoryTest.class.getClassLoader());
+ }
+}