yux created FLINK-35984:
---------------------------

             Summary: Job crashes when metadata column names present in 
transform rules
                 Key: FLINK-35984
                 URL: https://issues.apache.org/jira/browse/FLINK-35984
             Project: Flink
          Issue Type: Bug
          Components: Flink CDC
            Reporter: yux


Given such a transform rule:

transform:
  projection: \*, '__namespace_name__schema_name__table_name__' AS 
string_literal

Obviously this shouldn't insert any metadata columns. However since metadata 
column existence check was done by searching identifier string in statement, 
without considering any syntax info:

```java
// TransformParser.java#L357
if (transformStatement.contains(DEFAULT_NAMESPACE_NAME) ...
```

Transform operator will mistakenly append metadata columns into Janino 
arguments list, and crash the job:

Caused by: java.lang.IllegalArgumentException: wrong number of arguments
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.codehaus.janino.ExpressionEvaluator.evaluate(ExpressionEvaluator.java:541)
        at 
org.codehaus.janino.ExpressionEvaluator.evaluate(ExpressionEvaluator.java:533)
        at 
org.apache.flink.cdc.runtime.operators.transform.ProjectionColumnProcessor.evaluate(ProjectionColumnProcessor.java:64)
        at 
org.apache.flink.cdc.runtime.operators.transform.TransformProjectionProcessor.processData(TransformProjectionProcessor.java:153)
        at 
org.apache.flink.cdc.runtime.operators.transform.TransformDataOperator.processProjection(TransformDataOperator.java:387)
        at 
org.apache.flink.cdc.runtime.operators.transform.TransformDataOperator.processDataChangeEvent(TransformDataOperator.java:328)
        at 
org.apache.flink.cdc.runtime.operators.transform.TransformDataOperator.processElement(TransformDataOperator.java:190)
        at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:237)
        at 
org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:146)
        at 
org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:110)
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:562)
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:858)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:807)
        at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:953)
        at 
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:932)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:746)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562)
        at java.lang.Thread.run(Thread.java:748)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to