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

Put window aggregate logic into the various KsqlWindowExpression classes #377

Merged
merged 2 commits into from
Oct 17, 2017
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
@@ -0,0 +1,25 @@
/**
* Copyright 2017 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;

import org.apache.kafka.streams.kstream.Aggregator;
import org.apache.kafka.streams.kstream.Merger;

import io.confluent.ksql.GenericRow;

public interface UdafAggregator extends Aggregator<String, GenericRow, GenericRow> {
Merger<String, GenericRow> getMerger();
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,16 @@

import io.confluent.ksql.function.KsqlAggregateFunction;
import io.confluent.ksql.GenericRow;
import org.apache.kafka.streams.kstream.Aggregator;
import io.confluent.ksql.function.UdafAggregator;

import org.apache.kafka.streams.kstream.Merger;

import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;

public class KudafAggregator implements Aggregator<String, GenericRow, GenericRow> {
public class KudafAggregator implements UdafAggregator {

private Map<Integer, KsqlAggregateFunction> aggValToAggFunctionMap;
private Map<Integer, Integer> aggValToValColumnMap;
Expand Down Expand Up @@ -55,6 +56,7 @@ public GenericRow apply(String s, GenericRow rowValue, GenericRow aggRowValue) {
return aggRowValue;
}

@Override
public Merger<String, GenericRow> getMerger() {
return (key, aggRowOne, aggRowTwo) -> {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,10 @@

package io.confluent.ksql.structured;

import io.confluent.ksql.function.udaf.KudafAggregator;
import io.confluent.ksql.parser.tree.HoppingWindowExpression;
import io.confluent.ksql.parser.tree.SessionWindowExpression;
import io.confluent.ksql.parser.tree.TumblingWindowExpression;
import io.confluent.ksql.function.UdafAggregator;
import io.confluent.ksql.parser.tree.KsqlWindowExpression;
import io.confluent.ksql.parser.tree.WindowExpression;
import io.confluent.ksql.GenericRow;
import io.confluent.ksql.util.KsqlException;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.connect.data.Field;
Expand All @@ -31,10 +28,6 @@
import org.apache.kafka.streams.kstream.KGroupedStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.SessionWindows;
import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.state.SessionStore;
import org.apache.kafka.streams.state.WindowStore;

import java.util.List;
Expand All @@ -57,49 +50,26 @@ public class SchemaKGroupedStream {

@SuppressWarnings("unchecked")
public SchemaKTable aggregate(final Initializer initializer,
final KudafAggregator aggregator,
final UdafAggregator aggregator,
final WindowExpression windowExpression,
final Serde<GenericRow> topicValueSerDe,
final String storeName) {
boolean isWindowed = false;
KTable<Windowed<String>, GenericRow> aggKtable;
final KTable aggKtable;
if (windowExpression != null) {
isWindowed = true;
final Materialized<String, GenericRow, WindowStore<Bytes, byte[]>> materialized
final Materialized<String, GenericRow, ?> materialized
= Materialized.<String, GenericRow, WindowStore<Bytes, byte[]>>as(storeName)
.withValueSerde(topicValueSerDe);
if (windowExpression.getKsqlWindowExpression() instanceof TumblingWindowExpression) {
TumblingWindowExpression tumblingWindowExpression =
(TumblingWindowExpression) windowExpression.getKsqlWindowExpression();
aggKtable =
kgroupedStream.windowedBy(TimeWindows.of(tumblingWindowExpression.getSizeUnit().toMillis(tumblingWindowExpression.getSize())))
.aggregate(initializer, aggregator,
materialized);
} else if (windowExpression.getKsqlWindowExpression() instanceof HoppingWindowExpression) {
HoppingWindowExpression hoppingWindowExpression =
(HoppingWindowExpression) windowExpression.getKsqlWindowExpression();
aggKtable =
kgroupedStream.windowedBy(TimeWindows.of(
hoppingWindowExpression.getSizeUnit().toMillis(hoppingWindowExpression.getSize()))
.advanceBy(
hoppingWindowExpression.getAdvanceByUnit().toMillis(hoppingWindowExpression.getAdvanceBy())))
.aggregate(initializer, aggregator, materialized);
} else if (windowExpression.getKsqlWindowExpression() instanceof SessionWindowExpression) {
SessionWindowExpression sessionWindowExpression =
(SessionWindowExpression) windowExpression.getKsqlWindowExpression();
aggKtable =
kgroupedStream.windowedBy(SessionWindows.with(sessionWindowExpression.getSizeUnit().toMillis(sessionWindowExpression.getGap())))
.aggregate(initializer, aggregator, aggregator.getMerger(),
Materialized.<String, GenericRow, SessionStore<Bytes, byte[]>>as(storeName).withValueSerde(topicValueSerDe));
} else {
throw new KsqlException("Could not set the window expression for aggregate.");
}

final KsqlWindowExpression ksqlWindowExpression = windowExpression.getKsqlWindowExpression();
aggKtable = ksqlWindowExpression.applyAggregate(kgroupedStream, initializer, aggregator, materialized);
} else {
aggKtable =
kgroupedStream.aggregate(initializer, aggregator, Materialized.with(null, topicValueSerDe));
aggKtable = kgroupedStream.aggregate(initializer, aggregator, Materialized.with(null, topicValueSerDe));
}
return new SchemaKTable(schema, aggKtable, keyField, sourceSchemaKStreams, isWindowed,
SchemaKStream.Type.AGGREGATE);
return new SchemaKTable(schema, aggKtable,
keyField,
sourceSchemaKStreams,
windowExpression != null,
SchemaKStream.Type.AGGREGATE);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.test.TestUtils;

import java.util.Collections;
import java.util.HashMap;
Expand Down Expand Up @@ -162,6 +164,7 @@ public void start() throws Exception {
configMap.put("commit.interval.ms", 0);
configMap.put("cache.max.bytes.buffering", 0);
configMap.put("auto.offset.reset", "earliest");
configMap.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());

this.ksqlConfig = new KsqlConfig(configMap);
this.topicClient = new KafkaTopicClientImpl(ksqlConfig.getKsqlAdminClientConfigProps());
Expand Down
5 changes: 5 additions & 0 deletions ksql-parser/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,11 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>

</dependencies>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,19 @@

package io.confluent.ksql.parser.tree;

import org.apache.kafka.streams.kstream.Initializer;
import org.apache.kafka.streams.kstream.KGroupedStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.TimeWindows;

import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeUnit;

import io.confluent.ksql.GenericRow;
import io.confluent.ksql.function.UdafAggregator;

public class HoppingWindowExpression extends KsqlWindowExpression {

private final long size;
Expand All @@ -29,42 +38,19 @@ public class HoppingWindowExpression extends KsqlWindowExpression {

public HoppingWindowExpression(long size, TimeUnit sizeUnit,
long advanceBy, TimeUnit advanceByUnit) {
this(Optional.empty(), "", size, sizeUnit, advanceBy, advanceByUnit);
}

public HoppingWindowExpression(NodeLocation location, String windowName, long size,
TimeUnit
sizeUnit,
long advanceBy, TimeUnit advanceByUnit) {
this(Optional.of(location), windowName, size, sizeUnit, advanceBy, advanceByUnit);
this(Optional.empty(), size, sizeUnit, advanceBy, advanceByUnit);
}

private HoppingWindowExpression(Optional<NodeLocation> location, String windowName, long size,
TimeUnit sizeUnit,
long advanceBy, TimeUnit advanceByUnit) {
private HoppingWindowExpression(Optional<NodeLocation> location, long size,
TimeUnit sizeUnit,
long advanceBy, TimeUnit advanceByUnit) {
super(location);
this.size = size;
this.sizeUnit = sizeUnit;
this.advanceBy = advanceBy;
this.advanceByUnit = advanceByUnit;
}

public long getSize() {
return size;
}

public TimeUnit getSizeUnit() {
return sizeUnit;
}

public long getAdvanceBy() {
return advanceBy;
}

public TimeUnit getAdvanceByUnit() {
return advanceByUnit;
}

@Override
public String toString() {
return " HOPPING ( SIZE " + size + " " + sizeUnit + " , ADVANCE BY "
Expand All @@ -90,4 +76,14 @@ public boolean equals(Object o) {
&& hoppingWindowExpression.advanceBy == advanceBy && hoppingWindowExpression
.advanceByUnit == advanceByUnit;
}

@SuppressWarnings("unchecked")
@Override
public KTable applyAggregate(KGroupedStream groupedStream, Initializer initializer, UdafAggregator aggregator, Materialized<String, GenericRow, ?> materialized) {
return groupedStream.windowedBy(TimeWindows.of(
sizeUnit.toMillis(size))
.advanceBy(
advanceByUnit.toMillis(advanceBy)))
.aggregate(initializer, aggregator, materialized);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,24 @@

package io.confluent.ksql.parser.tree;

import org.apache.kafka.streams.kstream.Initializer;
import org.apache.kafka.streams.kstream.KGroupedStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized;

import java.util.Optional;

import io.confluent.ksql.GenericRow;
import io.confluent.ksql.function.UdafAggregator;

public abstract class KsqlWindowExpression extends Node {

protected KsqlWindowExpression(Optional<NodeLocation> location) {
super(location);
}

public abstract KTable applyAggregate(final KGroupedStream groupedStream,
final Initializer initializer,
final UdafAggregator aggregator,
final Materialized<String, GenericRow, ?> materialized);
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,39 +16,35 @@

package io.confluent.ksql.parser.tree;

import org.apache.kafka.streams.kstream.Initializer;
import org.apache.kafka.streams.kstream.KGroupedStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.SessionWindows;

import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeUnit;

import io.confluent.ksql.GenericRow;
import io.confluent.ksql.function.UdafAggregator;

public class SessionWindowExpression extends KsqlWindowExpression {

private final long gap;
private final TimeUnit sizeUnit;

public SessionWindowExpression(long gap, TimeUnit sizeUnit) {
this(Optional.empty(), "", gap, sizeUnit);
}

public SessionWindowExpression(NodeLocation location, String windowName,
long gap, TimeUnit sizeUnit) {
this(Optional.of(location), windowName, gap, sizeUnit);
this(Optional.empty(), gap, sizeUnit);
}

private SessionWindowExpression(Optional<NodeLocation> location, String windowName, long gap,
private SessionWindowExpression(Optional<NodeLocation> location, long gap,
TimeUnit sizeUnit) {
super(location);
this.gap = gap;
this.sizeUnit = sizeUnit;
}

public long getGap() {
return gap;
}

public TimeUnit getSizeUnit() {
return sizeUnit;
}

@Override
public String toString() {
return " SESSION ( " + gap + " " + sizeUnit + " ) ";
Expand All @@ -70,4 +66,15 @@ public boolean equals(Object o) {
SessionWindowExpression sessionWindowExpression = (SessionWindowExpression) o;
return sessionWindowExpression.gap == gap && sessionWindowExpression.sizeUnit == sizeUnit;
}

@SuppressWarnings("unchecked")
@Override
public KTable applyAggregate(final KGroupedStream groupedStream,
final Initializer initializer,
final UdafAggregator aggregator,
final Materialized<String, GenericRow, ?> materialized) {
return groupedStream.windowedBy(SessionWindows.with(sizeUnit.toMillis(gap)))
.aggregate(initializer, aggregator, aggregator.getMerger(),
materialized);
}
}
Loading