Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix: fix repartition semantics #4816

Merged
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
@@ -1,8 +1,8 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"; you may not use
* this file except in compliance with the License. You may obtain a copy of the
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
Expand All @@ -15,12 +15,21 @@

package io.confluent.ksql.name;

import io.confluent.ksql.schema.ksql.Column;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import java.util.OptionalInt;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

public final class ColumnNames {

private static final String AGGREGATE_COLUMN_PREFIX = "KSQL_AGG_VARIABLE_";
private static final String GENERATED_ALIAS_PREFIX = "KSQL_COL_";
private static final String SYNTHESISED_COLUMN_PREFIX = "KSQL_SYNTH_";

private static final Pattern GENERATED_ALIAS_PATTERN = Pattern
.compile(GENERATED_ALIAS_PREFIX + "(\\d+)");

private ColumnNames() {
}

Expand All @@ -32,24 +41,24 @@ public static ColumnName aggregateColumn(final int idx) {
}

/**
* Where the user hasn't specified an alias for an expression in a SELECT we generate them
* using this method. This value is exposed to the user in the output schema
* Where the user hasn't specified an alias for an expression in a SELECT we generate them using
* this method. This value is exposed to the user in the output schema
*/
public static ColumnName generatedColumnAlias(final int idx) {
return ColumnName.of(GENERATED_ALIAS_PREFIX + idx);
}

/**
* Used to generate a column name in an intermediate schema, e.g. for a column to hold
* values of a table function. These are never exposed to the user
* Used to generate a column name in an intermediate schema, e.g. for a column to hold values of a
* table function. These are never exposed to the user
*/
public static ColumnName synthesisedSchemaColumn(final int idx) {
return ColumnName.of(SYNTHESISED_COLUMN_PREFIX + idx);
}

/**
* Used to generate a column alias for a join where the a column with this name exists
* in both of the sources.
* Used to generate a column alias for a join where the a column with this name exists in both of
* the sources.
*/
public static ColumnName generatedJoinColumnAlias(
final SourceName sourceName,
Expand All @@ -61,4 +70,42 @@ public static ColumnName generatedJoinColumnAlias(
public static boolean isAggregate(final ColumnName name) {
return name.text().startsWith(AGGREGATE_COLUMN_PREFIX);
}

/**
* Determines the next unique column alias.
*
* <p>Finds any existing {@code KSQL_COL_x} column names in the supplied {@code sourceSchema} to
* ensure the returned generated column name is unique.
*
* @param sourceSchema the source schema.
* @return a column name in the form {@code KSQL_COL_x} which does not clash with source schema.
*/
public static ColumnName nextGeneratedColumnAlias(final LogicalSchema sourceSchema) {
big-andy-coates marked this conversation as resolved.
Show resolved Hide resolved
final int maxExistingIdx = maxGeneratedAliasIndex(sourceSchema);
return generatedColumnAlias(maxExistingIdx + 1);
}

/**
* Determines the highest index of generated column names like {@code KSQL_COL_x} in the supplied
* {@code sourceSchema}.
*
* @param sourceSchema the schema.
* @return the highest index or {@code -1}
*/
private static int maxGeneratedAliasIndex(final LogicalSchema sourceSchema) {
return sourceSchema.columns().stream()
.map(Column::name)
.map(ColumnNames::extractGeneratedAliasIndex)
.filter(OptionalInt::isPresent)
.mapToInt(OptionalInt::getAsInt)
.max()
.orElse(-1);
}

private static OptionalInt extractGeneratedAliasIndex(final ColumnName columnName) {
final Matcher matcher = GENERATED_ALIAS_PATTERN.matcher(columnName.text());
return matcher.matches()
? OptionalInt.of(Integer.parseInt(matcher.group(1)))
: OptionalInt.empty();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.name;

import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.is;

import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.schema.ksql.types.SqlTypes;
import org.junit.Test;

public class ColumnNamesTest {

@Test
public void shouldGenerateUniqueAliasesStartingAtZero() {
// Given:
final LogicalSchema schema = LogicalSchema.builder()
.build();

// When:
final ColumnName result = ColumnNames.nextGeneratedColumnAlias(schema);

// Then:
assertThat(result, is(ColumnName.of("KSQL_COL_0")));
}

@Test
public void shouldGenerateUniqueAliasesTakingAnyKeyColumnsIntoAccount() {
// Given:
final LogicalSchema schema = LogicalSchema.builder()
.keyColumn(ColumnName.of("Fred"), SqlTypes.STRING)
.keyColumn(ColumnNames.generatedColumnAlias(1), SqlTypes.STRING)
.keyColumn(ColumnName.of("George"), SqlTypes.STRING)
.build();

// When:
final ColumnName result = ColumnNames.nextGeneratedColumnAlias(schema);

// Then:
assertThat(result, is(ColumnName.of("KSQL_COL_2")));
}

@Test
public void shouldGenerateUniqueAliasesTakingAnyValueColumnsIntoAccount() {
// Given:
final LogicalSchema schema = LogicalSchema.builder()
.valueColumn(ColumnName.of("Fred"), SqlTypes.STRING)
.valueColumn(ColumnNames.generatedColumnAlias(1), SqlTypes.STRING)
.valueColumn(ColumnName.of("George"), SqlTypes.STRING)
.build();

// When:
final ColumnName result = ColumnNames.nextGeneratedColumnAlias(schema);

// Then:
assertThat(result, is(ColumnName.of("KSQL_COL_2")));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp;
import io.confluent.ksql.execution.expression.tree.VisitParentExpressionVisitor;
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.execution.streams.PartitionByParamsFactory;
import io.confluent.ksql.execution.streams.timestamp.TimestampExtractionPolicyFactory;
import io.confluent.ksql.execution.timestamp.TimestampColumn;
import io.confluent.ksql.execution.util.ExpressionTypeManager;
Expand Down Expand Up @@ -552,16 +553,25 @@ private LogicalSchema buildRepartitionedSchema(
return sourceSchema;
}

final ExpressionTypeManager typeManager =
new ExpressionTypeManager(sourceSchema, functionRegistry);
if (!ksqlConfig.getBoolean(KsqlConfig.KSQL_ANY_KEY_NAME_ENABLED)) {
final ExpressionTypeManager expressionTypeManager =
new ExpressionTypeManager(sourceSchema, functionRegistry);

final SqlType keyType = typeManager.getExpressionSqlType(partitionBy);
final SqlType keyType = expressionTypeManager
.getExpressionSqlType(partitionBy);

return LogicalSchema.builder()
.withRowTime()
.keyColumn(SchemaUtil.ROWKEY_NAME, keyType)
.valueColumns(sourceSchema.value())
.build();
return LogicalSchema.builder()
.withRowTime()
.keyColumn(SchemaUtil.ROWKEY_NAME, keyType)
.valueColumns(sourceSchema.value())
.build();
}

return PartitionByParamsFactory.buildSchema(
sourceSchema,
partitionBy,
functionRegistry
);
}

private static boolean exactlyMatchesKeyColumns(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,15 @@
import io.confluent.ksql.execution.builder.KsqlQueryBuilder;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.metastore.model.KeyField;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.name.SourceName;
import io.confluent.ksql.schema.ksql.Column;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.services.KafkaTopicClient;
import io.confluent.ksql.structured.SchemaKStream;
import java.util.List;
import java.util.Optional;
import java.util.stream.Stream;

@Immutable
public class RepartitionNode extends PlanNode {
Expand Down Expand Up @@ -78,4 +83,19 @@ public Expression getPartitionBy() {
public <C, R> R accept(final PlanVisitor<C, R> visitor, final C context) {
return visitor.visitRepartition(this, context);
}

@Override
public Stream<ColumnName> resolveSelectStar(
final Optional<SourceName> sourceName,
final boolean valueOnly
) {
final boolean sourceNameMatches = !sourceName.isPresent() || sourceName.equals(getSourceName());
if (sourceNameMatches && valueOnly) {
// Override set of value columns to take into account the repartition:
return getSchema().withoutMetaAndKeyColsInValue().value().stream().map(Column::name);
}

// Set of all columns not changed by a repartition:
return super.resolveSelectStar(sourceName, valueOnly);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import io.confluent.ksql.execution.plan.StreamGroupBy;
import io.confluent.ksql.execution.plan.StreamGroupByKey;
import io.confluent.ksql.execution.plan.StreamSelect;
import io.confluent.ksql.execution.plan.StreamSelectKey;
import io.confluent.ksql.execution.plan.StreamSink;
import io.confluent.ksql.execution.plan.StreamStreamJoin;
import io.confluent.ksql.execution.plan.StreamTableJoin;
Expand Down Expand Up @@ -191,8 +190,6 @@ KeyField findKeyField(final List<SelectExpression> selectExpressions) {
final Optional<ColumnName> filtered = found
// System columns can not be key fields:
.filter(f -> !SchemaUtil.isSystemColumn(f.name()))
// Key columns can not be key fields:
.filter(f -> !schema.isKeyColumn(f.name()))
.map(Column::name);

return KeyField.of(filtered);
Expand Down Expand Up @@ -338,11 +335,10 @@ public SchemaKStream<Struct> selectKey(
+ "See https://github.com/confluentinc/ksql/issues/4385.");
}

final StreamSelectKey step = ExecutionStepFactory.streamSelectKey(
contextStacker,
sourceStep,
keyExpression
);
final ExecutionStep<KStreamHolder<Struct>> step = ksqlConfig
.getBoolean(KsqlConfig.KSQL_ANY_KEY_NAME_ENABLED)
? ExecutionStepFactory.streamSelectKey(contextStacker, sourceStep, keyExpression)
: ExecutionStepFactory.streamSelectKeyV1(contextStacker, sourceStep, keyExpression);

return new SchemaKStream<>(
step,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import io.confluent.ksql.execution.plan.StreamGroupBy;
import io.confluent.ksql.execution.plan.StreamSelect;
import io.confluent.ksql.execution.plan.StreamSelectKey;
import io.confluent.ksql.execution.plan.StreamSelectKeyV1;
import io.confluent.ksql.execution.plan.StreamSink;
import io.confluent.ksql.execution.plan.StreamSource;
import io.confluent.ksql.execution.plan.StreamStreamJoin;
Expand Down Expand Up @@ -67,6 +68,7 @@ public class PlanSummary {
.put(StreamFlatMap.class, "FLAT_MAP")
.put(StreamGroupBy.class, "GROUP_BY")
.put(StreamSelect.class, "PROJECT")
.put(StreamSelectKeyV1.class, "REKEY")
.put(StreamSelectKey.class, "REKEY")
.put(StreamSink.class, "SINK")
.put(StreamSource.class, "SOURCE")
Expand Down
Loading