Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.TimestampData;
Expand All @@ -46,12 +45,10 @@
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;

import java.math.BigDecimal;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.sql.Time;
import java.time.LocalDateTime;
import java.util.ArrayList;
import java.util.Arrays;
Expand Down Expand Up @@ -358,31 +355,22 @@ void testLookupJoin(Caching caching) throws Exception {
}

private void validateCachedValues(LookupCache cache) {
// jdbc does support project push down, the cached row has been projected
RowData key1 = GenericRowData.of(1L);
RowData value1 =
GenericRowData.of(
1L,
TimestampData.fromLocalDateTime(
LocalDateTime.parse("2020-01-01T15:35:00.123456")),
TimestampData.fromLocalDateTime(
LocalDateTime.parse("2020-01-01T15:35:00.123456789")),
(int) (Time.valueOf("15:35:00").toLocalTime().toNanoOfDay() / 1_000_000L),
Float.valueOf("1.175E-37"),
Double.valueOf("1.79769E308"),
DecimalData.fromBigDecimal(BigDecimal.valueOf(100.1234), 10, 4));
Double.valueOf("1.79769E308"));

RowData key2 = GenericRowData.of(2L);
RowData value2 =
GenericRowData.of(
2L,
TimestampData.fromLocalDateTime(
LocalDateTime.parse("2020-01-01T15:36:01.123456")),
TimestampData.fromLocalDateTime(
LocalDateTime.parse("2020-01-01T15:36:01.123456789")),
(int) (Time.valueOf("15:36:01").toLocalTime().toNanoOfDay() / 1_000_000L),
Float.valueOf("-1.175E-37"),
Double.valueOf("-1.79769E308"),
DecimalData.fromBigDecimal(BigDecimal.valueOf(101.1234), 10, 4));
Double.valueOf("-1.79769E308"));

RowData key3 = GenericRowData.of(3L);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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://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.rules.logical;

import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.plan.RelRule;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.logical.LogicalProject;
import org.apache.calcite.rel.logical.LogicalSnapshot;

/** Transpose {@link LogicalProject} past into {@link LogicalSnapshot}. */
public class ProjectSnapshotTransposeRule extends RelRule<ProjectSnapshotTransposeRule.Config> {

public static final RelOptRule INSTANCE =
ProjectSnapshotTransposeRule.Config.EMPTY.as(Config.class).withOperator().toRule();

public ProjectSnapshotTransposeRule(Config config) {
super(config);
}

@Override
public boolean matches(RelOptRuleCall call) {
LogicalProject project = call.rel(0);
// Don't push a project which contains over into a snapshot, snapshot on window aggregate is
// unsupported for now.
return !project.containsOver();
}

@Override
public void onMatch(RelOptRuleCall call) {
LogicalProject project = call.rel(0);
LogicalSnapshot snapshot = call.rel(1);
RelNode newProject = project.copy(project.getTraitSet(), snapshot.getInputs());
RelNode newSnapshot =
snapshot.copy(snapshot.getTraitSet(), newProject, snapshot.getPeriod());
call.transformTo(newSnapshot);
}

/** Configuration for {@link ProjectSnapshotTransposeRule}. */
public interface Config extends RelRule.Config {

@Override
default RelOptRule toRule() {
return new ProjectSnapshotTransposeRule(this);
}

default ProjectSnapshotTransposeRule.Config withOperator() {
final RelRule.OperandTransform snapshotTransform =
operandBuilder -> operandBuilder.operand(LogicalSnapshot.class).noInputs();

final RelRule.OperandTransform projectTransform =
operandBuilder ->
operandBuilder
.operand(LogicalProject.class)
.oneInput(snapshotTransform);

return withOperandSupplier(projectTransform).as(Config.class);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,8 @@ object FlinkBatchRuleSets {
PushProjectIntoLegacyTableSourceScanRule.INSTANCE,
PushFilterIntoTableSourceScanRule.INSTANCE,
PushFilterIntoLegacyTableSourceScanRule.INSTANCE,

// transpose project and snapshot for scan optimization
ProjectSnapshotTransposeRule.INSTANCE,
// reorder sort and projection
CoreRules.SORT_PROJECT_TRANSPOSE,
// remove unnecessary sort rule
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,9 @@ object FlinkStreamRuleSets {
PushFilterIntoLegacyTableSourceScanRule.INSTANCE,
PushLimitIntoTableSourceScanRule.INSTANCE,

// transpose project and snapshot for scan optimization
ProjectSnapshotTransposeRule.INSTANCE,

// reorder the project and watermark assigner
ProjectWatermarkAssignerTransposeRule.INSTANCE,

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,10 @@
import org.apache.flink.table.functions.AsyncLookupFunction;
import org.apache.flink.table.functions.FunctionContext;
import org.apache.flink.table.functions.LookupFunction;
import org.apache.flink.table.runtime.generated.GeneratedProjection;
import org.apache.flink.table.runtime.generated.Projection;
import org.apache.flink.table.runtime.typeutils.InternalSerializers;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.test.util.SuccessException;
import org.apache.flink.types.Row;
import org.apache.flink.types.RowKind;
Expand All @@ -60,6 +64,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
Expand Down Expand Up @@ -558,23 +563,41 @@ public static class TestValuesLookupFunction extends LookupFunction {
private static final long serialVersionUID = 1L;
private final List<Row> data;
private final int[] lookupIndices;

private final RowType producedRowType;

private final LookupTableSource.DataStructureConverter converter;
private final GeneratedProjection generatedProjection;
private final boolean projectable;
private transient Map<RowData, List<RowData>> indexedData;
private transient boolean isOpenCalled = false;
private transient Projection<RowData, GenericRowData> projection;
private transient TypeSerializer<RowData> rowSerializer;

protected TestValuesLookupFunction(
List<Row> data,
int[] lookupIndices,
LookupTableSource.DataStructureConverter converter) {
RowType producedRowType,
LookupTableSource.DataStructureConverter converter,
Optional<GeneratedProjection> generatedProjection) {
this.data = data;
this.lookupIndices = lookupIndices;
this.producedRowType = producedRowType;
this.converter = converter;
this.projectable = generatedProjection.isPresent();
this.generatedProjection = generatedProjection.orElse(null);
}

@Override
public void open(FunctionContext context) throws Exception {
RESOURCE_COUNTER.incrementAndGet();
isOpenCalled = true;
if (projectable) {
projection =
generatedProjection.newInstance(
Thread.currentThread().getContextClassLoader());
}
rowSerializer = InternalSerializers.create(producedRowType);
indexDataByKey();
}

Expand All @@ -601,19 +624,23 @@ private void indexDataByKey() {
data.forEach(
record -> {
GenericRowData rowData = (GenericRowData) converter.toInternal(record);
if (projectable) {
rowData = projection.apply(rowData);
}
checkNotNull(
rowData, "Cannot convert record to internal GenericRowData type");
RowData key =
GenericRowData.of(
Arrays.stream(lookupIndices)
.mapToObj(rowData::getField)
.toArray());
RowData copiedRow = rowSerializer.copy(rowData);
List<RowData> list = indexedData.get(key);
if (list != null) {
list.add(rowData);
list.add(copiedRow);
} else {
list = new ArrayList<>();
list.add(rowData);
list.add(copiedRow);
indexedData.put(key, list);
}
});
Expand All @@ -629,25 +656,42 @@ public static class AsyncTestValueLookupFunction extends AsyncLookupFunction {
private static final long serialVersionUID = 1L;
private final List<Row> data;
private final int[] lookupIndices;
private final RowType producedRowType;
private final LookupTableSource.DataStructureConverter converter;

private final GeneratedProjection generatedProjection;
private final boolean projectable;
private final Random random;
private transient boolean isOpenCalled = false;
private transient ExecutorService executor;
private transient Map<RowData, List<RowData>> indexedData;
private transient Projection<RowData, GenericRowData> projection;
private transient TypeSerializer<RowData> rowSerializer;

protected AsyncTestValueLookupFunction(
List<Row> data,
int[] lookupIndices,
LookupTableSource.DataStructureConverter converter) {
RowType producedRowType,
LookupTableSource.DataStructureConverter converter,
Optional<GeneratedProjection> generatedProjection) {
this.data = data;
this.lookupIndices = lookupIndices;
this.producedRowType = producedRowType;
this.converter = converter;
this.projectable = generatedProjection.isPresent();
this.generatedProjection = generatedProjection.orElse(null);
this.random = new Random();
}

@Override
public void open(FunctionContext context) throws Exception {
RESOURCE_COUNTER.incrementAndGet();
if (projectable) {
projection =
generatedProjection.newInstance(
Thread.currentThread().getContextClassLoader());
}
rowSerializer = InternalSerializers.create(producedRowType);
isOpenCalled = true;
// generate unordered result for async lookup
executor = Executors.newFixedThreadPool(2);
Expand Down Expand Up @@ -689,19 +733,23 @@ private void indexDataByKey() {
data.forEach(
record -> {
GenericRowData rowData = (GenericRowData) converter.toInternal(record);
if (projectable) {
rowData = projection.apply(rowData);
}
checkNotNull(
rowData, "Cannot convert record to internal GenericRowData type");
RowData key =
GenericRowData.of(
Arrays.stream(lookupIndices)
.mapToObj(rowData::getField)
.toArray());
RowData copiedRow = rowSerializer.copy(rowData);
List<RowData> list = indexedData.get(key);
if (list != null) {
list.add(rowData);
list.add(copiedRow);
} else {
list = new ArrayList<>();
list.add(rowData);
list.add(copiedRow);
indexedData.put(key, list);
}
});
Expand All @@ -725,9 +773,11 @@ public static class TestNoLookupUntilNthAccessLookupFunction extends TestValuesL
protected TestNoLookupUntilNthAccessLookupFunction(
List<Row> data,
int[] lookupIndices,
RowType producedRowType,
LookupTableSource.DataStructureConverter converter,
Optional<GeneratedProjection> generatedProjection,
int lookupThreshold) {
super(data, lookupIndices, converter);
super(data, lookupIndices, producedRowType, converter, generatedProjection);
this.lookupThreshold = lookupThreshold;
}

Expand Down Expand Up @@ -771,9 +821,11 @@ public static class TestNoLookupUntilNthAccessAsyncLookupFunction
public TestNoLookupUntilNthAccessAsyncLookupFunction(
List<Row> data,
int[] lookupIndices,
RowType producedRowType,
LookupTableSource.DataStructureConverter converter,
Optional<GeneratedProjection> generatedProjection,
int lookupThreshold) {
super(data, lookupIndices, converter);
super(data, lookupIndices, producedRowType, converter, generatedProjection);
this.lookupThreshold = lookupThreshold;
}

Expand Down
Loading