[ https://issues.apache.org/jira/browse/FLINK-37905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Yanquan Lv resolved FLINK-37905. -------------------------------- Resolution: Fixed Fixed in master via 224a07510be35c5d0dcfc9aa0a8dcbd88b0e7733. > Transform will throw exceptions when handling chinese characters > ----------------------------------------------------------------- > > Key: FLINK-37905 > URL: https://issues.apache.org/jira/browse/FLINK-37905 > Project: Flink > Issue Type: Bug > Components: Flink CDC > Affects Versions: cdc-3.4.0 > Reporter: Ruan Hang > Assignee: yux > Priority: Major > Labels: pull-request-available > Fix For: cdc-3.5.0 > > > {code:java} > source: > type: mysql > hostname: ***** > port: 3306 > username: ***** > password: ***** > tables: test.task_table > server-id: 7001 > scan.incremental.snapshot.chunk.size: 10000 > scan.newly-added-table.enabled: true > > transform: > - source-table: test.task_table > filter: TASK_NAME = '测试' > description: filtering rows from source tablepipeline: > name: Sync MySQL Database to MySQL > parallelism: 1 {code} > This job will fail with the following exception. > {code:java} > 2025-05-30 11:03:58,265 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Transform:Data -> > SchemaOperator -> PrePartition (1/1) > (89ef20142901e12bd2a9853fe1c079c5_90bea66de1c231edf33913ecd54406c1_0_0) > switched from RUNNING to FAILED on localhost:33549-0c85b2 @ localhost > (dataPort=39423). > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Expression cannot be > compiled. This is a bug. Please file an issue. > Expression: import static > org.apache.flink.cdc.runtime.functions.SystemFunctionUtils.;valueEquals($0, > "&'\6d4b\8bd5") > Column name map: {TASK_NAME=$0} > at > org.apache.flink.cdc.runtime.operators.transform.TransformExpressionCompiler.compileExpression(TransformExpressionCompiler.java:85) > ~[?:?] > at > org.apache.flink.cdc.runtime.operators.transform.TransformFilterProcessor.(TransformFilterProcessor.java:67) > ~[?:?] > at > org.apache.flink.cdc.runtime.operators.transform.TransformFilterProcessor.of(TransformFilterProcessor.java:83) > ~[?:?] > at > org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator.processDataChangeEvent(PostTransformOperator.java:418) > ~[?:?] > at > org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator.processElement(PostTransformOperator.java:258) > ~[?:?] > at > org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:238) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:157) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:114) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:638) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:973) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:917) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:970) > ~[flink-dist-1.20.1.jar:1.20.1] > at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:949) > ~[flink-dist-1.20.1.jar:1.20.1] > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763) > ~[flink-dist-1.20.1.jar:1.20.1] > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) > ~[flink-dist-1.20.1.jar:1.20.1] > at java.lang.Thread.run(Thread.java:840) ~[?:?] > Caused by: > org.apache.flink.shaded.guava31.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Expression cannot be > compiled. This is a bug. Please file an issue. > Expression: import static > org.apache.flink.cdc.runtime.functions.SystemFunctionUtils.;valueEquals($0, > "&'\6d4b\8bd5") > Column name map: {TASK_NAME=$0} > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2055) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache.get(LocalCache.java:3966) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4863) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.cdc.runtime.operators.transform.TransformExpressionCompiler.compileExpression(TransformExpressionCompiler.java:52) > ~[?:?] > ... 17 more > Caused by: org.apache.flink.api.common.InvalidProgramException: Expression > cannot be compiled. This is a bug. Please file an issue. > Expression: import static > org.apache.flink.cdc.runtime.functions.SystemFunctionUtils.*;valueEquals($0, > "&'\6d4b\8bd5") > Column name map: {TASK_NAME=$0} > at > org.apache.flink.cdc.runtime.operators.transform.TransformExpressionCompiler.lambda$compileExpression$0(TransformExpressionCompiler.java:77) > ~[?:?] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4868) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3533) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2282) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2159) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2049) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache.get(LocalCache.java:3966) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4863) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.cdc.runtime.operators.transform.TransformExpressionCompiler.compileExpression(TransformExpressionCompiler.java:52) > ~[?:?] > ... 17 more > Caused by: org.codehaus.commons.compiler.CompileException: Line 1, Column 92: > Invalid escape sequence > at org.codehaus.janino.Scanner.scanLiteralCharacter(Scanner.java:669) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Scanner.scan(Scanner.java:373) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Scanner.produce(Scanner.java:292) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.TokenStreamImpl.produceToken(TokenStreamImpl.java:55) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.TokenStreamImpl.peek(TokenStreamImpl.java:94) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.TokenStreamImpl.peek(TokenStreamImpl.java:119) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.peek(Parser.java:3797) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseExpression(Parser.java:2733) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseArgumentList(Parser.java:3675) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseArguments(Parser.java:3661) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parsePrimary(Parser.java:3323) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseUnaryExpression(Parser.java:3124) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseMultiplicativeExpression(Parser.java:3083) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseAdditiveExpression(Parser.java:3062) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseShiftExpression(Parser.java:3041) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseRelationalExpression(Parser.java:2936) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseEqualityExpression(Parser.java:2910) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseAndExpression(Parser.java:2889) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseExclusiveOrExpression(Parser.java:2868) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseInclusiveOrExpression(Parser.java:2847) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseConditionalAndExpression(Parser.java:2826) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseConditionalOrExpression(Parser.java:2805) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseConditionalExpression(Parser.java:2786) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseAssignmentExpression(Parser.java:2767) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.Parser.parseExpression(Parser.java:2735) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at > org.codehaus.janino.ExpressionEvaluator.cook2(ExpressionEvaluator.java:480) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.ExpressionEvaluator.cook(ExpressionEvaluator.java:452) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.ExpressionEvaluator.cook(ExpressionEvaluator.java:443) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.ExpressionEvaluator.cook(ExpressionEvaluator.java:409) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.janino.ExpressionEvaluator.cook(ExpressionEvaluator.java:394) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:82) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:77) > ~[flink-table-runtime-1.20.1.jar:1.20.1] > at > org.apache.flink.cdc.runtime.operators.transform.TransformExpressionCompiler.lambda$compileExpression$0(TransformExpressionCompiler.java:74) > ~[?:?] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4868) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3533) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2282) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2159) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2049) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache.get(LocalCache.java:3966) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.shaded.guava31.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4863) > ~[flink-dist-1.20.1.jar:1.20.1] > at > org.apache.flink.cdc.runtime.operators.transform.TransformExpressionCompiler.compileExpression(TransformExpressionCompiler.java:52) > ~[?:?] > ... 17 more {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)