You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "yuxiqian (via GitHub)" <gi...@apache.org> on 2024/04/10 08:31:28 UTC

[PR] [FLINK-35702][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

yuxiqian opened a new pull request, #3215:
URL: https://github.com/apache/flink-cdc/pull/3215

   According to [Doris documentation](https://doris.apache.org/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN/), altering column types dynamically is supported (via `ALTER TABLE ... MODIFY COLUMN` statement) when lossless conversion is available. However, now Doris pipeline connector has no support for any `AlterColumnTypeEvent`s, and raises RuntimeException all the time.
   
   This PR adds `alterColumn` method which is not provided by Doris' `SchemaChangeManager` to apply `AlterColumnTypeEvent` to Doris sink. Only lossless conversions that were accepted by Doris will be allowed, and a `RuntimeException` will be raised for any incompatible narrowing casts.


-- 
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@flink.apache.org

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


Re: [PR] [FLINK-35072][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

Posted by "yuxiqian (via GitHub)" <gi...@apache.org>.
yuxiqian commented on code in PR #3215:
URL: https://github.com/apache/flink-cdc/pull/3215#discussion_r1565211096


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisSchemaChangeManager.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.cdc.connectors.doris.sink;
+
+import org.apache.doris.flink.catalog.doris.FieldSchema;
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+import org.apache.doris.flink.sink.schema.SchemaChangeManager;
+
+import java.io.IOException;
+
+/** Provides schema change operations based on {@link SchemaChangeManager}. */
+public class DorisSchemaChangeManager extends SchemaChangeManager {
+    public DorisSchemaChangeManager(DorisOptions dorisOptions) {
+        super(dorisOptions);
+    }
+
+    private static final String MODIFY_COLUMN_DDL = "ALTER TABLE %s MODIFY COLUMN %s %s";
+
+    public boolean alterColumn(
+            String database, String table, String columnName, String newColumnType)
+            throws IOException, IllegalArgumentException {
+        String tableIdentifier = String.format("%s.%s", database, table);
+        FieldSchema alterFieldSchema = new FieldSchema(columnName, newColumnType, "");
+
+        String alterColumnDDL =
+                String.format(
+                        MODIFY_COLUMN_DDL,
+                        tableIdentifier,
+                        columnName,
+                        alterFieldSchema.getTypeString());
+
+        try {
+            return this.schemaChange(
+                    database, table, buildRequestParam(true, columnName), alterColumnDDL);
+        } catch (RuntimeException ex) {
+            if (ex.getMessage().contains("Nothing is changed. please check your alter stmt.")) {

Review Comment:
   @lvyanquan Sure, but this error message comes from [Doris server side](https://github.com/apache/doris/blob/debb83f35d37cac9089b2e0282e2d555ad9c1d62/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java#L1508), which is wrapped as a generic `RuntimeException` on client-side, and could only be identified by checking error message. Any suggestions about this?



-- 
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@flink.apache.org

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


Re: [PR] [FLINK-35072][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

Posted by "yuxiqian (via GitHub)" <gi...@apache.org>.
yuxiqian commented on code in PR #3215:
URL: https://github.com/apache/flink-cdc/pull/3215#discussion_r1565211096


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisSchemaChangeManager.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.cdc.connectors.doris.sink;
+
+import org.apache.doris.flink.catalog.doris.FieldSchema;
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+import org.apache.doris.flink.sink.schema.SchemaChangeManager;
+
+import java.io.IOException;
+
+/** Provides schema change operations based on {@link SchemaChangeManager}. */
+public class DorisSchemaChangeManager extends SchemaChangeManager {
+    public DorisSchemaChangeManager(DorisOptions dorisOptions) {
+        super(dorisOptions);
+    }
+
+    private static final String MODIFY_COLUMN_DDL = "ALTER TABLE %s MODIFY COLUMN %s %s";
+
+    public boolean alterColumn(
+            String database, String table, String columnName, String newColumnType)
+            throws IOException, IllegalArgumentException {
+        String tableIdentifier = String.format("%s.%s", database, table);
+        FieldSchema alterFieldSchema = new FieldSchema(columnName, newColumnType, "");
+
+        String alterColumnDDL =
+                String.format(
+                        MODIFY_COLUMN_DDL,
+                        tableIdentifier,
+                        columnName,
+                        alterFieldSchema.getTypeString());
+
+        try {
+            return this.schemaChange(
+                    database, table, buildRequestParam(true, columnName), alterColumnDDL);
+        } catch (RuntimeException ex) {
+            if (ex.getMessage().contains("Nothing is changed. please check your alter stmt.")) {

Review Comment:
   @lvyanquan Sure, but this error message comes from [Doris server side](https://github.com/apache/doris/blob/debb83f35d37cac9089b2e0282e2d555ad9c1d62/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java#L1508), which is wrapped as a generic `DorisSchemaChangeException` on client-side, and could only be identified by checking error message. Any suggestions about this?



-- 
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@flink.apache.org

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


Re: [PR] [FLINK-35072][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

Posted by "lvyanquan (via GitHub)" <gi...@apache.org>.
lvyanquan commented on code in PR #3215:
URL: https://github.com/apache/flink-cdc/pull/3215#discussion_r1565217554


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisSchemaChangeManager.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.cdc.connectors.doris.sink;
+
+import org.apache.doris.flink.catalog.doris.FieldSchema;
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+import org.apache.doris.flink.sink.schema.SchemaChangeManager;
+
+import java.io.IOException;
+
+/** Provides schema change operations based on {@link SchemaChangeManager}. */
+public class DorisSchemaChangeManager extends SchemaChangeManager {
+    public DorisSchemaChangeManager(DorisOptions dorisOptions) {
+        super(dorisOptions);
+    }
+
+    private static final String MODIFY_COLUMN_DDL = "ALTER TABLE %s MODIFY COLUMN %s %s";
+
+    public boolean alterColumn(
+            String database, String table, String columnName, String newColumnType)
+            throws IOException, IllegalArgumentException {
+        String tableIdentifier = String.format("%s.%s", database, table);
+        FieldSchema alterFieldSchema = new FieldSchema(columnName, newColumnType, "");
+
+        String alterColumnDDL =
+                String.format(
+                        MODIFY_COLUMN_DDL,
+                        tableIdentifier,
+                        columnName,
+                        alterFieldSchema.getTypeString());
+
+        try {
+            return this.schemaChange(
+                    database, table, buildRequestParam(true, columnName), alterColumnDDL);
+        } catch (RuntimeException ex) {
+            if (ex.getMessage().contains("Nothing is changed. please check your alter stmt.")) {

Review Comment:
   Got it. 
   And I have checked the history of this code and it has been a long time since it was modified. It looks quite stable, so adding a comment to describe the source is enough.



-- 
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@flink.apache.org

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


Re: [PR] [FLINK-35072][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

Posted by "yuxiqian (via GitHub)" <gi...@apache.org>.
yuxiqian commented on PR #3215:
URL: https://github.com/apache/flink-cdc/pull/3215#issuecomment-2050843987

   I'm not familiar with Doris, could @lvyanquan @JNSimba take a look?


-- 
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@flink.apache.org

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


Re: [PR] [FLINK-35072][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

Posted by "lvyanquan (via GitHub)" <gi...@apache.org>.
lvyanquan commented on code in PR #3215:
URL: https://github.com/apache/flink-cdc/pull/3215#discussion_r1565182494


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisSchemaChangeManager.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.cdc.connectors.doris.sink;
+
+import org.apache.doris.flink.catalog.doris.FieldSchema;
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+import org.apache.doris.flink.sink.schema.SchemaChangeManager;
+
+import java.io.IOException;
+
+/** Provides schema change operations based on {@link SchemaChangeManager}. */
+public class DorisSchemaChangeManager extends SchemaChangeManager {
+    public DorisSchemaChangeManager(DorisOptions dorisOptions) {
+        super(dorisOptions);
+    }
+
+    private static final String MODIFY_COLUMN_DDL = "ALTER TABLE %s MODIFY COLUMN %s %s";
+
+    public boolean alterColumn(
+            String database, String table, String columnName, String newColumnType)
+            throws IOException, IllegalArgumentException {
+        String tableIdentifier = String.format("%s.%s", database, table);
+        FieldSchema alterFieldSchema = new FieldSchema(columnName, newColumnType, "");
+
+        String alterColumnDDL =
+                String.format(
+                        MODIFY_COLUMN_DDL,
+                        tableIdentifier,
+                        columnName,
+                        alterFieldSchema.getTypeString());
+
+        try {
+            return this.schemaChange(
+                    database, table, buildRequestParam(true, columnName), alterColumnDDL);
+        } catch (RuntimeException ex) {
+            if (ex.getMessage().contains("Nothing is changed. please check your alter stmt.")) {

Review Comment:
   Considering future maintenance, could you please add some description about the source code of this error message? and is there a method to determine through exception class.



-- 
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@flink.apache.org

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


Re: [PR] [FLINK-35072][cdc][doris] Support applying compatible `AlterColumnTypeEvent` to Doris sink [flink-cdc]

Posted by "yuxiqian (via GitHub)" <gi...@apache.org>.
yuxiqian commented on code in PR #3215:
URL: https://github.com/apache/flink-cdc/pull/3215#discussion_r1565211096


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisSchemaChangeManager.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.cdc.connectors.doris.sink;
+
+import org.apache.doris.flink.catalog.doris.FieldSchema;
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+import org.apache.doris.flink.sink.schema.SchemaChangeManager;
+
+import java.io.IOException;
+
+/** Provides schema change operations based on {@link SchemaChangeManager}. */
+public class DorisSchemaChangeManager extends SchemaChangeManager {
+    public DorisSchemaChangeManager(DorisOptions dorisOptions) {
+        super(dorisOptions);
+    }
+
+    private static final String MODIFY_COLUMN_DDL = "ALTER TABLE %s MODIFY COLUMN %s %s";
+
+    public boolean alterColumn(
+            String database, String table, String columnName, String newColumnType)
+            throws IOException, IllegalArgumentException {
+        String tableIdentifier = String.format("%s.%s", database, table);
+        FieldSchema alterFieldSchema = new FieldSchema(columnName, newColumnType, "");
+
+        String alterColumnDDL =
+                String.format(
+                        MODIFY_COLUMN_DDL,
+                        tableIdentifier,
+                        columnName,
+                        alterFieldSchema.getTypeString());
+
+        try {
+            return this.schemaChange(
+                    database, table, buildRequestParam(true, columnName), alterColumnDDL);
+        } catch (RuntimeException ex) {
+            if (ex.getMessage().contains("Nothing is changed. please check your alter stmt.")) {

Review Comment:
   @lvyanquan Sure, but this error message comes from [Doris API server side](https://github.com/apache/doris/blob/debb83f35d37cac9089b2e0282e2d555ad9c1d62/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java#L1508), which is wrapped as a generic `RuntimeException` on client-side, and could only be identified by checking error message. Any suggestions about this?



-- 
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@flink.apache.org

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