Skip to content

Conversation

@Fokko
Copy link
Contributor

@Fokko Fokko commented Mar 14, 2023

For Flink, we apply partition pruning, filtering based on metrics, and row-group skipping, but no row-level filtering.

See the issue for more details

Resolves #7022

@stevenzwu would you have time to take a peek at this one?

Copy link
Contributor

@doki23 doki23 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, except for a potential NPE problem.

@stevenzwu stevenzwu self-requested a review March 18, 2023 15:13
@github-actions github-actions bot added the API label Mar 19, 2023
Copy link
Contributor

@doki23 doki23 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great!


public class FlinkSourceFilter implements FilterFunction<RowData> {

private final RowDataWrapper wrapper;
Copy link
Contributor

@stevenzwu stevenzwu Mar 21, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we don't have to make RowDataWrapper serializable. it can be lazily initialized.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lazily initialized is ok, but it makes sense that making it serializable -- it's meaningful that we don't need to check the if statement per record.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm open to both. It is just one null check.

}
return env.createInput(format, typeInfo).setParallelism(parallelism);

DataStreamSource<RowData> source =
Copy link
Contributor

@stevenzwu stevenzwu Mar 21, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This covers one scenario. there are two other scenarios.

  1. Use FlinkInputFormat directly. e.g. StreamingReaderOperator.
  private void processSplits() throws IOException {
    FlinkInputSplit split = splits.poll();
    if (split == null) {
      currentSplitState = SplitState.IDLE;
      return;
    }

    format.open(split);
    try {
      RowData nextElement = null;
      while (!format.reachedEnd()) {
        nextElement = format.nextRecord(nextElement);
        sourceContext.collect(nextElement);
      }
    } finally {
      currentSplitState = SplitState.IDLE;
      format.close();
    }

    // Re-schedule to process the next split.
    enqueueProcessSplits();
  }
  1. new Flink FLIP-27 IcebergSource. Here is an example from IcebergTableSource that shows how users can construct the DataStream. We can fix it in IcebergTableSource. but we can't control users' code to add the filter in the DataStream. Note that FLIP-27 source will be the future Flink source.
  private DataStreamSource<RowData> createFLIP27Stream(StreamExecutionEnvironment env) {
    SplitAssignerType assignerType =
        readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE);
    IcebergSource<RowData> source =
        IcebergSource.forRowData()
            .tableLoader(loader)
            .assignerFactory(assignerType.factory())
            .properties(properties)
            .project(getProjectedSchema())
            .limit(limit)
            .filters(filters)
            .flinkConfig(readableConfig)
            .build();
    DataStreamSource stream =
        env.fromSource(
            source,
            WatermarkStrategy.noWatermarks(),
            source.name(),
            TypeInformation.of(RowData.class));
    return stream;
  }

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

another possible common place to evaluate the residual filter is probably RowDataFileScanTaskReader. This approach also brings a benefit that it doesn't change the shape of Flink DAG. E.g., users won't see an extra filter function/operator and wonder where does it come from.

@github-actions github-actions bot removed the API label Apr 10, 2023
@Fokko Fokko force-pushed the fd-flink branch 2 times, most recently from 982469e to b876007 Compare April 10, 2023 19:03
@Fokko
Copy link
Contributor Author

Fokko commented Apr 10, 2023

@stevenzwu Sorry for the long wait. I've updated the PR according to your suggestion. Looking at the FLIP27 tests, I think row filtering is already applied over there. I also took the liberty to change some public methods, since Flink 1.17 hasn't been released to the public.

Copy link
Contributor

@stevenzwu stevenzwu left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for making the change. It seems that performing the filtering at RowDataFileScanTaskReader is the right approach, as it is the common denominator.

}

@Test
public void testBasicRowDataFiltering() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this method won't be necessary if we add the new test method above to TestFlinkScan. Currently, TestFlinkScan#testFilterExp only covers filter with partition column. we can add the above test method as testResidualFilter where filter is constructed with non-partition column.

TestFlinkScan is the base class for both the old FlinkSource(covered by TestFlinkInputFormat and others) and the new FLIP-27 IcebergSource (covered by TestIcebergSourceBounded and others)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've added a test in TestFlinkScan that filters on a not-partitioned column

@Fokko
Copy link
Contributor Author

Fokko commented Apr 12, 2023

@stevenzwu thanks again for the review, could you do another pass?

DataFile dataFile = helper.writeFile(expectedRecords);
helper.appendToTable(dataFile);
List<Row> actual =
runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ideally, we also want to test the ignore case sensitive option with filter. but the current test structure seems very hard to do that. we need both filter and options

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a good point. I got it working quite easily when using the expression. Do you know if Flink supports case-insensitive mode when it comes to Flink SQL? Seems to be an option issue: https://issues.apache.org/jira/browse/FLINK-16175

Copy link
Contributor

@stevenzwu stevenzwu left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

added a comment for testing the case insensitive scenario. but seems difficult with the current test code structure.

}

@Test
public void testFilterExpCaseInsensitive() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: for now, maybe extract a private method to avoid test duplications. with Junit 5, this can be handled better.

@Fokko Fokko merged commit c57952e into apache:master Apr 19, 2023
@Fokko Fokko deleted the fd-flink branch April 19, 2023 07:29
@Fokko
Copy link
Contributor Author

Fokko commented Apr 19, 2023

Thanks @stevenzwu and @doki23 for the review!

@stevenzwu
Copy link
Contributor

@Fokko thx for fixing this issue

@stevenzwu
Copy link
Contributor

@Fokko can you also create the backport PR for 1.15 and 1.16?

@Fokko
Copy link
Contributor Author

Fokko commented Apr 21, 2023

@stevenzwu sure thing! Here you go: #7397

manisin pushed a commit to Snowflake-Labs/iceberg that referenced this pull request May 9, 2023
* Flink: Apply row level filtering

* Fix the tests

* Add test for case-sensitive

* Reduce duplication using a private method
Comment on lines +141 to +143
if (rowFilter != null) {
return CloseableIterable.filter(iter, rowFilter::filter);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Fokko @stevenzwu We have an internal request to add row filter dynamically, current impl requires the filter to be supplied at job startup time. After some investigation, I believe maybe we don't have to passing the filters all the way done to this class.. The task itself already has the row filter task.residual(). We could simply convert it to rowFilter here, such as:

//    if (rowFilter != null) {
//      return CloseableIterable.filter(iter, rowFilter::filter);
//    }
    if (task.residual() != null && !task.residual().isEquivalentTo(Expressions.alwaysTrue())) {
      FlinkSourceFilter dataFilter =
              new FlinkSourceFilter(this.projectedSchema, task.residual(), this.caseSensitive);
      return CloseableIterable.filter(iter, dataFilter::filter);
    }

WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe maybe we don't have to passing the filters all the way done to this class.. The task itself already has the row filter task.residual().

if that is the case, it is probably simpler.

add row filter dynamically,

can you explain what's the dynamic part and how is it related to the residual filter?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you explain what's the dynamic part and how is it related to the residual filter?

The feature we are developing needs to add some additional scan filters at planIcebergSourceSplits phase, the additional scan filter is added to scanContext, which would be part of task.residual.

The additional filter itself is dynamically added.

if that is the case, it is probably simpler.

Then, if we can just use task.residual, do you think we should refactor this PR to use that? Is it possible to revert the class interface change in this PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@advancedxy That sounds like a great approach. If we only have to apply the residual, then we're also more efficient. That would be an improvement over the current approach.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Flink: filters not applied at row-level for non-partition columns

4 participants