Skip to content

Commit

Permalink
[FLINK-21549][table-planner-blink] Support json serialization/deseria…
Browse files Browse the repository at this point in the history
…lization for the push-down result of DynamicTableSource

This closes apache#15062
  • Loading branch information
godfreyhe committed Mar 7, 2021
1 parent b696144 commit cf9a1e4
Show file tree
Hide file tree
Showing 41 changed files with 2,112 additions and 444 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,11 @@ public FlinkTypeFactory getTypeFactory() {
return typeFactory;
}

/** Returns the {@link FlinkContext}. */
public FlinkContext getFlinkContext() {
return context;
}

/**
* Creates a configured {@link FlinkRelBuilder} for a planning session.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.abilities.source;

import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.expressions.resolver.ExpressionResolver;
import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
import org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;

import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.TimeZone;
import java.util.stream.Collectors;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A sub-class of {@link SourceAbilitySpec} that can not only serialize/deserialize the filter
* to/from JSON, but also can push the filter into a {@link SupportsFilterPushDown}.
*/
@JsonTypeName("FilterPushDown")
public class FilterPushDownSpec extends SourceAbilitySpecBase {
public static final String FIELD_NAME_PREDICATES = "predicates";

@JsonProperty(FIELD_NAME_PREDICATES)
private final List<RexNode> predicates;

@JsonCreator
public FilterPushDownSpec(@JsonProperty(FIELD_NAME_PREDICATES) List<RexNode> predicates) {
this.predicates = new ArrayList<>(checkNotNull(predicates));
}

@Override
public void apply(DynamicTableSource tableSource, SourceAbilityContext context) {
SupportsFilterPushDown.Result result = apply(predicates, tableSource, context);
if (result.getAcceptedFilters().size() != predicates.size()) {
throw new TableException("All predicates should be accepted here.");
}
}

public static SupportsFilterPushDown.Result apply(
List<RexNode> predicates,
DynamicTableSource tableSource,
SourceAbilityContext context) {
if (tableSource instanceof SupportsFilterPushDown) {
RexNodeToExpressionConverter converter =
new RexNodeToExpressionConverter(
new RexBuilder(FlinkTypeFactory.INSTANCE()),
context.getSourceRowType().getFieldNames().toArray(new String[0]),
context.getFunctionCatalog(),
context.getCatalogManager(),
TimeZone.getTimeZone(context.getTableConfig().getLocalTimeZone()));
List<Expression> filters =
predicates.stream()
.map(
p -> {
scala.Option<ResolvedExpression> expr = p.accept(converter);
if (expr.isDefined()) {
return expr.get();
} else {
throw new TableException(
String.format(
"%s can not be converted to Expression, please make sure %s can accept %s.",
p.toString(),
tableSource.getClass().getSimpleName(),
p.toString()));
}
})
.collect(Collectors.toList());
ExpressionResolver resolver =
ExpressionResolver.resolverFor(
context.getTableConfig(),
name -> Optional.empty(),
context.getFunctionCatalog()
.asLookup(
str -> {
throw new TableException(
"We should not need to lookup any expressions at this point");
}),
context.getCatalogManager().getDataTypeFactory(),
(sqlExpression, inputSchema) -> {
throw new TableException(
"SQL expression parsing is not supported at this location.");
})
.build();
return ((SupportsFilterPushDown) tableSource).applyFilters(resolver.resolve(filters));
} else {
throw new TableException(
String.format(
"%s does not support SupportsFilterPushDown.",
tableSource.getClass().getName()));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.abilities.source;

import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

/**
* A sub-class of {@link SourceAbilitySpec} that can not only serialize/deserialize the limit value
* to/from JSON, but also can push the limit value into a {@link LimitPushDownSpec}.
*/
@JsonTypeName("LimitPushDown")
public class LimitPushDownSpec extends SourceAbilitySpecBase {
public static final String FIELD_NAME_LIMIT = "limit";

@JsonProperty(FIELD_NAME_LIMIT)
private final long limit;

@JsonCreator
public LimitPushDownSpec(@JsonProperty(FIELD_NAME_LIMIT) long limit) {
this.limit = limit;
}

@Override
public void apply(DynamicTableSource tableSource, SourceAbilityContext context) {
if (tableSource instanceof SupportsLimitPushDown) {
((SupportsLimitPushDown) tableSource).applyLimit(limit);
} else {
throw new TableException(
String.format(
"%s does not support SupportsLimitPushDown.",
tableSource.getClass().getName()));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.abilities.source;

import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsPartitionPushDown;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

import java.util.ArrayList;
import java.util.List;
import java.util.Map;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A sub-class of {@link SourceAbilitySpec} that can not only serialize/deserialize the partitions
* to/from JSON, but also can push the partitions into a {@link SupportsPartitionPushDown}.
*/
@JsonTypeName("PartitionPushDown")
public class PartitionPushDownSpec extends SourceAbilitySpecBase {
public static final String FIELD_NAME_PARTITIONS = "partitions";

@JsonProperty(FIELD_NAME_PARTITIONS)
private final List<Map<String, String>> partitions;

@JsonCreator
public PartitionPushDownSpec(
@JsonProperty(FIELD_NAME_PARTITIONS) List<Map<String, String>> partitions) {
this.partitions = new ArrayList<>(checkNotNull(partitions));
}

@Override
public void apply(DynamicTableSource tableSource, SourceAbilityContext context) {
if (tableSource instanceof SupportsPartitionPushDown) {
((SupportsPartitionPushDown) tableSource).applyPartitions(partitions);
} else {
throw new TableException(
String.format(
"%s does not support SupportsPartitionPushDown.",
tableSource.getClass().getName()));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.abilities.source;

import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
import org.apache.flink.table.types.logical.RowType;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A sub-class of {@link SourceAbilitySpec} that can not only serialize/deserialize the projection
* to/from JSON, but also can push the projection into a {@link SupportsProjectionPushDown}.
*/
@JsonTypeName("ProjectPushDown")
public class ProjectPushDownSpec extends SourceAbilitySpecBase {
public static final String FIELD_NAME_PROJECTED_FIELDS = "projectedFields";

@JsonProperty(FIELD_NAME_PROJECTED_FIELDS)
private final int[][] projectedFields;

@JsonCreator
public ProjectPushDownSpec(
@JsonProperty(FIELD_NAME_PROJECTED_FIELDS) int[][] projectedFields,
@JsonProperty(FIELD_NAME_PRODUCED_TYPE) RowType producedType) {
super(producedType);
this.projectedFields = checkNotNull(projectedFields);
}

@Override
public void apply(DynamicTableSource tableSource, SourceAbilityContext context) {
if (tableSource instanceof SupportsProjectionPushDown) {
((SupportsProjectionPushDown) tableSource).applyProjection(projectedFields);
} else {
throw new TableException(
String.format(
"%s does not support SupportsProjectionPushDown.",
tableSource.getClass().getName()));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.abilities.source;

import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.types.utils.TypeConversions;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

import java.util.ArrayList;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A sub-class of {@link SourceAbilitySpec} that can not only serialize/deserialize the metadata
* columns to/from JSON, but also can read the metadata columns from {@link
* SupportsReadingMetadata}.
*/
@JsonTypeName("ReadingMetadata")
public class ReadingMetadataSpec extends SourceAbilitySpecBase {
public static final String FIELD_NAME_METADATA_KEYS = "metadataKeys";

@JsonProperty(FIELD_NAME_METADATA_KEYS)
private final List<String> metadataKeys;

@JsonCreator
public ReadingMetadataSpec(
@JsonProperty(FIELD_NAME_METADATA_KEYS) List<String> metadataKeys,
@JsonProperty(FIELD_NAME_PRODUCED_TYPE) RowType producedType) {
super(producedType);
this.metadataKeys = new ArrayList<>(checkNotNull(metadataKeys));
}

@Override
public void apply(DynamicTableSource tableSource, SourceAbilityContext context) {
if (tableSource instanceof SupportsReadingMetadata) {
checkArgument(getProducedType().isPresent());
DataType producedDataType =
TypeConversions.fromLogicalToDataType(getProducedType().get());
((SupportsReadingMetadata) tableSource)
.applyReadableMetadata(metadataKeys, producedDataType);
} else {
throw new TableException(
String.format(
"%s does not support SupportsReadingMetadata.",
tableSource.getClass().getName()));
}
}
}
Loading

0 comments on commit cf9a1e4

Please sign in to comment.