gabrywu opened a new issue, #27733:
URL: https://github.com/apache/beam/issues/27733

   ### What happened?
   
   ```java
   Schema schema = Schema.builder()
                   .addInt64Field("id")
                   .addStringField("json")
                   .build();
   
           Schema outputSchema = Schema.builder()
                   .addInt64Field("id")
                   .addStringField("json")
                   .addStringField("json_element")
                   .build();
   
           Row row = Row.withSchema(schema)
                   .addValue(System.currentTimeMillis())
                   .addValue("[{\"id\":1},{\"id\":1123456789},{\"id\":-1}]")
                   .build();
   
           Pipeline pipeline = 
Pipeline.create(PipelineOptionsFactory.fromArgs("--runner=FlinkRunner"
                   , "--fasterCopy=true"
                   , "--disableMetrics=true"
                   , "--externalizedCheckpointsEnabled=false").create());
   
           pipeline.apply(Create.of(row))
                   .setRowSchema(schema)
                   .apply("filter", SqlTransform
                           .query("select * from" +
                                   " (select a.*,j.i from PCOLLECTION a" +
                                   " cross join unnest(to_json_str_array(json)) 
as j(i)" +
                                   ") where json_value(i,'$.id') <> -1 "))
                   .apply("output", MapElements.via(new SimpleFunction<Row, 
Row>() {
                       @Override
                       public Row apply(Row input) {
                           logger.info("current row {}", input);
                           return input;
                       }
                   }))
                   .setRowSchema(outputSchema);
           pipeline.run().waitUntilFinish();
   ```
   
   Hi, folks, I'm trying to filter the final result using a simple condition, 
however it doesn't works. All data are outputed in final result, can you help 
to look into it?
   
   
   ```java
   2023-07-28 15:46:32.772 
org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner INFO  - SQLPlan>
   LogicalProject(id=[$0], json=[$1], i=[$2])
     LogicalProject(id=[$0], json=[$1], i=[$2])
       LogicalProject(id=[$0], json=[$1], i=[$3])
         LogicalCorrelate(correlation=[$cor0], joinType=[inner], 
requiredColumns=[{2}])
           LogicalProject(id=[$0], json=[$1], $f2=[to_json_str_array($1)])
             BeamIOSourceRel(table=[[beam, PCOLLECTION]])
           LogicalFilter(condition=[<>(CAST(JSON_VALUE($0, '$.id')):INTEGER, 
-1)])
             LogicalProject(i=[$0])
               Uncollect
                 LogicalProject(EXPR$0=[$cor0.$f2])
                   LogicalValues(tuples=[[{ 0 }]])
   
   2023-07-28 15:46:32.964 
org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner INFO  - BEAMPlan>
   BeamCalcRel(expr#0..3=[{inputs}], proj#0..1=[{exprs}], i=[$t3])
     BeamUnnestRel(unnestIndices=[[2]])
       BeamCalcRel(expr#0..1=[{inputs}], expr#2=[to_json_str_array($t1)], 
proj#0..2=[{exprs}])
         BeamIOSourceRel(table=[[beam, PCOLLECTION]])
   ```
   
   It seems that `LogicalFilter(condition=[<>(CAST(JSON_VALUE($0, 
'$.id')):INTEGER, -1)])` is missing
   
   ### Issue Priority
   
   Priority: 0 (outage / urgent vulnerability)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [X] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [X] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to