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

MINOR: replace window-unit with TimeUnit #346

Merged
merged 1 commit into from
Oct 10, 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
Expand Up @@ -41,7 +41,6 @@ public class SchemaKGroupedStream {
private final Schema schema;
private final KGroupedStream kgroupedStream;
private final Field keyField;
private final GenericRowValueTypeEnforcer genericRowValueTypeEnforcer;
private final List<SchemaKStream> sourceSchemaKStreams;

public SchemaKGroupedStream(final Schema schema, final KGroupedStream kgroupedStream,
Expand All @@ -50,7 +49,6 @@ public SchemaKGroupedStream(final Schema schema, final KGroupedStream kgroupedSt
this.schema = schema;
this.kgroupedStream = kgroupedStream;
this.keyField = keyField;
this.genericRowValueTypeEnforcer = new GenericRowValueTypeEnforcer(schema);
this.sourceSchemaKStreams = sourceSchemaKStreams;
}

Expand All @@ -69,11 +67,7 @@ public SchemaKTable aggregate(final Initializer initializer,
aggKtable =
kgroupedStream
.aggregate(initializer, aggregator,
TimeWindows.of(
getWindowUnitInMillisecond(
tumblingWindowExpression.getSize(),
tumblingWindowExpression
.getSizeUnit())),
TimeWindows.of(tumblingWindowExpression.getSizeUnit().toMillis(tumblingWindowExpression.getSize())),
topicValueSerDe,
storeName);
} else if (windowExpression.getKsqlWindowExpression() instanceof HoppingWindowExpression) {
Expand All @@ -83,11 +77,9 @@ public SchemaKTable aggregate(final Initializer initializer,
kgroupedStream
.aggregate(initializer, aggregator,
TimeWindows.of(
getWindowUnitInMillisecond(hoppingWindowExpression.getSize(),
hoppingWindowExpression.getSizeUnit()))
.advanceBy(getWindowUnitInMillisecond(
hoppingWindowExpression.getAdvanceBy(),
hoppingWindowExpression.getAdvanceByUnit())),
hoppingWindowExpression.getSizeUnit().toMillis(hoppingWindowExpression.getSize()))
.advanceBy(
hoppingWindowExpression.getAdvanceByUnit().toMillis(hoppingWindowExpression.getAdvanceBy())),
topicValueSerDe, storeName);
} else if (windowExpression.getKsqlWindowExpression() instanceof SessionWindowExpression) {
SessionWindowExpression sessionWindowExpression =
Expand All @@ -96,11 +88,7 @@ public SchemaKTable aggregate(final Initializer initializer,
kgroupedStream
.aggregate(initializer, aggregator,
aggregator.getMerger(),
SessionWindows.with(
getWindowUnitInMillisecond(
sessionWindowExpression.getGap(),
sessionWindowExpression
.getSizeUnit())),
SessionWindows.with(sessionWindowExpression.getSizeUnit().toMillis(sessionWindowExpression.getGap())),
topicValueSerDe,
storeName);
} else {
Expand All @@ -114,22 +102,4 @@ public SchemaKTable aggregate(final Initializer initializer,
SchemaKStream.Type.AGGREGATE);
}

private long getWindowUnitInMillisecond(long value, WindowExpression.WindowUnit windowUnit) {

switch (windowUnit) {
case DAY:
return value * 24 * 60 * 60 * 1000;
case HOUR:
return value * 60 * 60 * 1000;
case MINUTE:
return value * 60 * 1000;
case SECOND:
return value * 1000;
case MILLISECOND:
return value;
default:
return -1;
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ public void testSimpleAggregateLogicalPlan() throws Exception {
Assert.assertTrue(aggregateNode.getFunctionList().size() == 2);
Assert.assertTrue(aggregateNode.getFunctionList().get(0).getName().getSuffix()
.equalsIgnoreCase("sum"));
Assert.assertTrue(aggregateNode.getWindowExpression().getKsqlWindowExpression().toString().equalsIgnoreCase(" TUMBLING ( SIZE 2 SECOND ) "));
Assert.assertTrue(aggregateNode.getWindowExpression().getKsqlWindowExpression().toString().equalsIgnoreCase(" TUMBLING ( SIZE 2 SECONDS ) "));
Assert.assertTrue(aggregateNode.getGroupByExpressions().size() == 1);
Assert.assertTrue(aggregateNode.getGroupByExpressions().get(0).toString().equalsIgnoreCase("TEST1.COL0"));
Assert.assertTrue(aggregateNode.getRequiredColumnList().size() == 2);
Expand All @@ -162,7 +162,7 @@ public void testComplexAggregateLogicalPlan() throws Exception {
Assert.assertTrue(aggregateNode.getFunctionList().size() == 2);
Assert.assertTrue(aggregateNode.getFunctionList().get(0).getName().getSuffix()
.equalsIgnoreCase("sum"));
Assert.assertTrue(aggregateNode.getWindowExpression().getKsqlWindowExpression().toString().equalsIgnoreCase(" HOPPING ( SIZE 2 SECOND , ADVANCE BY 1 SECOND ) "));
Assert.assertTrue(aggregateNode.getWindowExpression().getKsqlWindowExpression().toString().equalsIgnoreCase(" HOPPING ( SIZE 2 SECONDS , ADVANCE BY 1 SECONDS ) "));
Assert.assertTrue(aggregateNode.getGroupByExpressions().size() == 1);
Assert.assertTrue(aggregateNode.getGroupByExpressions().get(0).toString().equalsIgnoreCase("TEST1.COL0"));
Assert.assertTrue(aggregateNode.getRequiredColumnList().size() == 2);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,29 +18,30 @@

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

public class HoppingWindowExpression extends KsqlWindowExpression {

private final long size;
private final WindowExpression.WindowUnit sizeUnit;
private final TimeUnit sizeUnit;
private final long advanceBy;
private final WindowExpression.WindowUnit advanceByUnit;
private final TimeUnit advanceByUnit;

public HoppingWindowExpression(long size, WindowExpression.WindowUnit sizeUnit,
long advanceBy, WindowExpression.WindowUnit advanceByUnit) {
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,
WindowExpression.WindowUnit
TimeUnit
sizeUnit,
long advanceBy, WindowExpression.WindowUnit advanceByUnit) {
long advanceBy, TimeUnit advanceByUnit) {
this(Optional.of(location), windowName, size, sizeUnit, advanceBy, advanceByUnit);
}

private HoppingWindowExpression(Optional<NodeLocation> location, String windowName, long size,
WindowExpression.WindowUnit sizeUnit,
long advanceBy, WindowExpression.WindowUnit advanceByUnit) {
TimeUnit sizeUnit,
long advanceBy, TimeUnit advanceByUnit) {
super(location);
this.size = size;
this.sizeUnit = sizeUnit;
Expand All @@ -52,15 +53,15 @@ public long getSize() {
return size;
}

public WindowExpression.WindowUnit getSizeUnit() {
public TimeUnit getSizeUnit() {
return sizeUnit;
}

public long getAdvanceBy() {
return advanceBy;
}

public WindowExpression.WindowUnit getAdvanceByUnit() {
public TimeUnit getAdvanceByUnit() {
return advanceByUnit;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,23 +18,24 @@

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

public class SessionWindowExpression extends KsqlWindowExpression {

private final long gap;
private final WindowExpression.WindowUnit sizeUnit;
private final TimeUnit sizeUnit;

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

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

private SessionWindowExpression(Optional<NodeLocation> location, String windowName, long gap,
WindowExpression.WindowUnit sizeUnit) {
TimeUnit sizeUnit) {
super(location);
this.gap = gap;
this.sizeUnit = sizeUnit;
Expand All @@ -44,7 +45,7 @@ public long getGap() {
return gap;
}

public WindowExpression.WindowUnit getSizeUnit() {
public TimeUnit getSizeUnit() {
return sizeUnit;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,23 +18,24 @@

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

public class TumblingWindowExpression extends KsqlWindowExpression {

private final long size;
private final WindowExpression.WindowUnit sizeUnit;
private final TimeUnit sizeUnit;

public TumblingWindowExpression(long size, WindowExpression.WindowUnit sizeUnit) {
public TumblingWindowExpression(long size, TimeUnit sizeUnit) {
this(Optional.empty(), "", size, sizeUnit);
}

public TumblingWindowExpression(NodeLocation location, String windowName,
long size, WindowExpression.WindowUnit sizeUnit) {
long size, TimeUnit sizeUnit) {
this(Optional.of(location), windowName, size, sizeUnit);
}

private TumblingWindowExpression(Optional<NodeLocation> location, String windowName, long size,
WindowExpression.WindowUnit sizeUnit) {
TimeUnit sizeUnit) {
super(location);
this.size = size;
this.sizeUnit = sizeUnit;
Expand All @@ -44,7 +45,7 @@ public long getSize() {
return size;
}

public WindowExpression.WindowUnit getSizeUnit() {
public TimeUnit getSizeUnit() {
return sizeUnit;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

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

public class WindowExpression extends Node {

Expand Down Expand Up @@ -65,25 +66,24 @@ public String toString() {
return " WINDOW " + windowName + " " + ksqlWindowExpression.toString();
}

public static enum WindowUnit { DAY, HOUR, MINUTE, SECOND, MILLISECOND }

public static WindowUnit getWindowUnit(String windowUnitString) {
public static TimeUnit getWindowUnit(String windowUnitString) {
switch (windowUnitString) {
case "DAY":
case "DAYS":
return WindowUnit.DAY;
return TimeUnit.DAYS;
case "HOUR":
case "HOURS":
return WindowUnit.HOUR;
return TimeUnit.HOURS;
case "MINUTE":
case "MINUTES":
return WindowUnit.MINUTE;
return TimeUnit.MINUTES;
case "SECOND":
case "SECONDS":
return WindowUnit.SECOND;
return TimeUnit.SECONDS;
case "MILLISECOND":
case "MILLISECONDS":
return WindowUnit.MILLISECOND;
return TimeUnit.MILLISECONDS;
default:
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,10 @@

import java.util.List;

import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsInstanceOf.instanceOf;

public class KsqlParserTest {

private static final KsqlParser KSQL_PARSER = new KsqlParser();
Expand Down Expand Up @@ -464,11 +468,11 @@ public void testSelectTumblingWindow() throws Exception {
Assert.assertTrue("testSelectTumblingWindow failed.", querySpecification
.getWindowExpression().isPresent());
Assert.assertTrue("testSelectTumblingWindow failed.", querySpecification
.getWindowExpression().get().toString().equalsIgnoreCase(" WINDOW STREAMWINDOW TUMBLING ( SIZE 30 SECOND ) "));
.getWindowExpression().get().toString().equalsIgnoreCase(" WINDOW STREAMWINDOW TUMBLING ( SIZE 30 SECONDS ) "));
}

@Test
public void testSelectHuppingWindow() throws Exception {
public void testSelectHoppingWindow() throws Exception {

String
queryStr =
Expand All @@ -478,18 +482,17 @@ public void testSelectHuppingWindow() throws Exception {
+ "orderunits"
+ " > 5 group by itemid;";
Statement statement = KSQL_PARSER.buildAst(queryStr, metaStore).get(0);
Assert.assertTrue("testSelectTumblingWindow failed.", statement instanceof Query);
assertThat(statement, instanceOf(Query.class));
Query query = (Query) statement;
Assert.assertTrue("testSelectTumblingWindow failed.", query.getQueryBody() instanceof QuerySpecification);
assertThat(query.getQueryBody(), instanceOf(QuerySpecification.class));
QuerySpecification querySpecification = (QuerySpecification) query.getQueryBody();
Assert.assertTrue("testCreateTable failed.", querySpecification.getSelect().getSelectItems
().size() == 2);
Assert.assertTrue("testSelectTumblingWindow failed.", querySpecification.getWhere().get().toString().equalsIgnoreCase("(ORDERS.ORDERUNITS > 5)"));
Assert.assertTrue("testSelectTumblingWindow failed.", ((AliasedRelation)querySpecification.getFrom().get()).getAlias().equalsIgnoreCase("ORDERS"));
Assert.assertTrue("testSelectTumblingWindow failed.", querySpecification
assertThat(querySpecification.getSelect().getSelectItems().size(), equalTo(2));
assertThat(querySpecification.getWhere().get().toString(), equalTo("(ORDERS.ORDERUNITS > 5)"));
assertThat(((AliasedRelation)querySpecification.getFrom().get()).getAlias().toUpperCase(), equalTo("ORDERS"));
Assert.assertTrue("window expression isn't present", querySpecification
.getWindowExpression().isPresent());
Assert.assertTrue("testSelectTumblingWindow failed.", querySpecification
.getWindowExpression().get().toString().equalsIgnoreCase(" WINDOW STREAMWINDOW HOPPING ( SIZE 30 SECOND , ADVANCE BY 5 SECOND ) "));
assertThat(querySpecification.getWindowExpression().get().toString().toUpperCase(),
equalTo(" WINDOW STREAMWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 5 SECONDS ) "));
}

@Test
Expand All @@ -512,7 +515,7 @@ public void testSelectSessionWindow() throws Exception {
.getWindowExpression().isPresent());
Assert.assertTrue("testSelectSessionWindow failed.", querySpecification
.getWindowExpression().get().toString().equalsIgnoreCase(" WINDOW STREAMWINDOW SESSION "
+ "( 30 SECOND ) "));
+ "( 30 SECONDS ) "));
}

@Test
Expand Down