zhangjun0x01 commented on a change in pull request #2229:
URL: https://github.com/apache/iceberg/pull/2229#discussion_r572630207
##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -100,70 +96,67 @@ public boolean isBounded() {
.build();
}
- @Override
- public TableSchema getTableSchema() {
- return schema;
- }
-
- @Override
- public DataType getProducedDataType() {
- return getProjectedSchema().toRowDataType().bridgedTo(RowData.class);
- }
-
private TableSchema getProjectedSchema() {
- TableSchema fullSchema = getTableSchema();
if (projectedFields == null) {
- return fullSchema;
+ return schema;
} else {
- String[] fullNames = fullSchema.getFieldNames();
- DataType[] fullTypes = fullSchema.getFieldDataTypes();
+ String[] fullNames = schema.getFieldNames();
+ DataType[] fullTypes = schema.getFieldDataTypes();
return TableSchema.builder().fields(
Arrays.stream(projectedFields).mapToObj(i ->
fullNames[i]).toArray(String[]::new),
Arrays.stream(projectedFields).mapToObj(i ->
fullTypes[i]).toArray(DataType[]::new)).build();
}
}
@Override
- public String explainSource() {
- String explain = "Iceberg table: " + loader.toString();
- if (projectedFields != null) {
- explain += ", ProjectedFields: " + Arrays.toString(projectedFields);
- }
-
- if (isLimitPushDown) {
- explain += String.format(", LimitPushDown : %d", limit);
- }
+ public void applyLimit(long newLimit) {
+ this.limit = newLimit;
+ }
- if (isFilterPushedDown()) {
- explain += String.format(", FilterPushDown: %s", COMMA.join(filters));
+ @Override
+ public Result applyFilters(List<ResolvedExpression> flinkFilters) {
+ List<org.apache.iceberg.expressions.Expression> expressions =
Lists.newArrayList();
+ for (Expression predicate : flinkFilters) {
+ FlinkFilters.convert(predicate).ifPresent(expressions::add);
}
- return TableConnectorUtils.generateRuntimeName(getClass(),
getTableSchema().getFieldNames()) + explain;
+ this.filters = expressions;
+ return Result.of(flinkFilters, flinkFilters);
}
@Override
- public boolean isLimitPushedDown() {
- return isLimitPushDown;
+ public boolean supportsNestedProjection() {
+ return false;
}
@Override
- public TableSource<RowData> applyLimit(long newLimit) {
- return new IcebergTableSource(loader, schema, properties, projectedFields,
true, newLimit, filters, readableConfig);
+ public ChangelogMode getChangelogMode() {
Review comment:
now ,flink read only support `INSERT-only`, if we add other mode , it
will throw an exception ,like this:
`
Querying a table in batch mode is currently only possible for INSERT-only
table sources. But the source for table 'iceberg_catalog.default.t' produces
other changelog messages than just INSERT.
`
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]