blob: bc07d618e2e3b5ab9b5f83dba1f4a6e2f419745e [file] [log] [blame]
/*
* 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 org.apache.flink.sql.tests;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.fs.bucketing.BasePathBucketer;
import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.java.StreamTableEnvironment;
import org.apache.flink.table.sources.DefinedFieldMapping;
import org.apache.flink.table.sources.DefinedRowtimeAttributes;
import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
import org.apache.flink.table.sources.StreamTableSource;
import org.apache.flink.table.sources.tsextractors.ExistingField;
import org.apache.flink.table.sources.wmstrategies.BoundedOutOfOrderTimestamps;
import org.apache.flink.types.Row;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
* End-to-end test for Stream SQL queries.
*
* <p>Includes the following SQL features:
* - OVER window aggregation
* - keyed and non-keyed GROUP BY TUMBLE aggregation
* - windowed INNER JOIN
* - TableSource with event-time attribute
*
* <p>The stream is bounded and will complete after about a minute.
* The result is always constant.
* The job is killed on the first attempt and restarted.
*
* <p>Parameters:
* -outputPath Sets the path to where the result data is written.
*/
public class StreamSQLTestProgram {
public static void main(String[] args) throws Exception {
ParameterTool params = ParameterTool.fromArgs(args);
String outputPath = params.getRequired("outputPath");
StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment();
sEnv.setRestartStrategy(RestartStrategies.fixedDelayRestart(
3,
Time.of(10, TimeUnit.SECONDS)
));
sEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
sEnv.enableCheckpointing(4000);
sEnv.getConfig().setAutoWatermarkInterval(1000);
StreamTableEnvironment tEnv = TableEnvironment.getTableEnvironment(sEnv);
tEnv.registerTableSource("table1", new GeneratorTableSource(10, 100, 60, 0));
tEnv.registerTableSource("table2", new GeneratorTableSource(5, 0.2f, 60, 5));
int overWindowSizeSeconds = 1;
int tumbleWindowSizeSeconds = 10;
String overQuery = String.format(
"SELECT " +
" key, " +
" rowtime, " +
" COUNT(*) OVER (PARTITION BY key ORDER BY rowtime RANGE BETWEEN INTERVAL '%d' SECOND PRECEDING AND CURRENT ROW) AS cnt " +
"FROM table1",
overWindowSizeSeconds);
String tumbleQuery = String.format(
"SELECT " +
" key, " +
" CASE SUM(cnt) / COUNT(*) WHEN 101 THEN 1 ELSE 99 END AS correct, " +
" TUMBLE_START(rowtime, INTERVAL '%d' SECOND) AS wStart, " +
" TUMBLE_ROWTIME(rowtime, INTERVAL '%d' SECOND) AS rowtime " +
"FROM (%s) " +
"WHERE rowtime > TIMESTAMP '1970-01-01 00:00:01' " +
"GROUP BY key, TUMBLE(rowtime, INTERVAL '%d' SECOND)",
tumbleWindowSizeSeconds,
tumbleWindowSizeSeconds,
overQuery,
tumbleWindowSizeSeconds);
String joinQuery = String.format(
"SELECT " +
" t1.key, " +
" t2.rowtime AS rowtime, " +
" t2.correct," +
" t2.wStart " +
"FROM table2 t1, (%s) t2 " +
"WHERE " +
" t1.key = t2.key AND " +
" t1.rowtime BETWEEN t2.rowtime AND t2.rowtime + INTERVAL '%d' SECOND",
tumbleQuery,
tumbleWindowSizeSeconds);
String finalAgg = String.format(
"SELECT " +
" SUM(correct) AS correct, " +
" TUMBLE_START(rowtime, INTERVAL '20' SECOND) AS rowtime " +
"FROM (%s) " +
"GROUP BY TUMBLE(rowtime, INTERVAL '20' SECOND)",
joinQuery);
// get Table for SQL query
Table result = tEnv.sqlQuery(finalAgg);
// convert Table into append-only DataStream
DataStream<Row> resultStream =
tEnv.toAppendStream(result, Types.ROW(Types.INT, Types.SQL_TIMESTAMP));
// define bucketing sink to emit the result
BucketingSink<Row> sink = new BucketingSink<Row>(outputPath)
.setBucketer(new BasePathBucketer<>());
resultStream
// inject a KillMapper that forwards all records but terminates the first execution attempt
.map(new KillMapper()).setParallelism(1)
// add sink function
.addSink(sink).setParallelism(1);
sEnv.execute();
}
/**
* TableSource for generated data.
*/
public static class GeneratorTableSource
implements StreamTableSource<Row>, DefinedRowtimeAttributes, DefinedFieldMapping {
private final int numKeys;
private final float recordsPerKeyAndSecond;
private final int durationSeconds;
private final int offsetSeconds;
public GeneratorTableSource(int numKeys, float recordsPerKeyAndSecond, int durationSeconds, int offsetSeconds) {
this.numKeys = numKeys;
this.recordsPerKeyAndSecond = recordsPerKeyAndSecond;
this.durationSeconds = durationSeconds;
this.offsetSeconds = offsetSeconds;
}
@Override
public DataStream<Row> getDataStream(StreamExecutionEnvironment execEnv) {
return execEnv.addSource(new Generator(numKeys, recordsPerKeyAndSecond, durationSeconds, offsetSeconds));
}
@Override
public TypeInformation<Row> getReturnType() {
return Types.ROW(Types.INT, Types.LONG, Types.STRING);
}
@Override
public TableSchema getTableSchema() {
return new TableSchema(
new String[] {"key", "rowtime", "payload"},
new TypeInformation[] {Types.INT, Types.SQL_TIMESTAMP, Types.STRING});
}
@Override
public String explainSource() {
return "GeneratorTableSource";
}
@Override
public List<RowtimeAttributeDescriptor> getRowtimeAttributeDescriptors() {
return Collections.singletonList(
new RowtimeAttributeDescriptor(
"rowtime",
new ExistingField("ts"),
new BoundedOutOfOrderTimestamps(100)));
}
@Override
public Map<String, String> getFieldMapping() {
Map<String, String> mapping = new HashMap<>();
mapping.put("key", "f0");
mapping.put("ts", "f1");
mapping.put("payload", "f2");
return mapping;
}
}
/**
* Data-generating source function.
*/
public static class Generator implements SourceFunction<Row>, ResultTypeQueryable<Row>, ListCheckpointed<Long> {
private final int numKeys;
private final int offsetSeconds;
private final int sleepMs;
private final int durationMs;
private long ms = 0;
public Generator(int numKeys, float rowsPerKeyAndSecond, int durationSeconds, int offsetSeconds) {
this.numKeys = numKeys;
this.durationMs = durationSeconds * 1000;
this.offsetSeconds = offsetSeconds;
this.sleepMs = (int) (1000 / rowsPerKeyAndSecond);
}
@Override
public void run(SourceContext<Row> ctx) throws Exception {
long offsetMS = offsetSeconds * 2000L;
while (ms < durationMs) {
synchronized (ctx.getCheckpointLock()) {
for (int i = 0; i < numKeys; i++) {
ctx.collect(Row.of(i, ms + offsetMS, "Some payload..."));
}
ms += sleepMs;
}
Thread.sleep(sleepMs);
}
}
@Override
public void cancel() { }
@Override
public TypeInformation<Row> getProducedType() {
return Types.ROW(Types.INT, Types.LONG, Types.STRING);
}
@Override
public List<Long> snapshotState(long checkpointId, long timestamp) {
return Collections.singletonList(ms);
}
@Override
public void restoreState(List<Long> state) {
for (Long l : state) {
ms += l;
}
}
}
/**
* Kills the first execution attempt of an application when it receives the second record.
*/
public static class KillMapper implements MapFunction<Row, Row>, ListCheckpointed<Integer>, ResultTypeQueryable {
// counts all processed records of all previous execution attempts
private int saveRecordCnt = 0;
// counts all processed records of this execution attempt
private int lostRecordCnt = 0;
@Override
public Row map(Row value) {
// the both counts are the same only in the first execution attempt
if (saveRecordCnt == 1 && lostRecordCnt == 1) {
throw new RuntimeException("Kill this Job!");
}
// update checkpointed counter
saveRecordCnt++;
// update non-checkpointed counter
lostRecordCnt++;
// forward record
return value;
}
@Override
public TypeInformation getProducedType() {
return Types.ROW(Types.INT, Types.SQL_TIMESTAMP);
}
@Override
public List<Integer> snapshotState(long checkpointId, long timestamp) {
return Collections.singletonList(saveRecordCnt);
}
@Override
public void restoreState(List<Integer> state) {
for (Integer i : state) {
saveRecordCnt += i;
}
}
}
}