Skip to content

Commit

Permalink
[connector/flink] Fix NPE and improve project row code in Lookup
Browse files Browse the repository at this point in the history
  • Loading branch information
wuchong committed Dec 26, 2024
1 parent 7758df1 commit 2700ef5
Show file tree
Hide file tree
Showing 6 changed files with 101 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -279,7 +279,7 @@ public boolean isBounded() {
public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
LookupNormalizer lookupNormalizer =
LookupNormalizer.validateAndCreateLookupNormalizer(
context.getKeys(), primaryKeyIndexes, tableOutputType, projectedFields);
context.getKeys(), primaryKeyIndexes, tableOutputType);
if (lookupAsync) {
AsyncLookupFunction asyncLookupFunction =
new FlinkAsyncLookupFunction(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.alibaba.fluss.connector.flink.source.lookup.LookupNormalizer.RemainingFilter;
import com.alibaba.fluss.connector.flink.utils.FlinkConversions;
import com.alibaba.fluss.connector.flink.utils.FlinkRowToFlussRowConverter;
import com.alibaba.fluss.connector.flink.utils.FlinkUtils;
import com.alibaba.fluss.connector.flink.utils.FlussRowToFlinkRowConverter;
import com.alibaba.fluss.exception.TableNotExistException;
import com.alibaba.fluss.metadata.TablePath;
Expand Down Expand Up @@ -88,12 +89,17 @@ public void open(FunctionContext context) {
// TODO: convert to Fluss GenericRow to avoid unnecessary deserialization
flinkRowToFlussRowConverter =
FlinkRowToFlussRowConverter.create(
FlinkLookupFunction.filterRowType(flinkRowType, pkIndexes),
FlinkUtils.projectRowType(flinkRowType, pkIndexes),
table.getDescriptor().getKvFormat());

final RowType outputRowType;
if (projection == null) {
outputRowType = flinkRowType;
} else {
outputRowType = FlinkUtils.projectRowType(flinkRowType, projection);
}
flussRowToFlinkRowConverter =
new FlussRowToFlinkRowConverter(
FlinkConversions.toFlussRowType(
FlinkLookupFunction.filterRowType(flinkRowType, projection)));
new FlussRowToFlinkRowConverter(FlinkConversions.toFlussRowType(outputRowType));
LOG.info("end open.");
}

Expand Down Expand Up @@ -164,8 +170,7 @@ private void fetchResult(
resultFuture.complete(Collections.emptyList());
} else {
RowData flinkRow =
flussRowToFlinkRowConverter.toFlinkRowData(
ProjectedRow.from(projection).replaceRow(row));
flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row));
if (remainingFilter != null && !remainingFilter.isMatch(flinkRow)) {
resultFuture.complete(Collections.emptyList());
} else {
Expand All @@ -176,6 +181,14 @@ private void fetchResult(
});
}

private InternalRow maybeProject(InternalRow row) {
if (projection == null) {
return row;
}
// should not reuse objects for async operations
return ProjectedRow.from(projection).replaceRow(row);
}

@Override
public void close() throws Exception {
LOG.info("start close ...");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,15 @@
import com.alibaba.fluss.config.Configuration;
import com.alibaba.fluss.connector.flink.utils.FlinkConversions;
import com.alibaba.fluss.connector.flink.utils.FlinkRowToFlussRowConverter;
import com.alibaba.fluss.connector.flink.utils.FlinkUtils;
import com.alibaba.fluss.connector.flink.utils.FlussRowToFlinkRowConverter;
import com.alibaba.fluss.metadata.TablePath;
import com.alibaba.fluss.row.InternalRow;
import com.alibaba.fluss.row.ProjectedRow;

import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.utils.ProjectedRowData;
import org.apache.flink.table.functions.FunctionContext;
import org.apache.flink.table.functions.LookupFunction;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -59,7 +58,7 @@ public class FlinkLookupFunction extends LookupFunction {
private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter;
private transient Connection connection;
private transient Table table;
private transient ProjectedRowData projectedRowData;
@Nullable private transient ProjectedRow projectedRow;

public FlinkLookupFunction(
Configuration flussConfig,
Expand All @@ -78,16 +77,6 @@ public FlinkLookupFunction(
this.projection = projection;
}

static RowType filterRowType(RowType rowType, int[] filterIndex) {
LogicalType[] types = new LogicalType[filterIndex.length];
String[] names = new String[filterIndex.length];
for (int i = 0; i < filterIndex.length; i++) {
types[i] = rowType.getTypeAt(filterIndex[i]);
names[i] = rowType.getFieldNames().get(filterIndex[i]);
}
return RowType.of(rowType.isNullable(), types, names);
}

@Override
public void open(FunctionContext context) {
LOG.info("start open ...");
Expand All @@ -96,11 +85,21 @@ public void open(FunctionContext context) {
// TODO: convert to Fluss GenericRow to avoid unnecessary deserialization
flinkRowToFlussRowConverter =
FlinkRowToFlussRowConverter.create(
filterRowType(flinkRowType, pkIndexes),
FlinkUtils.projectRowType(flinkRowType, pkIndexes),
table.getDescriptor().getKvFormat());

final RowType outputRowType;
if (projection == null) {
outputRowType = flinkRowType;
projectedRow = null;
} else {
outputRowType = FlinkUtils.projectRowType(flinkRowType, projection);
// reuse the projected row
projectedRow = ProjectedRow.from(projection);
}
flussRowToFlinkRowConverter =
new FlussRowToFlinkRowConverter(
FlinkConversions.toFlussRowType(filterRowType(flinkRowType, projection)));
new FlussRowToFlinkRowConverter(FlinkConversions.toFlussRowType(outputRowType));

LOG.info("end open.");
}

Expand All @@ -125,8 +124,7 @@ public Collection<RowData> lookup(RowData keyRow) {
InternalRow row = table.lookup(flussKeyRow).get().getRow();
if (row != null) {
RowData flinkRow =
flussRowToFlinkRowConverter.toFlinkRowData(
ProjectedRow.from(projection).replaceRow(row));
flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row));
if (remainingFilter == null || remainingFilter.isMatch(flinkRow)) {
return Collections.singletonList(flinkRow);
} else {
Expand All @@ -153,6 +151,13 @@ public Collection<RowData> lookup(RowData keyRow) {
return Collections.emptyList();
}

private InternalRow maybeProject(InternalRow row) {
if (projectedRow == null) {
return row;
}
return projectedRow.replaceRow(row);
}

@Override
public void close() throws Exception {
LOG.info("start close ...");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.flink.table.data.RowData.FieldGetter;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.Preconditions;

import javax.annotation.Nullable;

Expand All @@ -35,6 +34,7 @@
import java.util.stream.Collectors;

import static com.alibaba.fluss.utils.Preconditions.checkState;
import static org.apache.flink.util.Preconditions.checkArgument;

/**
* A utility class to normalize the lookup key row to match the Fluss key fields order and drop the
Expand Down Expand Up @@ -141,10 +141,7 @@ public boolean fieldMatches(RowData result) {

/** Validate the lookup key indexes and primary keys, and create a {@link LookupNormalizer}. */
public static LookupNormalizer validateAndCreateLookupNormalizer(
int[][] lookupKeyIndexes,
int[] primaryKeys,
RowType schema,
@Nullable int[] projectedFields) {
int[][] lookupKeyIndexes, int[] primaryKeys, RowType schema) {
if (primaryKeys.length == 0) {
throw new UnsupportedOperationException(
"Fluss lookup function only support lookup table with primary key.");
Expand All @@ -160,15 +157,10 @@ public static LookupNormalizer validateAndCreateLookupNormalizer(
String[] lookupKeyNames = new String[lookupKeyIndexes.length];
for (int i = 0; i < lookupKeyNames.length; i++) {
int[] innerKeyArr = lookupKeyIndexes[i];
Preconditions.checkArgument(
innerKeyArr.length == 1, "Do not support nested lookup keys");
checkArgument(innerKeyArr.length == 1, "Do not support nested lookup keys");
// lookupKeyIndexes passed by Flink is key indexed after projection pushdown,
// we need to project on client side, so restore the lookup key indexes before pushdown
if (projectedFields != null) {
lookupKeys[i] = projectedFields[innerKeyArr[0]];
} else {
lookupKeys[i] = innerKeyArr[0];
}
// we do remaining condition filter on the projected row, so no remapping needed.
lookupKeys[i] = innerKeyArr[0];
lookupKeyNames[i] = columnNames[innerKeyArr[0]];
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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 com.alibaba.fluss.connector.flink.utils;

import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;

/** Utils for Flink classes. */
public class FlinkUtils {

/**
* Returns projected {@link RowType} by given projection indexes over original {@link RowType}.
*
* @param rowType the original row type
* @param projection the projection indexes
*/
public static RowType projectRowType(RowType rowType, int[] projection) {
LogicalType[] types = new LogicalType[projection.length];
String[] names = new String[projection.length];
for (int i = 0; i < projection.length; i++) {
types[i] = rowType.getTypeAt(projection[i]);
names[i] = rowType.getFieldNames().get(projection[i]);
}
return RowType.of(rowType.isNullable(), types, names);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -676,6 +676,17 @@ void testLookup1PkTableWith2Conditions(Caching caching, boolean async) throws Ex
CloseableIterator<Row> collected = tEnv.executeSql(dimJoinQuery).collect();
List<String> expected = Collections.singletonList("+I[3, name33, name3]");
assertResultsIgnoreOrder(collected, expected, true);

// project all columns from dim table
String dimJoinQuery2 =
String.format(
"SELECT a, b FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h"
+ " ON src.a = h.id AND h.name = 'name3'",
dim);

CloseableIterator<Row> collected2 = tEnv.executeSql(dimJoinQuery2).collect();
List<String> expected2 = Collections.singletonList("+I[3, name33]");
assertResultsIgnoreOrder(collected2, expected2, true);
}

/**
Expand Down

0 comments on commit 2700ef5

Please sign in to comment.