You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by lz...@apache.org on 2022/07/26 11:53:55 UTC

[flink-table-store] branch release-0.2 updated: [FLINK-28686] Spark table with column comment cannot be read/write by Flink

This is an automated email from the ASF dual-hosted git repository.

lzljs3620320 pushed a commit to branch release-0.2
in repository https://gitbox.apache.org/repos/asf/flink-table-store.git


The following commit(s) were added to refs/heads/release-0.2 by this push:
     new 3c76a391 [FLINK-28686] Spark table with column comment cannot be read/write by Flink
3c76a391 is described below

commit 3c76a3916e55271eb08e10d03b4124fb8325d6b8
Author: Jane Chan <55...@users.noreply.github.com>
AuthorDate: Tue Jul 26 19:53:18 2022 +0800

    [FLINK-28686] Spark table with column comment cannot be read/write by Flink
    
    This closes #242
---
 .../store/connector/AbstractTableStoreFactory.java | 22 +++++++-
 .../connector/AbstractTableStoreFactoryTest.java   | 60 ++++++++++++++++++++++
 2 files changed, 80 insertions(+), 2 deletions(-)

diff --git a/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/AbstractTableStoreFactory.java b/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/AbstractTableStoreFactory.java
index 34c71100..566686e4 100644
--- a/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/AbstractTableStoreFactory.java
+++ b/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/AbstractTableStoreFactory.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.store.connector;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.RuntimeExecutionMode;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.Configuration;
@@ -145,9 +146,9 @@ public abstract class AbstractTableStoreFactory
         List<String> partitionKeys = updateSchema.partitionKeys();
         List<String> primaryKeys = updateSchema.primaryKeys();
 
-        // compare fields to ignore isNullable for row type
+        // compare fields to ignore the outside nullability and nested fields' comments
         Preconditions.checkArgument(
-                tableSchema.logicalRowType().getFields().equals(rowType.getFields()),
+                schemaEquals(tableSchema.logicalRowType(), rowType),
                 "Flink schema and store schema are not the same, "
                         + "store schema is %s, Flink schema is %s",
                 tableSchema.logicalRowType(),
@@ -169,4 +170,21 @@ public abstract class AbstractTableStoreFactory
 
         return table;
     }
+
+    @VisibleForTesting
+    static boolean schemaEquals(RowType rowType1, RowType rowType2) {
+        List<RowType.RowField> fieldList1 = rowType1.getFields();
+        List<RowType.RowField> fieldList2 = rowType2.getFields();
+        if (fieldList1.size() != fieldList2.size()) {
+            return false;
+        }
+        for (int i = 0; i < fieldList1.size(); i++) {
+            RowType.RowField f1 = fieldList1.get(i);
+            RowType.RowField f2 = fieldList2.get(i);
+            if (!f1.getName().equals(f2.getName()) || !f1.getType().equals(f2.getType())) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git a/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/AbstractTableStoreFactoryTest.java b/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/AbstractTableStoreFactoryTest.java
new file mode 100644
index 00000000..4f08ac89
--- /dev/null
+++ b/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/AbstractTableStoreFactoryTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.store.connector;
+
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for {@link AbstractTableStoreFactory}. */
+public class AbstractTableStoreFactoryTest {
+
+    @Test
+    public void testSchemaEquals() {
+        innerTest(RowType.of(false), RowType.of(true), true);
+        innerTest(RowType.of(false), RowType.of(false, new VarCharType()), false);
+        innerTest(
+                RowType.of(new LogicalType[] {new VarCharType()}, new String[] {"foo"}),
+                RowType.of(new VarCharType()),
+                false);
+        innerTest(
+                new RowType(
+                        true,
+                        Arrays.asList(
+                                new RowType.RowField("foo", new VarCharType(), "comment about foo"),
+                                new RowType.RowField("bar", new IntType()))),
+                new RowType(
+                        false,
+                        Arrays.asList(
+                                new RowType.RowField("foo", new VarCharType()),
+                                new RowType.RowField("bar", new IntType(), "comment about bar"))),
+                true);
+    }
+
+    private void innerTest(RowType r1, RowType r2, boolean expectEquals) {
+        assertThat(AbstractTableStoreFactory.schemaEquals(r1, r2)).isEqualTo(expectEquals);
+    }
+}