You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@paimon.apache.org by "tsreaper (via GitHub)" <gi...@apache.org> on 2023/03/20 02:31:43 UTC

[GitHub] [incubator-paimon] tsreaper opened a new pull request, #649: [FLINK-31432] Introduce a special StoreWriteOperator to deal with schema changes

tsreaper opened a new pull request, #649:
URL: https://github.com/apache/incubator-paimon/pull/649

   ### Purpose
   
   Currently `StoreWriteOperator` is not able to deal with schema changes. We need to introduce a special `StoreWriteOperator` to deal with schema changes.
   
   ### Tests
   
   * SchemaAwareStoreWriteOperatorTest
   
   ### API and Format 
   
   N/A
   
   ### Documentation
   
   N/A
   


-- 
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: issues-unsubscribe@paimon.apache.org

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


[GitHub] [incubator-paimon] JingsongLi commented on a diff in pull request #649: [FLINK-31432] Introduce a special StoreWriteOperator to deal with schema changes

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on code in PR #649:
URL: https://github.com/apache/incubator-paimon/pull/649#discussion_r1141678599


##########
paimon-core/src/main/java/org/apache/paimon/cdc/Record.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.paimon.cdc;
+
+import org.apache.paimon.types.RowKind;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Objects;
+
+/** A data change message from the CDC source. */
+public class Record implements Serializable {

Review Comment:
   Maybe `RichRecord`? You need something to indicate `Map<String, String> fields` with field names.



-- 
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: issues-unsubscribe@paimon.apache.org

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


[GitHub] [incubator-paimon] tsreaper merged pull request #649: [FLINK-31432] Introduce a special StoreWriteOperator to deal with schema changes

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper merged PR #649:
URL: https://github.com/apache/incubator-paimon/pull/649


-- 
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: issues-unsubscribe@paimon.apache.org

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


[GitHub] [incubator-paimon] tsreaper commented on a diff in pull request #649: [FLINK-31432] Introduce a special StoreWriteOperator to deal with schema changes

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on code in PR #649:
URL: https://github.com/apache/incubator-paimon/pull/649#discussion_r1141694042


##########
paimon-core/src/main/java/org/apache/paimon/cdc/Record.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.paimon.cdc;
+
+import org.apache.paimon.types.RowKind;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Objects;
+
+/** A data change message from the CDC source. */
+public class Record implements Serializable {

Review Comment:
   If we only need to indicate that `Map<String, String>` contains both field names and values, we don't have to change class names. Just add some comments in this class or on that field.



-- 
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: issues-unsubscribe@paimon.apache.org

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


[GitHub] [incubator-paimon] tsreaper commented on a diff in pull request #649: [FLINK-31432] Introduce a special StoreWriteOperator to deal with schema changes

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on code in PR #649:
URL: https://github.com/apache/incubator-paimon/pull/649#discussion_r1141731832


##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java:
##########
@@ -136,4 +135,16 @@ public void close() throws Exception {
             write.close();
         }
     }
+
+    @Override
+    public void replace(Function<String, TableWriteImpl<?>> newWriteProvider) throws Exception {
+        if (commitUser == null) {
+            return;

Review Comment:
   See line 64~69.



-- 
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: issues-unsubscribe@paimon.apache.org

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


[GitHub] [incubator-paimon] JingsongLi commented on a diff in pull request #649: [FLINK-31432] Introduce a special StoreWriteOperator to deal with schema changes

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on code in PR #649:
URL: https://github.com/apache/incubator-paimon/pull/649#discussion_r1141691230


##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java:
##########
@@ -136,4 +135,16 @@ public void close() throws Exception {
             write.close();
         }
     }
+
+    @Override
+    public void replace(Function<String, TableWriteImpl<?>> newWriteProvider) throws Exception {
+        if (commitUser == null) {
+            return;

Review Comment:
   Which case?



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java:
##########
@@ -49,6 +51,9 @@ interface StoreSinkWrite {
 
     void close() throws Exception;
 
+    void replace(Function<String, TableWriteImpl<?>> newWriteProvider) throws Exception;

Review Comment:
   Documentation?
   I cannot get it.



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/SchemaAwareStoreWriteOperator.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.cdc.Record;
+import org.apache.paimon.data.GenericRow;
+import org.apache.paimon.flink.sink.AbstractStoreWriteOperator;
+import org.apache.paimon.flink.sink.LogSinkFunction;
+import org.apache.paimon.flink.sink.StoreSinkWrite;
+import org.apache.paimon.options.ConfigOption;
+import org.apache.paimon.options.ConfigOptions;
+import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.sink.SinkRecord;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.utils.TypeUtils;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Map;
+
+/**
+ * An {@link AbstractStoreWriteOperator} which is aware of schema changes.
+ *
+ * <p>When the input {@link Record} contains a field name not in the current {@link TableSchema}, it
+ * periodically queries the latest schema, until the latest schema contains that field name.
+ */
+public class SchemaAwareStoreWriteOperator extends AbstractStoreWriteOperator<Record> {
+
+    static final ConfigOption<Duration> SCHEMA_UPDATE_WAIT_TIME =

Review Comment:
   retry sleep time?
   It's easy to misunderstand it as just waiting for 500 ms.



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/SchemaAwareStoreWriteOperator.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.cdc.Record;
+import org.apache.paimon.data.GenericRow;
+import org.apache.paimon.flink.sink.AbstractStoreWriteOperator;
+import org.apache.paimon.flink.sink.LogSinkFunction;
+import org.apache.paimon.flink.sink.StoreSinkWrite;
+import org.apache.paimon.options.ConfigOption;
+import org.apache.paimon.options.ConfigOptions;
+import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.sink.SinkRecord;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.utils.TypeUtils;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Map;
+
+/**
+ * An {@link AbstractStoreWriteOperator} which is aware of schema changes.
+ *
+ * <p>When the input {@link Record} contains a field name not in the current {@link TableSchema}, it
+ * periodically queries the latest schema, until the latest schema contains that field name.
+ */
+public class SchemaAwareStoreWriteOperator extends AbstractStoreWriteOperator<Record> {
+
+    static final ConfigOption<Duration> SCHEMA_UPDATE_WAIT_TIME =
+            ConfigOptions.key("cdc.schema-update-wait-time")
+                    .durationType()
+                    .defaultValue(Duration.ofMillis(500));
+
+    private final long schemaUpdateWaitMillis;
+
+    public SchemaAwareStoreWriteOperator(
+            FileStoreTable table,
+            @Nullable LogSinkFunction logSinkFunction,
+            StoreSinkWrite.Provider storeSinkWriteProvider) {
+        super(table, logSinkFunction, storeSinkWriteProvider);
+        schemaUpdateWaitMillis =
+                table.options().toConfiguration().get(SCHEMA_UPDATE_WAIT_TIME).toMillis();
+    }
+
+    @Override
+    protected SinkRecord processRecord(Record record) throws Exception {
+        Map<String, String> fields = record.fields();
+
+        if (!schemaMatched(fields)) {
+            while (true) {
+                table = table.copyWithLatestSchema();
+                if (schemaMatched(fields)) {
+                    break;
+                }
+                Thread.sleep(schemaUpdateWaitMillis);
+            }
+            write.replace(commitUser -> table.newWrite(commitUser));
+        }
+
+        TableSchema schema = table.schema();
+        GenericRow row = new GenericRow(schema.fields().size());
+        row.setRowKind(record.kind());
+        for (Map.Entry<String, String> field : fields.entrySet()) {
+            String key = field.getKey();
+            String value = field.getValue();
+            int idx = schema.fieldNames().indexOf(key);
+            DataType type = schema.fields().get(idx).type();
+            row.setField(idx, TypeUtils.castFromString(value, type));

Review Comment:
   `TypeUtils.castFromString` is limited, can not support complex types.
   Add a TODO here?



-- 
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: issues-unsubscribe@paimon.apache.org

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