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

Add WindowStart and WindowEnd UDFs #1993

Merged
merged 18 commits into from
Oct 24, 2018
Merged
Show file tree
Hide file tree
Changes from 8 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 @@ -28,9 +28,9 @@

public class AggregateAnalyzer extends DefaultTraversalVisitor<Node, AnalysisContext> {

private AggregateAnalysis aggregateAnalysis;
private Analysis analysis;
private FunctionRegistry functionRegistry;
private final AggregateAnalysis aggregateAnalysis;
private final Analysis analysis;
private final FunctionRegistry functionRegistry;

private boolean hasAggregateFunction = false;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* Copyright 2018 Confluent Inc.
*
* Licensed 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 io.confluent.ksql.function.udaf.placeholder;

import io.confluent.ksql.function.udaf.TableUdaf;

/**
* A no-op {@link TableUdaf} that is used as a placeholder for some later hardcoded computation.
*/
public final class PlaceholderTableUdaf implements TableUdaf<Long, Long> {

public static final PlaceholderTableUdaf INSTANCE = new PlaceholderTableUdaf();

private PlaceholderTableUdaf(){
}

@Override
public Long undo(final Long valueToUndo, final Long aggregateValue) {
return null;
}

@Override
public Long initialize() {
return null;
}

@Override
public Long aggregate(final Long value, final Long aggregate) {
return null;
}

@Override
public Long merge(final Long aggOne, final Long aggTwo) {
return null;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Copyright 2018 Confluent Inc.
*
* Licensed 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 io.confluent.ksql.function.udaf.window;

import io.confluent.ksql.function.udaf.TableUdaf;
import io.confluent.ksql.function.udaf.UdafDescription;
import io.confluent.ksql.function.udaf.UdafFactory;
import io.confluent.ksql.function.udaf.placeholder.PlaceholderTableUdaf;

/**
* A placeholder KUDAF for extracting window end times.
*
* <p>The KUDAF itself does nothing. It's just a placeholder.
*
* @see WindowSelectMapper
*/
@SuppressWarnings("WeakerAccess") // Invoked via reflection.
@UdafDescription(name = "WindowEnd", author = "Confluent",
description = "Returns the window end time, in milliseconds, for the given record. "
+ "If the given record is not part of a window the function will return NULL.")
public final class WindowEndKudaf {

private WindowEndKudaf() {
}

static String getFunctionName() {
return "WindowEnd";
}

@UdafFactory(description = "Extracts the window end time")
public static TableUdaf<Long, Long> createWindowEnd() {
return PlaceholderTableUdaf.INSTANCE;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Copyright 2018 Confluent Inc.
*
* Licensed 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 io.confluent.ksql.function.udaf.window;

import com.google.common.collect.ImmutableMap;
import io.confluent.ksql.GenericRow;
import io.confluent.ksql.function.KsqlAggregateFunction;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.kafka.streams.kstream.ValueMapperWithKey;
import org.apache.kafka.streams.kstream.Window;
import org.apache.kafka.streams.kstream.Windowed;

/**
* Used to handle the special cased {@link WindowStartKudaf} and {@link WindowEndKudaf}.
*/
public final class WindowSelectMapper
implements ValueMapperWithKey<Windowed<?>, GenericRow, GenericRow> {

private static final Map<String, Type> WINDOW_FUNCTION_NAMES = ImmutableMap.of(
WindowStartKudaf.getFunctionName().toUpperCase(), Type.StartTime,
WindowEndKudaf.getFunctionName().toUpperCase(), Type.EndTime
);

private final Map<Integer, Type> windowSelects;

public WindowSelectMapper(
final Map<Integer, KsqlAggregateFunction> aggFunctionsByIndex) {
this.windowSelects = aggFunctionsByIndex.entrySet().stream()
.filter(e ->
WINDOW_FUNCTION_NAMES.containsKey(e.getValue().getFunctionName().toUpperCase()))
.collect(Collectors.toMap(
Map.Entry::getKey,
e -> WINDOW_FUNCTION_NAMES.get(e.getValue().getFunctionName().toUpperCase())));
}

public boolean hasSelects() {
return !windowSelects.isEmpty();
}

@Override
public GenericRow apply(final Windowed<?> readOnlyKey, final GenericRow row) {
final Window window = readOnlyKey.window();

windowSelects.forEach((index, type) ->
row.getColumns().set(index, type.mapper.apply(window)));

return row;
}

private enum Type {
StartTime(Window::start), EndTime(Window::end);

private final Function<Window, Object> mapper;

Type(final Function<Window, Object> mapper) {
this.mapper = mapper;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Copyright 2018 Confluent Inc.
*
* Licensed 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 io.confluent.ksql.function.udaf.window;

import io.confluent.ksql.function.udaf.TableUdaf;
import io.confluent.ksql.function.udaf.UdafDescription;
import io.confluent.ksql.function.udaf.UdafFactory;
import io.confluent.ksql.function.udaf.placeholder.PlaceholderTableUdaf;

/**
* A placeholder KUDAF for extracting window start times.
*
* <p>The KUDAF itself does nothing. It's just a placeholder.
*
* @see WindowSelectMapper
*/
@SuppressWarnings("WeakerAccess") // Invoked via reflection.
@UdafDescription(name = "WindowStart", author = "Confluent",
description = "Returns the window start time, in milliseconds, for the given record. "
+ "If the given record is not part of a window the function will return NULL.")
public final class WindowStartKudaf {

private WindowStartKudaf() {
}

static String getFunctionName() {
return "WindowStart";
}

@UdafFactory(description = "Extracts the window start time")
public static TableUdaf<Long, Long> createWindowStart() {
return PlaceholderTableUdaf.INSTANCE;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@
import io.confluent.ksql.util.KafkaTopicClient;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.Pair;
import io.confluent.ksql.util.SchemaUtil;
import io.confluent.ksql.util.SelectExpression;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
Expand All @@ -66,12 +66,9 @@ public class AggregateNode extends PlanNode {
private final List<Expression> groupByExpressions;
private final WindowExpression windowExpression;
private final List<Expression> aggregateFunctionArguments;

private final List<FunctionCall> functionList;
private final List<Expression> requiredColumnList;

private final List<Expression> finalSelectExpressions;

private final Expression havingExpressions;

@JsonCreator
Expand Down Expand Up @@ -139,8 +136,8 @@ public List<Expression> getRequiredColumnList() {
return requiredColumnList;
}

private List<Pair<String, Expression>> getFinalSelectExpressions() {
final List<Pair<String, Expression>> finalSelectExpressionList = new ArrayList<>();
private List<SelectExpression> getFinalSelectExpressions() {
final List<SelectExpression> finalSelectExpressionList = new ArrayList<>();
if (finalSelectExpressions.size() != schema.fields().size()) {
throw new RuntimeException(
"Incompatible aggregate schema, field count must match, "
Expand All @@ -150,7 +147,7 @@ private List<Pair<String, Expression>> getFinalSelectExpressions() {
+ schema.fields().size());
}
for (int i = 0; i < finalSelectExpressions.size(); i++) {
finalSelectExpressionList.add(new Pair<>(
finalSelectExpressionList.add(SelectExpression.of(
schema.fields().get(i).name(),
finalSelectExpressions.get(i)
));
Expand Down Expand Up @@ -232,17 +229,13 @@ public SchemaKStream buildStream(
);

final KudafInitializer initializer = new KudafInitializer(aggValToValColumnMap.size());

final Map<Integer, KsqlAggregateFunction> aggValToFunctionMap = createAggValToFunctionMap(
aggregateArgExpanded, aggregateSchema, initializer, aggValToValColumnMap.size(),
functionRegistry, internalSchema);

final SchemaKTable schemaKTable = schemaKGroupedStream.aggregate(
initializer,
createAggValToFunctionMap(
aggregateArgExpanded,
aggregateSchema,
initializer,
aggValToValColumnMap.size(),
functionRegistry,
internalSchema),
aggValToValColumnMap,
getWindowExpression(),
initializer, aggValToFunctionMap, aggValToValColumnMap, getWindowExpression(),
aggValueGenericRowSerde);

SchemaKTable result = new SchemaKTable(
Expand Down Expand Up @@ -368,7 +361,7 @@ private Schema buildAggregateSchema(
}

private static class InternalSchema {
private final List<Pair<String, Expression>> aggArgExpansionList = new ArrayList<>();
private final List<SelectExpression> aggArgExpansionList = new ArrayList<>();
private final Map<String, Integer> internalNameToIndexMap = new HashMap<>();
private final Map<String, String> expressionToInternalColumnNameMap = new HashMap<>();

Expand All @@ -389,14 +382,13 @@ private void collectAggregateArgExpressions(
final String internalColumnName = INTERNAL_COLUMN_NAME_PREFIX
+ aggArgExpansionList.size();
internalNameToIndexMap.put(internalColumnName, aggArgExpansionList.size());
aggArgExpansionList.add(new Pair<>(internalColumnName, expression));
if (!expressionToInternalColumnNameMap.containsKey(expression.toString())) {
expressionToInternalColumnNameMap.put(expression.toString(), internalColumnName);
}
aggArgExpansionList.add(SelectExpression.of(internalColumnName, expression));
expressionToInternalColumnNameMap
.putIfAbsent(expression.toString(), internalColumnName);
});
}

List<Expression> getInternalExpressionList(final List<Expression> expressionList) {
private List<Expression> getInternalExpressionList(final List<Expression> expressionList) {
return expressionList.stream()
.map(argExpression -> argExpression instanceof Literal
? argExpression
Expand All @@ -406,27 +398,31 @@ List<Expression> getInternalExpressionList(final List<Expression> expressionList
.collect(Collectors.toList());
}

List<Pair<String, Expression>> updateFinalSelectExpressions(
final List<Pair<String, Expression>> finalSelectExpressions) {
private List<SelectExpression> updateFinalSelectExpressions(
final List<SelectExpression> finalSelectExpressions
) {
return finalSelectExpressions.stream()
.map(finalSelectExpression ->
expressionToInternalColumnNameMap
.containsKey(finalSelectExpression.getRight().toString())
? new Pair<>(finalSelectExpression.getLeft(),
(Expression)
new QualifiedNameReference(
QualifiedName.of(
expressionToInternalColumnNameMap
.get(finalSelectExpression.getRight().toString()))))
: new Pair<>(finalSelectExpression.getLeft(), finalSelectExpression.getRight()))
.map(finalSelectExpression -> {
final String internal = expressionToInternalColumnNameMap
.get(finalSelectExpression.getExpression().toString());

if (internal == null) {
return finalSelectExpression;
}

return SelectExpression.of(
finalSelectExpression.getName(),
new QualifiedNameReference(QualifiedName.of(internal)));

})
.collect(Collectors.toList());
}

String getInternalColumnForExpression(final Expression expression) {
return expressionToInternalColumnNameMap.get(expression.toString());
}

List<Pair<String, Expression>> getAggArgExpansionList() {
List<SelectExpression> getAggArgExpansionList() {
return aggArgExpansionList;
}

Expand Down
Loading