You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2022/10/20 12:51:58 UTC

[GitHub] [inlong] thesumery opened a new pull request, #6247: [INLONG-6246][Sort] Import multiple sink option and schema update handle policy

thesumery opened a new pull request, #6247:
URL: https://github.com/apache/inlong/pull/6247

   [INLONG-6246][Sort] Import multiple sink option and schema update handle policy
   
   ### Prepare a Pull Request
   - Title Example: [INLONG-6246][Sort] Import multiple sink option and schema update handle policy
   - Fixes #6246
   
   ### Motivation
   
   *Import multiple sink option and schema update handle policy*
   
   ### Modifications
   
   *Import multiple sink option and schema update handle policy*
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] yunqingmoswu commented on a diff in pull request #6247: [INLONG-6246][Sort] Import multiple sink option and schema update handle policy

Posted by GitBox <gi...@apache.org>.
yunqingmoswu commented on code in PR #6247:
URL: https://github.com/apache/inlong/pull/6247#discussion_r1001295145


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/sink/TableChange.java:
##########
@@ -0,0 +1,195 @@
+/*
+ *  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.inlong.sort.base.sink;
+
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.Objects;
+
+public interface TableChange {

Review Comment:
   Add some comments?



##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/Constants.java:
##########
@@ -145,15 +146,15 @@ public final class Constants {
                     .withDescription("The option 'sink.multiple.enable' "
                             + "is used to determine whether to support multiple sink writing, default is 'false'.");
 
-    public static final ConfigOption<String> SINK_MULTIPLE_ADD_COLUMN_POLICY =
+    public static final ConfigOption<SchemaUpdateExceptionPolicy> SINK_MULTIPLE_ADD_COLUMN_POLICY =

Review Comment:
   Add some comments?



##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/sink/MultipleSinkOption.java:
##########
@@ -0,0 +1,130 @@
+/*
+ *  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.inlong.sort.base.sink;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+import static org.apache.inlong.sort.base.sink.SchemaUpdateExceptionPolicy.ALERT_WITH_IGNORE;
+import static org.apache.inlong.sort.base.sink.SchemaUpdateExceptionPolicy.LOG_WITH_IGNORE;
+import static org.apache.inlong.sort.base.sink.SchemaUpdateExceptionPolicy.TRY_IT_BEST;
+
+public class MultipleSinkOption implements Serializable {

Review Comment:
   Add some comments?



##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/Constants.java:
##########
@@ -145,15 +146,15 @@ public final class Constants {
                     .withDescription("The option 'sink.multiple.enable' "
                             + "is used to determine whether to support multiple sink writing, default is 'false'.");
 
-    public static final ConfigOption<String> SINK_MULTIPLE_ADD_COLUMN_POLICY =
+    public static final ConfigOption<SchemaUpdateExceptionPolicy> SINK_MULTIPLE_ADD_COLUMN_POLICY =
             ConfigOptions.key("sink.multiple.add-column.policy")
-                    .stringType()
-                    .noDefaultValue()
-                    .withDescription("");
+                    .enumType(SchemaUpdateExceptionPolicy.class)
+                    .defaultValue(SchemaUpdateExceptionPolicy.TRY_IT_BEST)
+                    .withDescription("The action to deal with column add.");
 
-    public static final ConfigOption<String> SINK_MULTIPLE_DEL_COLUMN_POLICY =
+    public static final ConfigOption<SchemaUpdateExceptionPolicy> SINK_MULTIPLE_DEL_COLUMN_POLICY =

Review Comment:
   Add some comments?



##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/sink/SchemaUpdateExceptionPolicy.java:
##########
@@ -0,0 +1,32 @@
+/*
+ *  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.inlong.sort.base.sink;
+
+public enum SchemaUpdateExceptionPolicy {

Review Comment:
   Add some comments?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] EMsnap commented on a diff in pull request #6247: [INLONG-6246][Sort] Import multiple sink option and schema update handle policy

Posted by GitBox <gi...@apache.org>.
EMsnap commented on code in PR #6247:
URL: https://github.com/apache/inlong/pull/6247#discussion_r1002981867


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/sink/TableChange.java:
##########
@@ -0,0 +1,198 @@
+/*
+ *  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.inlong.sort.base.sink;
+
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.Objects;
+
+/**
+ * TableChange represent requested changes to a table.
+ */
+public interface TableChange {
+    final class First implements ColumnPosition {
+        private static final First INSTANCE = new First();
+
+        private First() {
+
+        }
+
+        @Override
+        public String toString() {
+            return "FIRST";
+        }
+    }
+
+    final class After implements ColumnPosition {
+        private final String column;
+
+        private After(String column) {
+            assert column != null;
+            this.column = column;
+        }
+
+        public String column() {
+            return column;
+        }
+
+        @Override
+        public String toString() {
+            return "AFTER " + column;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            After after = (After) o;
+            return column.equals(after.column);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(column);
+        }
+    }
+
+    interface ColumnPosition {
+
+        static ColumnPosition first() {
+            return First.INSTANCE;
+        }
+
+        static ColumnPosition after(String column) {
+            return new After(column);
+        }
+    }
+
+    interface ColumnChange extends TableChange {
+        String[] fieldNames();
+    }
+
+    final class AddColumn implements ColumnChange {
+        private final String[] fieldNames;
+        private final LogicalType dataType;
+        private final boolean isNullable;
+        private final String comment;
+        private final ColumnPosition position;
+
+        public AddColumn(
+                String[] fieldNames,
+                LogicalType dataType,
+                boolean isNullable,
+                String comment,
+                ColumnPosition position) {
+            Preconditions.checkArgument(fieldNames.length > 0,
+                    "Invalid field name: at least one name is required");
+            this.fieldNames = fieldNames;
+            this.dataType = dataType;
+            this.isNullable = isNullable;
+            this.comment = comment;
+            this.position = position;
+        }
+
+        @Override
+        public String[] fieldNames() {
+            return fieldNames;
+        }
+
+        public LogicalType dataType() {
+            return dataType;
+        }
+
+        public boolean isNullable() {
+            return isNullable;
+        }
+
+        @Nullable
+        public String comment() {
+            return comment;
+        }
+
+        @Nullable
+        public ColumnPosition position() {
+            return position;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            AddColumn addColumn = (AddColumn) o;
+            return isNullable == addColumn.isNullable
+                    && Arrays.equals(fieldNames, addColumn.fieldNames)
+                    && dataType.equals(addColumn.dataType)
+                    && Objects.equals(comment, addColumn.comment)
+                    && Objects.equals(position, addColumn.position);
+        }
+
+        @Override
+        public int hashCode() {
+            int result = Objects.hash(dataType, isNullable, comment, position);
+            result = 31 * result + Arrays.hashCode(fieldNames);
+            return result;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("ADD COLUMNS `%s` %s %s %s %s",
+                    fieldNames[fieldNames.length - 1],
+                    dataType,
+                    isNullable ? "" : "NOT NULL",
+                    comment,
+                    position);
+        }
+    }
+
+    final class DeleteColumn implements ColumnChange {
+        @Override
+        public String[] fieldNames() {
+            return new String[0];
+        }
+    }
+
+    final class UnknownColumnChange implements ColumnChange {

Review Comment:
   please add some doc



##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/sink/SchemaUpdateExceptionPolicy.java:
##########
@@ -0,0 +1,42 @@
+/*
+ *  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.inlong.sort.base.sink;
+
+/**
+ * Multiple sink scenes will meet different table data.
+ * Maybe one table data have different schema, once it's schema mismatch with catalog schema, how to handle
+ * this table data. For example schema mismatch:
+ *
+ * <pre>
+ * data : {a : int, b : string, c : date}
+ * catalog : {a : string, b : timestamp}
+ * </pre>
+ */
+public enum SchemaUpdateExceptionPolicy {
+    TRY_IT_BEST("Try it best to handle schema update, if can not handle it, just ignore it."),

Review Comment:
   try it best -> try best



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] EMsnap merged pull request #6247: [INLONG-6246][Sort] Import multiple sink option and schema update handle policy

Posted by GitBox <gi...@apache.org>.
EMsnap merged PR #6247:
URL: https://github.com/apache/inlong/pull/6247


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org