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

[issue-265] Reader side: Implement watermark with Pravega (for master branch aka. Flink 1.9) #279

Merged
merged 1 commit into from
Oct 10, 2019
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
4 changes: 2 additions & 2 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@
language: java
install: true
jdk:
- oraclejdk8
- openjdk8

# the secure configurations in env: section is for BINTRAY_USER=<USER> and BINTRAY_KEY=<KEY> properties
# which will be used for publishing artifacts to snapshot repository
Expand Down Expand Up @@ -55,5 +55,5 @@ notifications:
secure: Gv0RJx1Sa/y5fmvLNwY+2ivfWZYCM0ekrr6UAHqsegnid6P/DFZrSrfSpwvcVh2OVNH8DHLV0BoiuDJ7amtl1eMDMXz5/lLz8tFWFKaHv4yDSadm8ILY/KnYUoP4IRuM3NyKQmBrmZB9Or5KFXboG6ex6UkgbuYy0Zyl6syEe168Iw8hlCRx26Jei7/y+8eE2MIGFh09TLRZ/944YbULum9H3KQLYv8nFdPc7GmR5AK461fnwZ7iYjb7MXkCctE5Vml3p9+2Qliv1ZJqNsQeKmSFW6IhiP6pNZ1V8VJEWMQmX/nBr9745l/N+CoLQz9ajLonlxn9xHdWms4TEu1ynFk6uxEJjlcpXcvcEaKhqAKcTMl0GMMRab2m+/Vt3S/VutJnVXQmnhZGT9glLFQHwcdHNqM/LEbXtyisB7zmGImUQpF2InCwO25IXug5gv64IfOHGMzL56yNIhbRgBY9Ud4Tux+pmkV5ZxJiBkul7/FiHQX7tQLUrzQosD0oyCOmaWD7kmbt15A0TOkLgup4HE+sSS1ASwisa7J2+HsbI3Upy3rNVKuIJP0L4KSTn4HSlDlMLLcWM+nz/YCEfuwSRXJTIstotNYHdsLUZAZSYAX7ejpeiuBRed4a4AlCROeKbKKwCcSvqCOjmCaPTpwJAGeJByOXLL2hfQzpDMKCIKM=
email:
- [email protected]
- vijayaraghavan.srinivasaraghavan@emc.com
- brian.zhou@emc.com
- [email protected]
2 changes: 1 addition & 1 deletion gradle.properties
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ usePravegaVersionSubModule=false

# These properties are only needed for publishing to maven central
# Pravega Signing Key
signing.keyId=05949AF6
signing.keyId=F56ED3A7
# This will be defaulted to ~/.gnupg/secring.gpg if not provided as a command line property
signing.secretKeyRingFile=

Expand Down
4 changes: 2 additions & 2 deletions gradle/java.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,6 @@ tasks.withType(Test) {
testLogging.showStackTraces = true
testLogging.events = ["PASSED", "FAILED"]
maxParallelForks = System.properties['maxParallelForks'] ? System.properties['maxParallelForks'].toInteger() : 1
minHeapSize = "128m"
maxHeapSize = "512m"
minHeapSize = "512m"
maxHeapSize = "2048m"
}
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,10 @@
import com.google.common.base.Preconditions;
import io.pravega.client.stream.ReaderGroupConfig;
import io.pravega.connectors.flink.util.FlinkPravegaUtils;
import io.pravega.connectors.flink.watermark.AssignerWithTimeWindows;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.util.SerializedValue;

import java.util.Optional;

Expand Down Expand Up @@ -115,7 +117,7 @@ public B withEventReadTimeout(Time eventReadTimeout) {
/**
* Configures the maximum outstanding checkpoint requests to Pravega (default=3).
* Upon requesting more checkpoints than the specified maximum,
* (say a checkpoint request timesout on the ReaderCheckpointHook but Pravega is still working on it),
* (say a checkpoint request times out on the ReaderCheckpointHook but Pravega is still working on it),
* this configurations allows Pravega to limit any further checkpoint request being made to the ReaderGroup.
* This configuration is particularly relevant when multiple checkpoint requests need to be honored (e.g., frequent savepoint requests being triggered concurrently).
*
Expand All @@ -128,6 +130,8 @@ public B withMaxOutstandingCheckpointRequest(int maxOutstandingCheckpointRequest

protected abstract DeserializationSchema<T> getDeserializationSchema();

protected abstract SerializedValue<AssignerWithTimeWindows<T>> getAssignerWithTimeWindows();

/**
* Builds a {@link FlinkPravegaReader} based on the configuration.
*
Expand All @@ -148,6 +152,7 @@ FlinkPravegaReader<T> buildSourceFunction() {
readerGroupInfo.getReaderGroupScope(),
readerGroupInfo.getReaderGroupName(),
getDeserializationSchema(),
getAssignerWithTimeWindows(),
this.eventReadTimeout,
this.checkpointInitiateTimeout,
isMetricsEnabled());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,11 +9,13 @@
*/
package io.pravega.connectors.flink;

import io.pravega.connectors.flink.watermark.AssignerWithTimeWindows;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.descriptors.ConnectorDescriptor;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.types.Row;
import org.apache.flink.util.SerializedValue;

import java.util.function.Supplier;

Expand Down Expand Up @@ -84,6 +86,11 @@ protected io.pravega.connectors.flink.serialization.JsonRowDeserializationSchema
return deserSchema;
}

@Override
protected SerializedValue<AssignerWithTimeWindows<Row>> getAssignerWithTimeWindows() {
return null;
}

/**
* Builds a {@link FlinkPravegaReader} based on the configuration.
*
Expand Down
126 changes: 124 additions & 2 deletions src/main/java/io/pravega/connectors/flink/FlinkPravegaReader.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,13 @@
import io.pravega.client.stream.ReaderGroupConfig;
import io.pravega.client.stream.Stream;
import io.pravega.client.stream.StreamCut;
import io.pravega.connectors.flink.watermark.AssignerWithTimeWindows;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.Gauge;
Expand All @@ -32,8 +35,14 @@
import org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.SerializedValue;

import java.io.IOException;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
Expand Down Expand Up @@ -95,6 +104,9 @@ public class FlinkPravegaReader<T>
// The supplied event deserializer.
final DeserializationSchema<T> deserializationSchema;

// The supplied event timestamp and watermark assigner.
final SerializedValue<AssignerWithTimeWindows<T>> assignerWithTimeWindows;

// the timeout for reading events from Pravega
final Time eventReadTimeout;

Expand Down Expand Up @@ -136,13 +148,16 @@ public class FlinkPravegaReader<T>
* @param readerGroupScope The reader group scope name.
* @param readerGroupName The reader group name.
* @param deserializationSchema The implementation to deserialize events from Pravega streams.
* @param assignerWithTimeWindows The serialized value of the implementation to extract timestamp from deserialized events (only in event-time mode).
* @param eventReadTimeout The event read timeout.
* @param checkpointInitiateTimeout The checkpoint initiation timeout.
* @param enableMetrics Flag to indicate whether metrics needs to be enabled or not.
*/
protected FlinkPravegaReader(String hookUid, ClientConfig clientConfig,
ReaderGroupConfig readerGroupConfig, String readerGroupScope, String readerGroupName,
DeserializationSchema<T> deserializationSchema, Time eventReadTimeout, Time checkpointInitiateTimeout,
DeserializationSchema<T> deserializationSchema,
SerializedValue<AssignerWithTimeWindows<T>> assignerWithTimeWindows,
Time eventReadTimeout, Time checkpointInitiateTimeout,
boolean enableMetrics) {

this.hookUid = Preconditions.checkNotNull(hookUid, "hookUid");
Expand All @@ -154,6 +169,7 @@ protected FlinkPravegaReader(String hookUid, ClientConfig clientConfig,
this.eventReadTimeout = Preconditions.checkNotNull(eventReadTimeout, "eventReadTimeout");
this.checkpointInitiateTimeout = Preconditions.checkNotNull(checkpointInitiateTimeout, "checkpointInitiateTimeout");
this.enableMetrics = enableMetrics;
this.assignerWithTimeWindows = assignerWithTimeWindows;
}

/**
Expand All @@ -164,6 +180,57 @@ void initialize() {
// See https://github.com/pravega/flink-connectors/issues/130.
log.info("Creating reader group: {}/{} for the Flink job", this.readerGroupScope, this.readerGroupName);
createReaderGroup();
if (isEventTimeMode()) {
Preconditions.checkArgument(readerGroup.getStreamNames().size() == 1,
"Only 1 Pravega stream is allowed in the event-time mode");

}
}

private boolean isEventTimeMode() {
return assignerWithTimeWindows != null;
}

private long autoWatermarkInterval() {
return getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval();
}

private class PeriodicWatermarkEmitter implements ProcessingTimeCallback {

private EventStreamReader<?> pravegaReader;
private final SourceContext<?> ctx;
private final ProcessingTimeService timerService;
private long lastWatermarkTimestamp;
private AssignerWithTimeWindows<?> userAssigner;

protected PeriodicWatermarkEmitter(
EventStreamReader<?> pravegaReader, SourceContext<?> ctx, ClassLoader userCodeClassLoader,
ProcessingTimeService timerService) throws Exception {
this.pravegaReader = Preconditions.checkNotNull(pravegaReader);
this.ctx = Preconditions.checkNotNull(ctx);
this.timerService = Preconditions.checkNotNull(timerService);
this.lastWatermarkTimestamp = Long.MIN_VALUE;
this.userAssigner = assignerWithTimeWindows.deserializeValue(userCodeClassLoader);
}

protected void start() {
timerService.registerTimer(timerService.getCurrentProcessingTime() + autoWatermarkInterval(), this);
}

@Override
public void onProcessingTime(long timestamp) {
Stream stream = Stream.of(readerGroup.getStreamNames().iterator().next());
Watermark watermark = userAssigner.getWatermark(pravegaReader.getCurrentTimeWindow(stream));

if (watermark != null && watermark.getTimestamp() > lastWatermarkTimestamp) {
lastWatermarkTimestamp = watermark.getTimestamp();
log.debug("Emit watermark with timestamp: {}", watermark.getTimestamp());
ctx.emitWatermark(watermark);
}

// schedule the next watermark
timerService.registerTimer(timerService.getCurrentProcessingTime() + autoWatermarkInterval(), this);
}
}

// ------------------------------------------------------------------------
Expand All @@ -182,6 +249,23 @@ public void run(SourceContext<T> ctx) throws Exception {

log.info("Starting Pravega reader '{}' for controller URI {}", readerId, this.clientConfig.getControllerURI());

long previousTimestamp = Long.MIN_VALUE;
AssignerWithTimeWindows<T> assigner = null;
// If it is event time, register a watermark emitter
if (isEventTimeMode()) {
assigner = assignerWithTimeWindows.deserializeValue(getRuntimeContext().getUserCodeClassLoader());
PeriodicWatermarkEmitter periodicEmitter = new PeriodicWatermarkEmitter(
pravegaReader,
ctx,
getRuntimeContext().getUserCodeClassLoader(),
((StreamingRuntimeContext) getRuntimeContext()).getProcessingTimeService());

log.info("Periodic Watermark Emitter for Reader ID: {} has started with an interval of {}", readerId,
autoWatermarkInterval());

periodicEmitter.start();
}

// main work loop, which this task is running
while (this.running) {
final EventRead<T> eventRead = pravegaReader.readNextEvent(eventReadTimeout.toMilliseconds());
Expand All @@ -198,7 +282,13 @@ public void run(SourceContext<T> ctx) throws Exception {
}

synchronized (ctx.getCheckpointLock()) {
ctx.collect(event);
if (isEventTimeMode()) {
long currentTimestamp = assigner.extractTimestamp(event, previousTimestamp);
ctx.collectWithTimestamp(event, currentTimestamp);
previousTimestamp = currentTimestamp;
} else {
ctx.collect(event);
}
}
}

Expand All @@ -207,6 +297,10 @@ public void run(SourceContext<T> ctx) throws Exception {
triggerCheckpoint(eventRead.getCheckpointName());
}
}

if (isEventTimeMode()) {
ctx.emitWatermark(Watermark.MAX_WATERMARK);
}
}
}

Expand All @@ -226,6 +320,11 @@ public void open(Configuration parameters) throws Exception {
if (enableMetrics) {
registerMetrics();
}
if (isEventTimeMode()) {
Preconditions.checkArgument(autoWatermarkInterval() > 0,
"Periodic watermark interval should be positive, " +
"please use env.getConfig().setAutoWatermarkInterval() to set a positive number. Recommended value: 10000");
}
}

@Override
Expand Down Expand Up @@ -473,6 +572,7 @@ public static <T> FlinkPravegaReader.Builder<T> builder() {
public static class Builder<T> extends AbstractStreamingReaderBuilder<T, Builder<T>> {

private DeserializationSchema<T> deserializationSchema;
private SerializedValue<AssignerWithTimeWindows<T>> assignerWithTimeWindows;

protected Builder<T> builder() {
return this;
Expand All @@ -482,18 +582,40 @@ protected Builder<T> builder() {
* Sets the deserialization schema.
*
* @param deserializationSchema The deserialization schema
* @return Builder instance.
*/
public Builder<T> withDeserializationSchema(DeserializationSchema<T> deserializationSchema) {
this.deserializationSchema = deserializationSchema;
return builder();
}

/**
* Sets the timestamp and watermark assigner.
*
* @param assignerWithTimeWindows The timestamp and watermark assigner.
* @return Builder instance.
*/
public Builder<T> withTimestampAssigner(AssignerWithTimeWindows<T> assignerWithTimeWindows) {
try {
ClosureCleaner.clean(assignerWithTimeWindows, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
this.assignerWithTimeWindows = new SerializedValue<>(assignerWithTimeWindows);
} catch (IOException e) {
throw new IllegalArgumentException("The given assigner is not serializable", e);
}
return this;
}

@Override
protected DeserializationSchema<T> getDeserializationSchema() {
Preconditions.checkState(deserializationSchema != null, "Deserialization schema must not be null.");
return deserializationSchema;
}

@Override
protected SerializedValue<AssignerWithTimeWindows<T>> getAssignerWithTimeWindows() {
return assignerWithTimeWindows;
}

/**
* Builds a {@link FlinkPravegaReader} based on the configuration.
* @throws IllegalStateException if the configuration is invalid.
Expand Down
Loading