From fb1fbd6bdcc81acd20d422842789fce0c0872580 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 22 Jul 2015 20:53:11 +0300 Subject: [PATCH 001/175] Solved the #887 issue: removing the akka.framesize size limitation for the result of a job. --- .../apache/flink/client/program/Client.java | 5 +- .../accumulators/AccumulatorHelper.java | 17 +++ flink-examples/flink-java-examples/pom.xml | 2 +- .../accumulators/AccumulatorRegistry.java | 93 ++++++++++++- ...shot.java => BaseAccumulatorSnapshot.java} | 26 +--- .../LargeAccumulatorSnapshot.java | 58 ++++++++ .../SmallAccumulatorSnapshot.java | 54 ++++++++ .../runtime/blob/BlobServerProtocol.java | 2 +- .../flink/runtime/client/JobClient.java | 126 +++++++++++++++--- .../client/SerializedJobExecutionResult.java | 95 +++++++++++-- .../librarycache/BlobLibraryCacheManager.java | 1 + .../FallbackLibraryCacheManager.java | 5 + .../librarycache/LibraryCacheManager.java | 7 + .../runtime/executiongraph/Execution.java | 37 +++-- .../executiongraph/ExecutionGraph.java | 106 ++++++++++++--- .../flink/runtime/taskmanager/Task.java | 28 +++- .../taskmanager/TaskExecutionState.java | 8 +- .../flink/runtime/util/SerializedValue.java | 14 ++ .../apache/flink/runtime/akka/AkkaUtils.scala | 5 + .../flink/runtime/jobmanager/JobManager.scala | 73 +++++----- .../runtime/jobmanager/MemoryArchivist.scala | 2 +- .../messages/TaskManagerMessages.scala | 4 +- .../runtime/taskmanager/TaskManager.scala | 21 ++- .../SerializedJobExecutionResultTest.java | 5 +- .../testingUtils/TestingJobManager.scala | 2 +- .../test/misc/MiscellaneousIssuesITCase.java | 29 +++- 26 files changed, 681 insertions(+), 144 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/{AccumulatorSnapshot.java => BaseAccumulatorSnapshot.java} (67%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 50590dfb61a5d..19fc365f60784 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -412,10 +412,11 @@ public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramIn try{ if (wait) { - SerializedJobExecutionResult result = JobClient.submitJobAndWait(actorSystem, + SerializedJobExecutionResult partialResult = JobClient.submitJobAndWait(actorSystem, jobManagerGateway, jobGraph, timeout, printStatusDuringExecution); try { - return result.toJobExecutionResult(this.userCodeClassLoader); + return JobClient.returnFinalJobExecutionResult(jobManagerGateway, partialResult, + this.userCodeClassLoader, timeout); } catch (Exception e) { throw new ProgramInvocationException( diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java index 3907004369f66..16ca096df9912 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java @@ -51,6 +51,23 @@ public static void mergeInto(Map> target, Map target, Accumulator toMerge) { + // Both should have the same type + AccumulatorHelper.compareAccumulatorTypes(name, target.getClass(), toMerge.getClass()); + // Merge counter from chained task into counter from stub + mergeSingle(target, toMerge); + } + /** * Workaround method for type safety */ diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml index 8448a93db434d..33190abbf9563 100644 --- a/flink-examples/flink-java-examples/pom.xml +++ b/flink-examples/flink-java-examples/pom.xml @@ -319,7 +319,7 @@ under the License. - + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java index 0ef3650292b48..6d53f2045f697 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java @@ -21,13 +21,22 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.util.SerializedValue; +import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; @@ -38,6 +47,8 @@ public class AccumulatorRegistry { protected static final Logger LOG = LoggerFactory.getLogger(AccumulatorRegistry.class); + protected final Configuration jobConfiguration; + protected final InetSocketAddress blobServerAddress; protected final JobID jobID; protected final ExecutionAttemptID taskID; @@ -62,26 +73,100 @@ public enum Metric { NUM_BYTES_OUT } - public AccumulatorRegistry(JobID jobID, ExecutionAttemptID taskID) { + this(null, jobID, taskID, null); + } + + public AccumulatorRegistry(Configuration jobConfig, JobID jobID, ExecutionAttemptID taskID, InetSocketAddress blobServerAddress) { + this.jobConfiguration = jobConfig; this.jobID = jobID; this.taskID = taskID; + this.blobServerAddress = blobServerAddress; this.reporter = new ReadWriteReporter(flinkAccumulators); } /** - * Creates a snapshot of this accumulator registry. + * Creates a snapshot of this accumulator registry. If they are oversized (i.e. bigger than + * akka.framesize), this method stores them in the BlobCache and sends only the + * corresponding BlobKeys in the final snapshot. If they are not, it sends the actual + * accumulators in the accumulator snapshot. + * * @return a serialized accumulator map */ - public AccumulatorSnapshot getSnapshot() { + public BaseAccumulatorSnapshot getSnapshot() { + BaseAccumulatorSnapshot snapshot; + Map> largeAccumulatorBlobKeys; + SerializedValue>> serializedAccumulators; + try { - return new AccumulatorSnapshot(jobID, taskID, flinkAccumulators, userAccumulators); + serializedAccumulators = new SerializedValue>>(userAccumulators); + if (serializedAccumulators.getSizeInBytes() > 0.8 * AkkaUtils.getFramesize(jobConfiguration)) { + + largeAccumulatorBlobKeys = storeAccumulatorsToBlobCache(blobServerAddress, userAccumulators); + snapshot = new LargeAccumulatorSnapshot(jobID, taskID, + flinkAccumulators, largeAccumulatorBlobKeys); + + } else { + snapshot = new SmallAccumulatorSnapshot(jobID, taskID, + flinkAccumulators, serializedAccumulators); + } + return snapshot; } catch (IOException e) { LOG.warn("Failed to serialize accumulators for task.", e); return null; } } + /** + * Puts the blobs of the large accumulators on the BlobCache. + * @param blobServerAddress the address of the server to the blobCache. + * @param accumulators the accumulators to be stored in the cache. + * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. + * */ + private Map> storeAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, + Map> accumulators) { + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> keys = new HashMap>(); + BlobClient bc = null; + try { + bc = new BlobClient(blobServerAddress); + + for (Map.Entry> entry : accumulators.entrySet()) { + + String accumulatorName = entry.getKey(); + Accumulator accumulator = entry.getValue(); + + byte[] serializedAccumulator = InstantiationUtil.serializeObject(accumulator); + BlobKey blobKey = bc.put(serializedAccumulator); + + List accKeys = keys.get(accumulatorName); + if (accKeys == null) { + accKeys = new ArrayList(); + } + accKeys.add(blobKey); + keys.put(accumulatorName, accKeys); + } + } catch (IOException e) { + LOG.error("Failed to send oversized accumulators to the BlobCache: ", e); + } finally { + try { + if(bc != null) { + bc.close(); + } + } catch (IOException e) { + LOG.error("Failed to close BlobClient: ", e); + } + + } + return keys; + } + /** * Gets the map for user-defined accumulators. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java similarity index 67% rename from flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java index 0f1911db2f137..09bed90dfb763 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java @@ -28,11 +28,10 @@ import java.util.Map; /** - * This class encapsulates a map of accumulators for a single task. It is used - * for the transfer from TaskManagers to the JobManager and from the JobManager - * to the Client. + * LICENSE SHOULD GO HERE. + * Created by kkloudas on 7/23/15. */ -public class AccumulatorSnapshot implements Serializable { +public class BaseAccumulatorSnapshot implements Serializable { private static final long serialVersionUID = 42L; @@ -44,18 +43,11 @@ public class AccumulatorSnapshot implements Serializable { */ private final SerializedValue>> flinkAccumulators; - /** - * Serialized user accumulators which may require the custom user class loader. - */ - private final SerializedValue>> userAccumulators; - - public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, - Map> flinkAccumulators, - Map> userAccumulators) throws IOException { + public BaseAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators) throws IOException { this.jobID = jobID; this.executionAttemptID = executionAttemptID; this.flinkAccumulators = new SerializedValue>>(flinkAccumulators); - this.userAccumulators = new SerializedValue>>(userAccumulators); } public JobID getJobID() { @@ -73,12 +65,4 @@ public ExecutionAttemptID getExecutionAttemptID() { public Map> deserializeFlinkAccumulators() throws IOException, ClassNotFoundException { return flinkAccumulators.deserializeValue(ClassLoader.getSystemClassLoader()); } - - /** - * Gets the user-defined accumulators values. - * @return the serialized map - */ - public Map> deserializeUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { - return userAccumulators.deserializeValue(classLoader); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java new file mode 100644 index 0000000000000..a4283f165cc26 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java @@ -0,0 +1,58 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * LICENSE SHOULD GO HERE. + * Created by kkloudas on 7/23/15. + */ +public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { + + /** + * In case some accumulators do not fit in akka, we store them in the blobCache and put + * in the snapshot only the mapping between the name of the accumulator, and its blobKey + * in the cache. This list holds exactly this mapping. + * */ + private final Map> largeUserAccumulatorBlobs; + + public LargeAccumulatorSnapshot( + JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + Map> oversizedUserAccumulatorBlobKeys) throws IOException { + super(jobID, executionAttemptID, flinkAccumulators); + this.largeUserAccumulatorBlobs = oversizedUserAccumulatorBlobKeys; + } + + /** + * Gets the Flink (internal) accumulators values. + * @return the serialized map + */ + public Map> getLargeAccumulatorBlobKeys() { + return largeUserAccumulatorBlobs; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java new file mode 100644 index 0000000000000..1911ebac6c321 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java @@ -0,0 +1,54 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.util.SerializedValue; + +import java.io.IOException; +import java.util.Map; + +/** + * LICENSE SHOULD GO HERE. + * Created by kkloudas on 7/23/15. + */ +public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { + + /** + * Serialized user accumulators which may require the custom user class loader. + */ + private final SerializedValue>> userAccumulators; + + public SmallAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + SerializedValue>> userAccumulators) throws IOException { + super(jobID, executionAttemptID, flinkAccumulators); + this.userAccumulators = userAccumulators; + } + + /** + * Gets the user-defined accumulators values. + * @return the serialized map + */ + public Map> deserializeSmallUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { + return userAccumulators.deserializeValue(classLoader); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java index 6df78110b9a2c..c5bcce52b1594 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java @@ -24,7 +24,7 @@ public class BlobServerProtocol { // -------------------------------------------------------------------------------------------- /** The buffer size in bytes for network transfers. */ - static final int BUFFER_SIZE = 65536; // 64 K + public static final int BUFFER_SIZE = 65536; // 64 K /** The maximum key length allowed for storing BLOBs. */ static final int MAX_KEY_LENGTH = 64; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index aeefa612ba4f9..3b64b8293b96a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -27,15 +27,20 @@ import akka.pattern.Patterns; import akka.util.Timeout; import com.google.common.base.Preconditions; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.blob.BlobServerProtocol; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.JobClientMessages; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +51,17 @@ import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeoutException; /** @@ -187,9 +199,29 @@ public static SerializedJobExecutionResult submitJobAndWait( } } + public static JobExecutionResult returnFinalJobExecutionResult( + ActorGateway jobManagerGateway, SerializedJobExecutionResult partialResult, + ClassLoader userCodeClassLoader, FiniteDuration timeout) + throws IOException, ClassNotFoundException { + + if (jobManagerGateway == null || partialResult == null || userCodeClassLoader == null || timeout == null) { + throw new NullPointerException(); + } + + Map> blobsToFetch = partialResult.getBlobKeysToLargeAccumulators(); + + Map>> accumulatorBlobs; + try { + accumulatorBlobs = getLargeAccumulatorBlobs(jobManagerGateway, blobsToFetch, timeout); + } catch (IOException e) { + throw new IOException("Failed to fetch the oversized accumulators from the BlobCache", e); + } + return partialResult.mergeToJobExecutionResult(userCodeClassLoader, accumulatorBlobs); + } + /** * Submits a job in detached mode. The method sends the JobGraph to the - * JobManager and waits for the answer whether teh job could be started or not. + * JobManager and waits for the answer whether the job could be started or not. * * @param jobManagerGateway Gateway to the JobManager which will execute the jobs * @param jobGraph The job @@ -250,29 +282,87 @@ public static void uploadJarFiles( FiniteDuration timeout) throws IOException { if (jobGraph.hasUsercodeJarFiles()) { + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + jobGraph.uploadRequiredJarFiles(serverAddress); + } + } - Future futureBlobPort = jobManagerGateway.ask( - JobManagerMessages.getRequestBlobManagerPort(), - timeout); + /** + * If the result of the job contained oversized (i.e. bigger that the akka.framesize) accumulators + * then these are put in the BlobCache for the client to fetch and merge. This method gets + * them from the BlobCache (if there was any). If the list of blobs to fetch is empty, then + * an empty result is returned, as all (partial) accumulators were small enough to be sent + * directly to the JobManager and be merged there. + * + * @param jobManagerGateway the reference to the jobManager actor. + * @param keys the accumulators to fetch (based on their name) along with their associated BlobKeys. + * @param timeout the timeout to wait for the connection to the blob server. + * @return the serialized accumulators, grouped by name. + * */ + private static Map>> getLargeAccumulatorBlobs( + ActorGateway jobManagerGateway, Map> keys, FiniteDuration timeout) throws IOException { + + if (keys.isEmpty()) { + return Collections.emptyMap(); + } - int port; - try { - Object result = Await.result(futureBlobPort, timeout); - if (result instanceof Integer) { - port = (Integer) result; - } else { - throw new Exception("Expected port number (int) as answer, received " + result); + Map>> accumulatorBlobs = + new HashMap>>(); + + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + BlobClient bc = new BlobClient(serverAddress); + + final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE]; + for(String accName: keys.keySet()) { + List accBlobKeys = keys.get(accName); + List> accBlobs = new ArrayList>(); + + for(BlobKey bk: accBlobKeys) { + InputStream is = bc.get(bk); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + while (true) { + final int read = is.read(buf); + if (read < 0) { + break; + } + os.write(buf, 0, read); } + os.flush(); + byte[] blob = os.toByteArray(); + accBlobs.add(new SerializedValue(blob)); + is.close(); + os.close(); + + // after getting them, clean up and delete the blobs from the BlobCache. + bc.delete(bk); } - catch (Exception e) { - throw new IOException("Could not retrieve the JobManager's blob port.", e); - } + accumulatorBlobs.put(accName, accBlobs); + } + bc.close(); + return accumulatorBlobs; + } - Option jmHost = jobManagerGateway.actor().path().address().host(); - String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; - InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, port); + private static InetSocketAddress getBlobServerAddress(ActorGateway jobManagerGateway, FiniteDuration timeout) throws IOException { + Timeout tOut = new Timeout(timeout); + Future futureBlobPort = jobManagerGateway.ask( + JobManagerMessages.getRequestBlobManagerPort(), + timeout); - jobGraph.uploadRequiredJarFiles(serverAddress); + int port; + try { + Object result = Await.result(futureBlobPort, timeout); + if (result instanceof Integer) { + port = (Integer) result; + } else { + throw new Exception("Expected port number (int) as answer, received " + result); + } + } catch (Exception e) { + throw new IOException("Could not retrieve the JobManager's blob port.", e); } + + Option jmHost = jobManagerGateway.actor().path().address().host(); + String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; + InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, port); + return serverAddress; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java index 029bc3876f738..0b1115dbe326a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java @@ -20,11 +20,15 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.util.SerializedValue; import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -38,7 +42,13 @@ public class SerializedJobExecutionResult implements java.io.Serializable { private final JobID jobId; - private final Map> accumulatorResults; + // holds the result value of an accumulator + private final Map> smallAccumulatorResults; + + // holds the BlobKeys pointing to the blobs in the BlobCache + // containing the serialized accumulators + private final Map> largeAccumulatorBlobRefs; + private final long netRuntime; @@ -47,13 +57,17 @@ public class SerializedJobExecutionResult implements java.io.Serializable { * * @param jobID The job's ID. * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer) in milliseconds - * @param accumulators A map of all accumulator results produced by the job, in serialized form + * @param smallAccumulators A map of all SMALL accumulator results produced by the job, in serialized form + * @param largeAccumulatorBlobRefs A map of all the references to the blobs holding the LARGE accumulators that had + * to be sent through the BlobCache. */ public SerializedJobExecutionResult(JobID jobID, long netRuntime, - Map> accumulators) { + Map> smallAccumulators, + Map> largeAccumulatorBlobRefs) { this.jobId = jobID; this.netRuntime = netRuntime; - this.accumulatorResults = accumulators; + this.smallAccumulatorResults = smallAccumulators; + this.largeAccumulatorBlobRefs = largeAccumulatorBlobRefs; } public JobID getJobId() { @@ -76,17 +90,28 @@ public long getNetRuntime(TimeUnit desiredUnit) { } public Map> getSerializedAccumulatorResults() { - return this.accumulatorResults; + return this.smallAccumulatorResults; + } + + /** + * If the result of the job contained oversized (i.e. bigger that the akka.framesize) accumulators + * then these are put in the BlobCache for the client to fetch and merge. This method gets + * their BlobKeys. If the result is empty, then this means that all (partial) accumulators + * were small enough to be sent directly to the JobManager and be merged there. + * @return the BlobKeys to the blobs containing the oversized accumulators. + * */ + public Map> getBlobKeysToLargeAccumulators() { + return this.largeAccumulatorBlobRefs; } public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOException, ClassNotFoundException { Map accumulators = null; - if (accumulatorResults != null) { - accumulators = accumulatorResults.isEmpty() ? + if (smallAccumulatorResults != null) { + accumulators = smallAccumulatorResults.isEmpty() ? Collections.emptyMap() : - new HashMap(this.accumulatorResults.size()); + new HashMap(this.smallAccumulatorResults.size()); - for (Map.Entry> entry : this.accumulatorResults.entrySet()) { + for (Map.Entry> entry : this.smallAccumulatorResults.entrySet()) { Object o = entry.getValue() == null ? null : entry.getValue().deserializeValue(loader); accumulators.put(entry.getKey(), o); } @@ -94,4 +119,56 @@ public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOExce return new JobExecutionResult(jobId, netRuntime, accumulators); } + + /** + * Merges the data of the small accumulators in this class, with the accumulatorsToMerge, and + * returns the result in a JobExecutionResult. This method is used by the Client to merge + * the results received from the JobManager, with those in the oversized accumulators that were fetched + * from the BlobCache. + * @param loader + * the ClassLoader to used to deserialize the data and the Accumulators. + * @param accumulatorsToMerge + * the Accumulators received from the BlobCache as blobs, that are to be merged with local data. + * @return the final result after the merging of the different partial accumulators. + * */ + public JobExecutionResult mergeToJobExecutionResult(ClassLoader loader, Map>> accumulatorsToMerge) throws IOException, ClassNotFoundException { + if(accumulatorsToMerge == null || accumulatorsToMerge.isEmpty()) { + return toJobExecutionResult(loader); + } + + Map accumulators = new HashMap(); + + for(String name: accumulatorsToMerge.keySet()) { + List> blobs = accumulatorsToMerge.get(name); + + // merge the serialized accumulators + Accumulator acc = (Accumulator) blobs.get(0).deserializeValue(loader); + for(int i = 1; i < blobs.size(); i++) { + AccumulatorHelper.mergeAccumulators(name, acc, + (Accumulator) blobs.get(i).deserializeValue(loader)); + } + + // add also the data from the non-oversized (i.e. the ones that were sent through akka) + // accumulators, if any + if (smallAccumulatorResults != null) { + SerializedValue localObject = smallAccumulatorResults.remove(name); + if (localObject != null) { + acc.add(localObject.deserializeValue(loader)); + } + } + + // and put the data with the associated accumulator name to the list + accumulators.put(name, acc.getLocalValue()); + } + + // finally, put the remaining accumulators in the list. + if (smallAccumulatorResults != null) { + for (Map.Entry> entry : this.smallAccumulatorResults.entrySet()) { + Object o = entry.getValue() == null ? null : entry.getValue().deserializeValue(loader); + accumulators.put(entry.getKey(), o); + } + } + return new JobExecutionResult(jobId, netRuntime, accumulators); + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 88be5e1f37735..ad482cf4596fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -181,6 +181,7 @@ public File getFile(BlobKey blobKey) throws IOException { return new File(blobService.getURL(blobKey).getFile()); } + @Override public int getBlobServerPort() { return blobService.getPort(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java index 66bda45873014..b0d5077b1820f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java @@ -32,6 +32,11 @@ public class FallbackLibraryCacheManager implements LibraryCacheManager { private static Logger LOG = LoggerFactory.getLogger(FallbackLibraryCacheManager.class); + @Override + public int getBlobServerPort() { + return 0; + } + @Override public ClassLoader getClassLoader(JobID id) { return getClass().getClassLoader(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java index 52a8048867804..c1f4f365e6ada 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java @@ -27,6 +27,13 @@ import java.util.Collection; public interface LibraryCacheManager { + + /** + * Returns the port the service is listening to. + * @return the port of the cache server. + * */ + int getBlobServerPort(); + /** * Returns the user code class loader associated with id. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index aa0f98148548a..4c08a44462a04 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionLocation; @@ -140,7 +141,10 @@ public class Execution implements Serializable { private final Object accumulatorLock = new Object(); /* Continuously updated map of user-defined accumulators */ - private volatile Map> userAccumulators; + private volatile Map> smallUserAccumulators; + + /* Keep track of the large accumulators that are stored in the BlobCache */ + private volatile Map> largeUserAccumulators; /* Continuously updated map of internal accumulators */ private volatile Map> flinkAccumulators; @@ -605,12 +609,14 @@ void markFailed(Throwable t) { } void markFinished() { - markFinished(null, null); + markFinished(null, null, null); } - void markFinished(Map> flinkAccumulators, Map> userAccumulators) { + void markFinished(Map> flinkAccumulators, + Map> userAccumulators, + Map> userLargeAccumulators) { - // this call usually comes during RUNNING, but may also come while still in deploying (very fast tasks!) + // this call usually comes during RUNNING, but may also come while still in DEPLOYING (very fast tasks!) while (true) { ExecutionState current = this.state; @@ -631,7 +637,8 @@ void markFinished(Map> flinkAccumu synchronized (accumulatorLock) { this.flinkAccumulators = flinkAccumulators; - this.userAccumulators = userAccumulators; + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulators = userLargeAccumulators; } assignedResource.releaseSlot(); @@ -865,8 +872,7 @@ public void onComplete(Throwable failure, Object success) throws Throwable { } else { TaskOperationResult result = (TaskOperationResult) success; if (!result.success()) { - LOG.debug("Cancel task call did not find task. Probably akka message call" + - " race."); + LOG.debug("Cancel task call did not find task. Probably akka message call race."); } } } @@ -960,18 +966,27 @@ public String getVertexWithAttempt() { * Update accumulators (discarded when the Execution has already been terminated). * @param flinkAccumulators the flink internal accumulators * @param userAccumulators the user accumulators + * @param userLargeAccumulatorBlobKeys the keys to the blobs storing the oversized accumulators sent by + * (some) tasks. */ public void setAccumulators(Map> flinkAccumulators, - Map> userAccumulators) { + Map> userAccumulators, + Map> userLargeAccumulatorBlobKeys) { synchronized (accumulatorLock) { if (!state.isTerminal()) { this.flinkAccumulators = flinkAccumulators; - this.userAccumulators = userAccumulators; + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulators = userLargeAccumulatorBlobKeys; } } } - public Map> getUserAccumulators() { - return userAccumulators; + + public Map> getSmallUserAccumulators() { + return smallUserAccumulators; + } + + public Map> getLargeUserAccumulatorBlobKeys() { + return largeUserAccumulators; } public Map> getFlinkAccumulators() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 833518ce84513..788982eef8517 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -24,8 +24,10 @@ import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.LargeAccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.SmallAccumulatorSnapshot; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; @@ -60,6 +62,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.Set; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -142,17 +146,25 @@ public class ExecutionGraph implements Serializable { * through the UpdateTaskExecutionState message. * @param accumulatorSnapshot The serialized flink and user-defined accumulators */ - public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { + public void updateAccumulators(BaseAccumulatorSnapshot accumulatorSnapshot) { Map> flinkAccumulators; - Map> userAccumulators; + Map> smallUserAccumulators = null; + Map> largeUserAccumulators = null; try { flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - userAccumulators = accumulatorSnapshot.deserializeUserAccumulators(userClassLoader); + + if(accumulatorSnapshot instanceof SmallAccumulatorSnapshot) { + smallUserAccumulators = ((SmallAccumulatorSnapshot) accumulatorSnapshot). + deserializeSmallUserAccumulators(userClassLoader); + } else if(accumulatorSnapshot instanceof LargeAccumulatorSnapshot) { + largeUserAccumulators = ((LargeAccumulatorSnapshot) accumulatorSnapshot). + getLargeAccumulatorBlobKeys(); + } ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); Execution execution = currentExecutions.get(execID); if (execution != null) { - execution.setAccumulators(flinkAccumulators, userAccumulators); + execution.setAccumulators(flinkAccumulators, smallUserAccumulators, largeUserAccumulators); } else { LOG.warn("Received accumulator result for unknown execution {}.", execID); } @@ -563,18 +575,42 @@ public Map> * Merges all accumulator results from the tasks previously executed in the Executions. * @return The accumulator map */ - public Map> aggregateUserAccumulators() { + public Map> aggregateSmallUserAccumulators() { - Map> userAccumulators = new HashMap>(); + Map> smallUserAccumulators = new HashMap>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { - Map> next = vertex.getCurrentExecutionAttempt().getUserAccumulators(); + Map> next = vertex.getCurrentExecutionAttempt().getSmallUserAccumulators(); if (next != null) { - AccumulatorHelper.mergeInto(userAccumulators, next); + AccumulatorHelper.mergeInto(smallUserAccumulators, next); } } + return smallUserAccumulators; + } - return userAccumulators; + /** + * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the + * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, + * thus had to be sent through the BlobCache. + * @return The accumulator map + */ + public Map> aggregateLargeUserAccumulatorBlobKeys() { + Map> largeUserAccumulatorRefs = new HashMap>(); + + for (ExecutionVertex vertex : getAllExecutionVertices()) { + Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); + if (next != null) { + for (Map.Entry> otherEntry : next.entrySet()) { + List existing = largeUserAccumulatorRefs.get(otherEntry.getKey()); + if (existing == null) { + largeUserAccumulatorRefs.put(otherEntry.getKey(), otherEntry.getValue()); + } else { + existing.addAll(otherEntry.getValue()); + } + } + } + } + return largeUserAccumulatorRefs; } /** @@ -582,9 +618,9 @@ public Map> aggregateUserAccumulators() { * @return The accumulator map with serialized accumulator values. * @throws IOException */ - public Map> getAccumulatorsSerialized() throws IOException { + public Map> getSmallAccumulatorsSerialized() throws IOException { - Map> accumulatorMap = aggregateUserAccumulators(); + Map> accumulatorMap = aggregateSmallUserAccumulators(); Map> result = new HashMap>(); for (Map.Entry> entry : accumulatorMap.entrySet()) { @@ -599,14 +635,18 @@ public Map> getAccumulatorsSerialized() throws I * @return an Array containing the StringifiedAccumulatorResult objects */ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { + Map> smallAccumulatorMap = aggregateSmallUserAccumulators(); + Map> largeAccumulatorMap = aggregateLargeUserAccumulatorBlobKeys(); - Map> accumulatorMap = aggregateUserAccumulators(); + // get the total number of (unique) accumulators + Set uniqAccumulators = new HashSet(); + uniqAccumulators.addAll(smallAccumulatorMap.keySet()); + uniqAccumulators.addAll(largeAccumulatorMap.keySet()); + int num = uniqAccumulators.size(); - int num = accumulatorMap.size(); StringifiedAccumulatorResult[] resultStrings = new StringifiedAccumulatorResult[num]; - int i = 0; - for (Map.Entry> entry : accumulatorMap.entrySet()) { + for (Map.Entry> entry : smallAccumulatorMap.entrySet()) { StringifiedAccumulatorResult result; Accumulator value = entry.getValue(); @@ -619,6 +659,21 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { resultStrings[i++] = result; } + for (Map.Entry> entry : largeAccumulatorMap.entrySet()) { + + if(!smallAccumulatorMap.containsKey(entry.getKey())) { + StringBuilder str = new StringBuilder(); + str.append("BlobKeys=[ "); + for (BlobKey bk : entry.getValue()) { + str.append(bk + " "); + } + str.append("]"); + + StringifiedAccumulatorResult result = + new StringifiedAccumulatorResult(entry.getKey(), "Blob/Serialized", str.toString()); + resultStrings[i++] = result; + } + } return resultStrings; } @@ -941,18 +996,27 @@ public boolean updateState(TaskExecutionState state) { case RUNNING: return attempt.switchToRunning(); case FINISHED: + BaseAccumulatorSnapshot accumulatorSnapshot = state.getAccumulators(); + Map> flinkAccumulators = null; - Map> userAccumulators = null; + Map> smallUserAccumulators = null; + Map> largeUserAccumulators = null; try { - AccumulatorSnapshot accumulators = state.getAccumulators(); - flinkAccumulators = accumulators.deserializeFlinkAccumulators(); - userAccumulators = accumulators.deserializeUserAccumulators(userClassLoader); + flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); + + if(accumulatorSnapshot instanceof SmallAccumulatorSnapshot) { + smallUserAccumulators = ((SmallAccumulatorSnapshot) accumulatorSnapshot). + deserializeSmallUserAccumulators(userClassLoader); + } else if(accumulatorSnapshot instanceof LargeAccumulatorSnapshot) { + largeUserAccumulators = ((LargeAccumulatorSnapshot) accumulatorSnapshot). + getLargeAccumulatorBlobKeys(); + } } catch (Exception e) { // Exceptions would be thrown in the future here LOG.error("Failed to deserialize final accumulator results.", e); } - attempt.markFinished(flinkAccumulators, userAccumulators); + attempt.markFinished(flinkAccumulators, smallUserAccumulators, largeUserAccumulators); return true; case CANCELED: attempt.cancelingComplete(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index c4f62fbcb4c21..f02006c596912 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -59,8 +59,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; +import scala.runtime.AbstractFunction0; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -249,7 +251,6 @@ public Task(TaskDeploymentDescriptor tdd, this.memoryManager = checkNotNull(memManager); this.ioManager = checkNotNull(ioManager); this.broadcastVariableManager = checkNotNull(bcVarManager); - this.accumulatorRegistry = new AccumulatorRegistry(jobId, executionId); this.jobManager = checkNotNull(jobManagerActor); this.taskManager = checkNotNull(taskManagerActor); @@ -259,6 +260,10 @@ public Task(TaskDeploymentDescriptor tdd, this.fileCache = checkNotNull(fileCache); this.network = checkNotNull(networkEnvironment); + + this.accumulatorRegistry = new AccumulatorRegistry(tdd.getJobConfiguration(), + jobId, executionId, getBlobCacheServerAddress()); + this.executionListenerActors = new CopyOnWriteArrayList(); // create the reader and writer structures @@ -309,6 +314,27 @@ public Task(TaskDeploymentDescriptor tdd, executingThread = new Thread(TASK_THREADS_GROUP, this, taskNameWithSubtask); } + /** + * Gets the address where the blobCache is listening to. + * @return the address where the blobCache is listening to. + * */ + private InetSocketAddress getBlobCacheServerAddress() { + if(jobManager == null || libraryCache == null) { + throw new RuntimeException("TaskManager not associated to JobManager."); + } + + String jmHost = this.jobManager.actor().path().address().host().getOrElse( + new AbstractFunction0() { + @Override + public String apply() { + return "localhost"; + } + }); + + int blobPort = this.libraryCache.getBlobServerPort(); + return new InetSocketAddress(jmHost, blobPort); + } + // ------------------------------------------------------------------------ // Accessors // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java index 063701752c175..fac573aa861eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java @@ -20,7 +20,7 @@ import java.util.Arrays; -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.api.common.JobID; @@ -54,7 +54,7 @@ public class TaskExecutionState implements java.io.Serializable { private transient Throwable cachedError; /** Serialized flink and user-defined accumulators */ - private final AccumulatorSnapshot accumulators; + private final BaseAccumulatorSnapshot accumulators; /** * Creates a new task execution state update, with no attached exception and no accumulators. @@ -102,7 +102,7 @@ public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, */ public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState executionState, Throwable error, - AccumulatorSnapshot accumulators) { + BaseAccumulatorSnapshot accumulators) { if (jobID == null || executionId == null || executionState == null) { @@ -205,7 +205,7 @@ public JobID getJobID() { /** * Gets flink and user-defined accumulators in serialized form. */ - public AccumulatorSnapshot getAccumulators() { + public BaseAccumulatorSnapshot getAccumulators() { return accumulators; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java index 6a5468a5355ed..8fdf5351d1bc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java @@ -47,6 +47,9 @@ public SerializedValue(T value) throws IOException { this.serializedData = value == null ? null : InstantiationUtil.serializeObject(value); } + public SerializedValue(byte[] data) { + this.serializedData = data == null ? null : data; + } @SuppressWarnings("unchecked") public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundException { @@ -57,6 +60,17 @@ public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundE return serializedData == null ? null : (T) InstantiationUtil.deserializeObject(serializedData, loader); } + /** + * Gets the size of the serialized content. + * @return the size in bytes of the data stored in the blob. + * */ + public long getSizeInBytes() { + if(serializedData == null) { + throw new RuntimeException("No data in this Blob."); + } + return serializedData.length; + } + // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index b8cce41fe9109..1fcadadb175a6 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -421,4 +421,9 @@ object AkkaUtils { val duration = Duration(ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT) new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) } + + def getFramesize(config: Configuration): Long = config.getString( + ConfigConstants.AKKA_FRAMESIZE, + ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d.]", "").toLong + } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 7bf4447f6bc28..2de2d646e17d4 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -19,27 +19,33 @@ package org.apache.flink.runtime.jobmanager import java.io.{File, IOException} -import java.lang.reflect.{InvocationTargetException, Constructor} +import java.lang.reflect.{Constructor, InvocationTargetException} import java.net.InetSocketAddress -import java.util.{UUID, Collections} +import java.util.{Collections, UUID} +import _root_.akka.pattern.ask import akka.actor.Status.{Failure, Success} import akka.actor._ -import _root_.akka.pattern.ask - import grizzled.slf4j.Logger - import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot -import org.apache.flink.runtime.blob.BlobServer +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.blob.{BlobKey, BlobServer} import org.apache.flink.runtime.client._ +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex} +import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager} +import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID} +import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged +import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect} +import org.apache.flink.runtime.messages.RegistrationMessages._ +import org.apache.flink.runtime.messages.TaskManagerMessages.{Heartbeat, SendStackTrace} import org.apache.flink.runtime.messages.TaskMessages.{PartitionState, UpdateTaskExecutionState} import org.apache.flink.runtime.messages.accumulators._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint} @@ -48,28 +54,16 @@ import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner import org.apache.flink.runtime.taskmanager.TaskManager -import org.apache.flink.runtime.util.ZooKeeperUtil -import org.apache.flink.runtime.util.{SerializedValue, EnvironmentInformation} +import org.apache.flink.runtime.util.{EnvironmentInformation, SerializedValue, ZooKeeperUtil} import org.apache.flink.runtime.webmonitor.WebMonitor -import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessages} -import org.apache.flink.runtime.{LogMessages} -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager -import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager} -import org.apache.flink.runtime.jobgraph.{JobVertexID, JobGraph, JobStatus} -import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} -import org.apache.flink.runtime.messages.JobManagerMessages._ -import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.TaskManagerMessages.{SendStackTrace, Heartbeat} +import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.util.{ExceptionUtils, InstantiationUtil} -import _root_.akka.pattern.ask +import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps -import scala.collection.JavaConverters._ -import scala.collection.JavaConversions._ /** * The job manager is responsible for receiving Flink jobs, scheduling the tasks, gathering the @@ -338,16 +332,21 @@ class JobManager( // is the client waiting for the job result? newJobStatus match { case JobStatus.FINISHED => - val accumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { - executionGraph.getAccumulatorsSerialized + + val smallAccumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { + executionGraph.getSmallAccumulatorsSerialized } catch { case e: Exception => log.error(s"Cannot fetch serialized accumulators for job $jobID", e) Collections.emptyMap() } - val result = new SerializedJobExecutionResult(jobID, jobInfo.duration, - accumulatorResults) - jobInfo.client ! decorateMessage(JobResultSuccess(result)) + + val largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = + executionGraph.aggregateLargeUserAccumulatorBlobKeys() + + val result = new SerializedJobExecutionResult(jobID, + jobInfo.duration, smallAccumulatorResults, largeAccumulatorResults) + jobInfo.client ! JobResultSuccess(result) case JobStatus.CANCELED => jobInfo.client ! decorateMessage( @@ -770,8 +769,8 @@ class JobManager( try { currentJobs.get(jobID) match { case Some((graph, jobInfo)) => - val accumulatorValues = graph.getAccumulatorsSerialized() - sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) + val accumulatorValues = graph.getSmallAccumulatorsSerialized + sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) case None => archive.forward(message) } @@ -888,7 +887,7 @@ class JobManager( val finished = new java.util.ArrayList[JobID]() val canceled = new java.util.ArrayList[JobID]() val failed = new java.util.ArrayList[JobID]() - + currentJobs.values.foreach { case (graph, _) => graph.getState() match { case JobStatus.FINISHED => finished.add(graph.getJobID) @@ -932,7 +931,7 @@ class JobManager( * Updates the accumulators reported from a task manager via the Heartbeat message. * @param accumulators list of accumulator snapshots */ - private def updateAccumulators(accumulators : Seq[AccumulatorSnapshot]) = { + private def updateAccumulators(accumulators : Seq[BaseAccumulatorSnapshot]) = { accumulators foreach { case accumulatorEvent => currentJobs.get(accumulatorEvent.getJobID) match { @@ -1131,7 +1130,7 @@ object JobManager { // start the job manager web frontend if (configuration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) { LOG.info("Starting NEW JobManger web frontend") - + // start the new web frontend. we need to load this dynamically // because it is not in the same project/dependencies startWebRuntimeMonitor(configuration, jobManager, archiver) @@ -1565,10 +1564,10 @@ object JobManager { /** * Starts the web runtime monitor. Because the actual implementation of the * runtime monitor is in another project, we load the runtime monitor dynamically. - * + * * Because failure to start the web runtime monitor is not considered fatal, * this method does not throw any exceptions, but only logs them. - * + * * @param config The configuration for the runtime monitor. * @param jobManager The JobManager actor. * @param archiver The execution graph archive actor. @@ -1582,7 +1581,7 @@ object JobManager { val classname = "org.apache.flink.runtime.webmonitor.WebRuntimeMonitor" val clazz: Class[_ <: WebMonitor] = Class.forName(classname) .asSubclass(classOf[WebMonitor]) - + val ctor: Constructor[_ <: WebMonitor] = clazz.getConstructor(classOf[Configuration], classOf[ActorRef], classOf[ActorRef]) @@ -1601,13 +1600,13 @@ object JobManager { LOG.error("Failed to instantiate web runtime monitor.", t) null } - + if (monitor != null) { try { monitor.start() } catch { - case e: Exception => + case e: Exception => LOG.error("Failed to start web runtime monitor", e) } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala index 9f228ed4da833..5b6ba24616fda 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala @@ -132,7 +132,7 @@ class MemoryArchivist(private val max_entries: Int) try { graphs.get(jobID) match { case Some(graph) => - val accumulatorValues = graph.getAccumulatorsSerialized() + val accumulatorValues = graph.getSmallAccumulatorsSerialized() sender() ! AccumulatorResultsFound(jobID, accumulatorValues) case None => sender() ! AccumulatorResultsNotFound(jobID) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala index 6cb571ce7c63e..482cb6995d3de 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala @@ -18,7 +18,7 @@ package org.apache.flink.runtime.messages -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot import org.apache.flink.runtime.instance.InstanceID /** @@ -56,7 +56,7 @@ object TaskManagerMessages { * @param accumulators Accumulators of tasks serialized as Tuple2[internal, user-defined] */ case class Heartbeat(instanceID: InstanceID, metricsReport: Array[Byte], - accumulators: Seq[AccumulatorSnapshot]) + accumulators: Seq[BaseAccumulatorSnapshot]) // -------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index f974946df7d2f..364e7a8dea8a7 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -19,6 +19,8 @@ package org.apache.flink.runtime.taskmanager import java.io.{File, IOException} +import java.lang.management.{ManagementFactory, OperatingSystemMXBean} +import java.lang.reflect.Method import java.net.{InetAddress, InetSocketAddress} import java.util.UUID import java.util.concurrent.TimeUnit @@ -31,18 +33,18 @@ import _root_.akka.util.Timeout import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.codahale.metrics.json.MetricsModule -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} - +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration.{Configuration, ConfigConstants, GlobalConfiguration, IllegalConfigurationException} -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage} import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.blob.{BlobService, BlobCache} +import org.apache.flink.runtime.blob.{BlobCache, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor} import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} @@ -56,20 +58,25 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.io.network.netty.NettyConfig import org.apache.flink.runtime.jobgraph.IntermediateDataSetID import org.apache.flink.runtime.jobmanager.JobManager -import org.apache.flink.runtime.memorymanager.{MemoryManager, DefaultMemoryManager} +import org.apache.flink.runtime.memorymanager.{DefaultMemoryManager, MemoryManager} import org.apache.flink.runtime.messages.Messages._ import org.apache.flink.runtime.messages.RegistrationMessages._ import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ +import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} import org.apache.flink.runtime.net.NetUtils import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner -import org.apache.flink.runtime.util.{ZooKeeperUtil, MathUtils, EnvironmentInformation} +import org.apache.flink.runtime.util.{EnvironmentInformation, MathUtils, ZooKeeperUtil} +import org.apache.flink.runtime.StreamingMode +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool +import scala.language.postfixOps import scala.util.{Failure, Success} import scala.collection.JavaConverters._ import scala.collection.JavaConversions._ @@ -1046,7 +1053,7 @@ class TaskManager( val metricsReport: Array[Byte] = metricRegistryMapper.writeValueAsBytes(metricRegistry) val accumulatorEvents = - scala.collection.mutable.Buffer[AccumulatorSnapshot]() + scala.collection.mutable.Buffer[BaseAccumulatorSnapshot]() runningTasks foreach { case (execID, task) => diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java index 5c9ffa7c6747f..ee577fefff3c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java @@ -47,7 +47,7 @@ public void testSerialization() { origMap.put("name1", new SerializedValue(723L)); origMap.put("name2", new SerializedValue("peter")); - SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap); + SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap, null); // serialize and deserialize the object SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); @@ -84,12 +84,13 @@ public void testSerialization() { @Test public void testSerializationWithNullValues() { try { - SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null); + SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null, null); SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); assertNull(cloned.getJobId()); assertEquals(0L, cloned.getNetRuntime()); assertNull(cloned.getSerializedAccumulatorResults()); + assertNull(cloned.getBlobKeysToLargeAccumulators()); JobExecutionResult jResult = result.toJobExecutionResult(getClass().getClassLoader()); assertNull(jResult.getJobID()); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 8a7297b30370a..19467b8de1c04 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -181,7 +181,7 @@ trait TestingJobManager extends FlinkActor { currentJobs.get(jobID) match { case Some((graph, jobInfo)) => val flinkAccumulators = graph.getFlinkAccumulators - val userAccumulators = graph.aggregateUserAccumulators + val userAccumulators = graph.aggregateSmallUserAccumulators actors foreach { actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators) } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 01e6f624cda04..31f406d46bfd7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -44,7 +44,8 @@ * - when null records are passed through the system. * - when disjoint dataflows are executed * - when accumulators are used chained after a non-udf operator. - * + * - when an accumulator is bigger than the akka.framesize. + * * The tests are bundled into one class to reuse the same test cluster. This speeds * up test execution, as the majority of the test time goes usually into starting/stopping the * test cluster. @@ -61,6 +62,8 @@ public static void startCluster() { config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + config.setString(ConfigConstants.AKKA_FRAMESIZE, ConfigConstants.DEFAULT_AKKA_FRAMESIZE); + cluster = new ForkableFlinkMiniCluster(config, false); } catch (Exception e) { @@ -174,4 +177,28 @@ public void flatMap(Long value, Collector out) { fail(e.getMessage()); } } + + @Test + public void testOversizedAccumulators() { + try { + + ExecutionEnvironment env = + ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + + int noOfParallelism = 5; + int longsPerTask = 1200000; + + env.setParallelism(noOfParallelism); + env.getConfig().disableSysoutLogging(); + + DataSet bigEnough = env.generateSequence(1, noOfParallelism * longsPerTask); + long theCount = bigEnough.collect().size(); + + assertEquals(noOfParallelism * longsPerTask, theCount); + + }catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } } From 34d3e433eb0ce976539de166288550c9c7612eb4 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 22 Jul 2015 20:53:11 +0300 Subject: [PATCH 002/175] Solved the #887 issue: removing the akka.framesize size limitation for the result of a job. --- .../apache/flink/client/program/Client.java | 5 +- .../accumulators/AccumulatorHelper.java | 17 +++ flink-examples/flink-java-examples/pom.xml | 2 +- .../accumulators/AccumulatorRegistry.java | 93 ++++++++++++- ...shot.java => BaseAccumulatorSnapshot.java} | 26 +--- .../LargeAccumulatorSnapshot.java | 58 ++++++++ .../SmallAccumulatorSnapshot.java | 54 ++++++++ .../runtime/blob/BlobServerProtocol.java | 2 +- .../flink/runtime/client/JobClient.java | 126 +++++++++++++++--- .../client/SerializedJobExecutionResult.java | 95 +++++++++++-- .../librarycache/BlobLibraryCacheManager.java | 1 + .../FallbackLibraryCacheManager.java | 5 + .../librarycache/LibraryCacheManager.java | 7 + .../runtime/executiongraph/Execution.java | 37 +++-- .../executiongraph/ExecutionGraph.java | 106 ++++++++++++--- .../flink/runtime/taskmanager/Task.java | 28 +++- .../taskmanager/TaskExecutionState.java | 8 +- .../flink/runtime/util/SerializedValue.java | 14 ++ .../apache/flink/runtime/akka/AkkaUtils.scala | 5 + .../flink/runtime/jobmanager/JobManager.scala | 73 +++++----- .../runtime/jobmanager/MemoryArchivist.scala | 2 +- .../messages/TaskManagerMessages.scala | 4 +- .../runtime/taskmanager/TaskManager.scala | 21 ++- .../SerializedJobExecutionResultTest.java | 5 +- .../testingUtils/TestingJobManager.scala | 2 +- .../test/misc/MiscellaneousIssuesITCase.java | 29 +++- 26 files changed, 681 insertions(+), 144 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/{AccumulatorSnapshot.java => BaseAccumulatorSnapshot.java} (67%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 50590dfb61a5d..19fc365f60784 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -412,10 +412,11 @@ public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramIn try{ if (wait) { - SerializedJobExecutionResult result = JobClient.submitJobAndWait(actorSystem, + SerializedJobExecutionResult partialResult = JobClient.submitJobAndWait(actorSystem, jobManagerGateway, jobGraph, timeout, printStatusDuringExecution); try { - return result.toJobExecutionResult(this.userCodeClassLoader); + return JobClient.returnFinalJobExecutionResult(jobManagerGateway, partialResult, + this.userCodeClassLoader, timeout); } catch (Exception e) { throw new ProgramInvocationException( diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java index 3907004369f66..16ca096df9912 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java @@ -51,6 +51,23 @@ public static void mergeInto(Map> target, Map target, Accumulator toMerge) { + // Both should have the same type + AccumulatorHelper.compareAccumulatorTypes(name, target.getClass(), toMerge.getClass()); + // Merge counter from chained task into counter from stub + mergeSingle(target, toMerge); + } + /** * Workaround method for type safety */ diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml index 8448a93db434d..33190abbf9563 100644 --- a/flink-examples/flink-java-examples/pom.xml +++ b/flink-examples/flink-java-examples/pom.xml @@ -319,7 +319,7 @@ under the License. - + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java index 0ef3650292b48..6d53f2045f697 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java @@ -21,13 +21,22 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.util.SerializedValue; +import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; @@ -38,6 +47,8 @@ public class AccumulatorRegistry { protected static final Logger LOG = LoggerFactory.getLogger(AccumulatorRegistry.class); + protected final Configuration jobConfiguration; + protected final InetSocketAddress blobServerAddress; protected final JobID jobID; protected final ExecutionAttemptID taskID; @@ -62,26 +73,100 @@ public enum Metric { NUM_BYTES_OUT } - public AccumulatorRegistry(JobID jobID, ExecutionAttemptID taskID) { + this(null, jobID, taskID, null); + } + + public AccumulatorRegistry(Configuration jobConfig, JobID jobID, ExecutionAttemptID taskID, InetSocketAddress blobServerAddress) { + this.jobConfiguration = jobConfig; this.jobID = jobID; this.taskID = taskID; + this.blobServerAddress = blobServerAddress; this.reporter = new ReadWriteReporter(flinkAccumulators); } /** - * Creates a snapshot of this accumulator registry. + * Creates a snapshot of this accumulator registry. If they are oversized (i.e. bigger than + * akka.framesize), this method stores them in the BlobCache and sends only the + * corresponding BlobKeys in the final snapshot. If they are not, it sends the actual + * accumulators in the accumulator snapshot. + * * @return a serialized accumulator map */ - public AccumulatorSnapshot getSnapshot() { + public BaseAccumulatorSnapshot getSnapshot() { + BaseAccumulatorSnapshot snapshot; + Map> largeAccumulatorBlobKeys; + SerializedValue>> serializedAccumulators; + try { - return new AccumulatorSnapshot(jobID, taskID, flinkAccumulators, userAccumulators); + serializedAccumulators = new SerializedValue>>(userAccumulators); + if (serializedAccumulators.getSizeInBytes() > 0.8 * AkkaUtils.getFramesize(jobConfiguration)) { + + largeAccumulatorBlobKeys = storeAccumulatorsToBlobCache(blobServerAddress, userAccumulators); + snapshot = new LargeAccumulatorSnapshot(jobID, taskID, + flinkAccumulators, largeAccumulatorBlobKeys); + + } else { + snapshot = new SmallAccumulatorSnapshot(jobID, taskID, + flinkAccumulators, serializedAccumulators); + } + return snapshot; } catch (IOException e) { LOG.warn("Failed to serialize accumulators for task.", e); return null; } } + /** + * Puts the blobs of the large accumulators on the BlobCache. + * @param blobServerAddress the address of the server to the blobCache. + * @param accumulators the accumulators to be stored in the cache. + * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. + * */ + private Map> storeAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, + Map> accumulators) { + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> keys = new HashMap>(); + BlobClient bc = null; + try { + bc = new BlobClient(blobServerAddress); + + for (Map.Entry> entry : accumulators.entrySet()) { + + String accumulatorName = entry.getKey(); + Accumulator accumulator = entry.getValue(); + + byte[] serializedAccumulator = InstantiationUtil.serializeObject(accumulator); + BlobKey blobKey = bc.put(serializedAccumulator); + + List accKeys = keys.get(accumulatorName); + if (accKeys == null) { + accKeys = new ArrayList(); + } + accKeys.add(blobKey); + keys.put(accumulatorName, accKeys); + } + } catch (IOException e) { + LOG.error("Failed to send oversized accumulators to the BlobCache: ", e); + } finally { + try { + if(bc != null) { + bc.close(); + } + } catch (IOException e) { + LOG.error("Failed to close BlobClient: ", e); + } + + } + return keys; + } + /** * Gets the map for user-defined accumulators. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java similarity index 67% rename from flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java index 0f1911db2f137..09bed90dfb763 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java @@ -28,11 +28,10 @@ import java.util.Map; /** - * This class encapsulates a map of accumulators for a single task. It is used - * for the transfer from TaskManagers to the JobManager and from the JobManager - * to the Client. + * LICENSE SHOULD GO HERE. + * Created by kkloudas on 7/23/15. */ -public class AccumulatorSnapshot implements Serializable { +public class BaseAccumulatorSnapshot implements Serializable { private static final long serialVersionUID = 42L; @@ -44,18 +43,11 @@ public class AccumulatorSnapshot implements Serializable { */ private final SerializedValue>> flinkAccumulators; - /** - * Serialized user accumulators which may require the custom user class loader. - */ - private final SerializedValue>> userAccumulators; - - public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, - Map> flinkAccumulators, - Map> userAccumulators) throws IOException { + public BaseAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators) throws IOException { this.jobID = jobID; this.executionAttemptID = executionAttemptID; this.flinkAccumulators = new SerializedValue>>(flinkAccumulators); - this.userAccumulators = new SerializedValue>>(userAccumulators); } public JobID getJobID() { @@ -73,12 +65,4 @@ public ExecutionAttemptID getExecutionAttemptID() { public Map> deserializeFlinkAccumulators() throws IOException, ClassNotFoundException { return flinkAccumulators.deserializeValue(ClassLoader.getSystemClassLoader()); } - - /** - * Gets the user-defined accumulators values. - * @return the serialized map - */ - public Map> deserializeUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { - return userAccumulators.deserializeValue(classLoader); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java new file mode 100644 index 0000000000000..a4283f165cc26 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java @@ -0,0 +1,58 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * LICENSE SHOULD GO HERE. + * Created by kkloudas on 7/23/15. + */ +public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { + + /** + * In case some accumulators do not fit in akka, we store them in the blobCache and put + * in the snapshot only the mapping between the name of the accumulator, and its blobKey + * in the cache. This list holds exactly this mapping. + * */ + private final Map> largeUserAccumulatorBlobs; + + public LargeAccumulatorSnapshot( + JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + Map> oversizedUserAccumulatorBlobKeys) throws IOException { + super(jobID, executionAttemptID, flinkAccumulators); + this.largeUserAccumulatorBlobs = oversizedUserAccumulatorBlobKeys; + } + + /** + * Gets the Flink (internal) accumulators values. + * @return the serialized map + */ + public Map> getLargeAccumulatorBlobKeys() { + return largeUserAccumulatorBlobs; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java new file mode 100644 index 0000000000000..1911ebac6c321 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java @@ -0,0 +1,54 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.util.SerializedValue; + +import java.io.IOException; +import java.util.Map; + +/** + * LICENSE SHOULD GO HERE. + * Created by kkloudas on 7/23/15. + */ +public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { + + /** + * Serialized user accumulators which may require the custom user class loader. + */ + private final SerializedValue>> userAccumulators; + + public SmallAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + SerializedValue>> userAccumulators) throws IOException { + super(jobID, executionAttemptID, flinkAccumulators); + this.userAccumulators = userAccumulators; + } + + /** + * Gets the user-defined accumulators values. + * @return the serialized map + */ + public Map> deserializeSmallUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { + return userAccumulators.deserializeValue(classLoader); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java index 6df78110b9a2c..c5bcce52b1594 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java @@ -24,7 +24,7 @@ public class BlobServerProtocol { // -------------------------------------------------------------------------------------------- /** The buffer size in bytes for network transfers. */ - static final int BUFFER_SIZE = 65536; // 64 K + public static final int BUFFER_SIZE = 65536; // 64 K /** The maximum key length allowed for storing BLOBs. */ static final int MAX_KEY_LENGTH = 64; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index aeefa612ba4f9..3b64b8293b96a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -27,15 +27,20 @@ import akka.pattern.Patterns; import akka.util.Timeout; import com.google.common.base.Preconditions; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.blob.BlobServerProtocol; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.JobClientMessages; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +51,17 @@ import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeoutException; /** @@ -187,9 +199,29 @@ public static SerializedJobExecutionResult submitJobAndWait( } } + public static JobExecutionResult returnFinalJobExecutionResult( + ActorGateway jobManagerGateway, SerializedJobExecutionResult partialResult, + ClassLoader userCodeClassLoader, FiniteDuration timeout) + throws IOException, ClassNotFoundException { + + if (jobManagerGateway == null || partialResult == null || userCodeClassLoader == null || timeout == null) { + throw new NullPointerException(); + } + + Map> blobsToFetch = partialResult.getBlobKeysToLargeAccumulators(); + + Map>> accumulatorBlobs; + try { + accumulatorBlobs = getLargeAccumulatorBlobs(jobManagerGateway, blobsToFetch, timeout); + } catch (IOException e) { + throw new IOException("Failed to fetch the oversized accumulators from the BlobCache", e); + } + return partialResult.mergeToJobExecutionResult(userCodeClassLoader, accumulatorBlobs); + } + /** * Submits a job in detached mode. The method sends the JobGraph to the - * JobManager and waits for the answer whether teh job could be started or not. + * JobManager and waits for the answer whether the job could be started or not. * * @param jobManagerGateway Gateway to the JobManager which will execute the jobs * @param jobGraph The job @@ -250,29 +282,87 @@ public static void uploadJarFiles( FiniteDuration timeout) throws IOException { if (jobGraph.hasUsercodeJarFiles()) { + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + jobGraph.uploadRequiredJarFiles(serverAddress); + } + } - Future futureBlobPort = jobManagerGateway.ask( - JobManagerMessages.getRequestBlobManagerPort(), - timeout); + /** + * If the result of the job contained oversized (i.e. bigger that the akka.framesize) accumulators + * then these are put in the BlobCache for the client to fetch and merge. This method gets + * them from the BlobCache (if there was any). If the list of blobs to fetch is empty, then + * an empty result is returned, as all (partial) accumulators were small enough to be sent + * directly to the JobManager and be merged there. + * + * @param jobManagerGateway the reference to the jobManager actor. + * @param keys the accumulators to fetch (based on their name) along with their associated BlobKeys. + * @param timeout the timeout to wait for the connection to the blob server. + * @return the serialized accumulators, grouped by name. + * */ + private static Map>> getLargeAccumulatorBlobs( + ActorGateway jobManagerGateway, Map> keys, FiniteDuration timeout) throws IOException { + + if (keys.isEmpty()) { + return Collections.emptyMap(); + } - int port; - try { - Object result = Await.result(futureBlobPort, timeout); - if (result instanceof Integer) { - port = (Integer) result; - } else { - throw new Exception("Expected port number (int) as answer, received " + result); + Map>> accumulatorBlobs = + new HashMap>>(); + + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + BlobClient bc = new BlobClient(serverAddress); + + final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE]; + for(String accName: keys.keySet()) { + List accBlobKeys = keys.get(accName); + List> accBlobs = new ArrayList>(); + + for(BlobKey bk: accBlobKeys) { + InputStream is = bc.get(bk); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + while (true) { + final int read = is.read(buf); + if (read < 0) { + break; + } + os.write(buf, 0, read); } + os.flush(); + byte[] blob = os.toByteArray(); + accBlobs.add(new SerializedValue(blob)); + is.close(); + os.close(); + + // after getting them, clean up and delete the blobs from the BlobCache. + bc.delete(bk); } - catch (Exception e) { - throw new IOException("Could not retrieve the JobManager's blob port.", e); - } + accumulatorBlobs.put(accName, accBlobs); + } + bc.close(); + return accumulatorBlobs; + } - Option jmHost = jobManagerGateway.actor().path().address().host(); - String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; - InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, port); + private static InetSocketAddress getBlobServerAddress(ActorGateway jobManagerGateway, FiniteDuration timeout) throws IOException { + Timeout tOut = new Timeout(timeout); + Future futureBlobPort = jobManagerGateway.ask( + JobManagerMessages.getRequestBlobManagerPort(), + timeout); - jobGraph.uploadRequiredJarFiles(serverAddress); + int port; + try { + Object result = Await.result(futureBlobPort, timeout); + if (result instanceof Integer) { + port = (Integer) result; + } else { + throw new Exception("Expected port number (int) as answer, received " + result); + } + } catch (Exception e) { + throw new IOException("Could not retrieve the JobManager's blob port.", e); } + + Option jmHost = jobManagerGateway.actor().path().address().host(); + String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; + InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, port); + return serverAddress; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java index 029bc3876f738..0b1115dbe326a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java @@ -20,11 +20,15 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.util.SerializedValue; import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -38,7 +42,13 @@ public class SerializedJobExecutionResult implements java.io.Serializable { private final JobID jobId; - private final Map> accumulatorResults; + // holds the result value of an accumulator + private final Map> smallAccumulatorResults; + + // holds the BlobKeys pointing to the blobs in the BlobCache + // containing the serialized accumulators + private final Map> largeAccumulatorBlobRefs; + private final long netRuntime; @@ -47,13 +57,17 @@ public class SerializedJobExecutionResult implements java.io.Serializable { * * @param jobID The job's ID. * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer) in milliseconds - * @param accumulators A map of all accumulator results produced by the job, in serialized form + * @param smallAccumulators A map of all SMALL accumulator results produced by the job, in serialized form + * @param largeAccumulatorBlobRefs A map of all the references to the blobs holding the LARGE accumulators that had + * to be sent through the BlobCache. */ public SerializedJobExecutionResult(JobID jobID, long netRuntime, - Map> accumulators) { + Map> smallAccumulators, + Map> largeAccumulatorBlobRefs) { this.jobId = jobID; this.netRuntime = netRuntime; - this.accumulatorResults = accumulators; + this.smallAccumulatorResults = smallAccumulators; + this.largeAccumulatorBlobRefs = largeAccumulatorBlobRefs; } public JobID getJobId() { @@ -76,17 +90,28 @@ public long getNetRuntime(TimeUnit desiredUnit) { } public Map> getSerializedAccumulatorResults() { - return this.accumulatorResults; + return this.smallAccumulatorResults; + } + + /** + * If the result of the job contained oversized (i.e. bigger that the akka.framesize) accumulators + * then these are put in the BlobCache for the client to fetch and merge. This method gets + * their BlobKeys. If the result is empty, then this means that all (partial) accumulators + * were small enough to be sent directly to the JobManager and be merged there. + * @return the BlobKeys to the blobs containing the oversized accumulators. + * */ + public Map> getBlobKeysToLargeAccumulators() { + return this.largeAccumulatorBlobRefs; } public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOException, ClassNotFoundException { Map accumulators = null; - if (accumulatorResults != null) { - accumulators = accumulatorResults.isEmpty() ? + if (smallAccumulatorResults != null) { + accumulators = smallAccumulatorResults.isEmpty() ? Collections.emptyMap() : - new HashMap(this.accumulatorResults.size()); + new HashMap(this.smallAccumulatorResults.size()); - for (Map.Entry> entry : this.accumulatorResults.entrySet()) { + for (Map.Entry> entry : this.smallAccumulatorResults.entrySet()) { Object o = entry.getValue() == null ? null : entry.getValue().deserializeValue(loader); accumulators.put(entry.getKey(), o); } @@ -94,4 +119,56 @@ public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOExce return new JobExecutionResult(jobId, netRuntime, accumulators); } + + /** + * Merges the data of the small accumulators in this class, with the accumulatorsToMerge, and + * returns the result in a JobExecutionResult. This method is used by the Client to merge + * the results received from the JobManager, with those in the oversized accumulators that were fetched + * from the BlobCache. + * @param loader + * the ClassLoader to used to deserialize the data and the Accumulators. + * @param accumulatorsToMerge + * the Accumulators received from the BlobCache as blobs, that are to be merged with local data. + * @return the final result after the merging of the different partial accumulators. + * */ + public JobExecutionResult mergeToJobExecutionResult(ClassLoader loader, Map>> accumulatorsToMerge) throws IOException, ClassNotFoundException { + if(accumulatorsToMerge == null || accumulatorsToMerge.isEmpty()) { + return toJobExecutionResult(loader); + } + + Map accumulators = new HashMap(); + + for(String name: accumulatorsToMerge.keySet()) { + List> blobs = accumulatorsToMerge.get(name); + + // merge the serialized accumulators + Accumulator acc = (Accumulator) blobs.get(0).deserializeValue(loader); + for(int i = 1; i < blobs.size(); i++) { + AccumulatorHelper.mergeAccumulators(name, acc, + (Accumulator) blobs.get(i).deserializeValue(loader)); + } + + // add also the data from the non-oversized (i.e. the ones that were sent through akka) + // accumulators, if any + if (smallAccumulatorResults != null) { + SerializedValue localObject = smallAccumulatorResults.remove(name); + if (localObject != null) { + acc.add(localObject.deserializeValue(loader)); + } + } + + // and put the data with the associated accumulator name to the list + accumulators.put(name, acc.getLocalValue()); + } + + // finally, put the remaining accumulators in the list. + if (smallAccumulatorResults != null) { + for (Map.Entry> entry : this.smallAccumulatorResults.entrySet()) { + Object o = entry.getValue() == null ? null : entry.getValue().deserializeValue(loader); + accumulators.put(entry.getKey(), o); + } + } + return new JobExecutionResult(jobId, netRuntime, accumulators); + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 88be5e1f37735..ad482cf4596fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -181,6 +181,7 @@ public File getFile(BlobKey blobKey) throws IOException { return new File(blobService.getURL(blobKey).getFile()); } + @Override public int getBlobServerPort() { return blobService.getPort(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java index 66bda45873014..b0d5077b1820f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java @@ -32,6 +32,11 @@ public class FallbackLibraryCacheManager implements LibraryCacheManager { private static Logger LOG = LoggerFactory.getLogger(FallbackLibraryCacheManager.class); + @Override + public int getBlobServerPort() { + return 0; + } + @Override public ClassLoader getClassLoader(JobID id) { return getClass().getClassLoader(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java index 52a8048867804..c1f4f365e6ada 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java @@ -27,6 +27,13 @@ import java.util.Collection; public interface LibraryCacheManager { + + /** + * Returns the port the service is listening to. + * @return the port of the cache server. + * */ + int getBlobServerPort(); + /** * Returns the user code class loader associated with id. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index aa0f98148548a..4c08a44462a04 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionLocation; @@ -140,7 +141,10 @@ public class Execution implements Serializable { private final Object accumulatorLock = new Object(); /* Continuously updated map of user-defined accumulators */ - private volatile Map> userAccumulators; + private volatile Map> smallUserAccumulators; + + /* Keep track of the large accumulators that are stored in the BlobCache */ + private volatile Map> largeUserAccumulators; /* Continuously updated map of internal accumulators */ private volatile Map> flinkAccumulators; @@ -605,12 +609,14 @@ void markFailed(Throwable t) { } void markFinished() { - markFinished(null, null); + markFinished(null, null, null); } - void markFinished(Map> flinkAccumulators, Map> userAccumulators) { + void markFinished(Map> flinkAccumulators, + Map> userAccumulators, + Map> userLargeAccumulators) { - // this call usually comes during RUNNING, but may also come while still in deploying (very fast tasks!) + // this call usually comes during RUNNING, but may also come while still in DEPLOYING (very fast tasks!) while (true) { ExecutionState current = this.state; @@ -631,7 +637,8 @@ void markFinished(Map> flinkAccumu synchronized (accumulatorLock) { this.flinkAccumulators = flinkAccumulators; - this.userAccumulators = userAccumulators; + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulators = userLargeAccumulators; } assignedResource.releaseSlot(); @@ -865,8 +872,7 @@ public void onComplete(Throwable failure, Object success) throws Throwable { } else { TaskOperationResult result = (TaskOperationResult) success; if (!result.success()) { - LOG.debug("Cancel task call did not find task. Probably akka message call" + - " race."); + LOG.debug("Cancel task call did not find task. Probably akka message call race."); } } } @@ -960,18 +966,27 @@ public String getVertexWithAttempt() { * Update accumulators (discarded when the Execution has already been terminated). * @param flinkAccumulators the flink internal accumulators * @param userAccumulators the user accumulators + * @param userLargeAccumulatorBlobKeys the keys to the blobs storing the oversized accumulators sent by + * (some) tasks. */ public void setAccumulators(Map> flinkAccumulators, - Map> userAccumulators) { + Map> userAccumulators, + Map> userLargeAccumulatorBlobKeys) { synchronized (accumulatorLock) { if (!state.isTerminal()) { this.flinkAccumulators = flinkAccumulators; - this.userAccumulators = userAccumulators; + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulators = userLargeAccumulatorBlobKeys; } } } - public Map> getUserAccumulators() { - return userAccumulators; + + public Map> getSmallUserAccumulators() { + return smallUserAccumulators; + } + + public Map> getLargeUserAccumulatorBlobKeys() { + return largeUserAccumulators; } public Map> getFlinkAccumulators() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 833518ce84513..788982eef8517 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -24,8 +24,10 @@ import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.LargeAccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.SmallAccumulatorSnapshot; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; @@ -60,6 +62,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.Set; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -142,17 +146,25 @@ public class ExecutionGraph implements Serializable { * through the UpdateTaskExecutionState message. * @param accumulatorSnapshot The serialized flink and user-defined accumulators */ - public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { + public void updateAccumulators(BaseAccumulatorSnapshot accumulatorSnapshot) { Map> flinkAccumulators; - Map> userAccumulators; + Map> smallUserAccumulators = null; + Map> largeUserAccumulators = null; try { flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - userAccumulators = accumulatorSnapshot.deserializeUserAccumulators(userClassLoader); + + if(accumulatorSnapshot instanceof SmallAccumulatorSnapshot) { + smallUserAccumulators = ((SmallAccumulatorSnapshot) accumulatorSnapshot). + deserializeSmallUserAccumulators(userClassLoader); + } else if(accumulatorSnapshot instanceof LargeAccumulatorSnapshot) { + largeUserAccumulators = ((LargeAccumulatorSnapshot) accumulatorSnapshot). + getLargeAccumulatorBlobKeys(); + } ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); Execution execution = currentExecutions.get(execID); if (execution != null) { - execution.setAccumulators(flinkAccumulators, userAccumulators); + execution.setAccumulators(flinkAccumulators, smallUserAccumulators, largeUserAccumulators); } else { LOG.warn("Received accumulator result for unknown execution {}.", execID); } @@ -563,18 +575,42 @@ public Map> * Merges all accumulator results from the tasks previously executed in the Executions. * @return The accumulator map */ - public Map> aggregateUserAccumulators() { + public Map> aggregateSmallUserAccumulators() { - Map> userAccumulators = new HashMap>(); + Map> smallUserAccumulators = new HashMap>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { - Map> next = vertex.getCurrentExecutionAttempt().getUserAccumulators(); + Map> next = vertex.getCurrentExecutionAttempt().getSmallUserAccumulators(); if (next != null) { - AccumulatorHelper.mergeInto(userAccumulators, next); + AccumulatorHelper.mergeInto(smallUserAccumulators, next); } } + return smallUserAccumulators; + } - return userAccumulators; + /** + * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the + * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, + * thus had to be sent through the BlobCache. + * @return The accumulator map + */ + public Map> aggregateLargeUserAccumulatorBlobKeys() { + Map> largeUserAccumulatorRefs = new HashMap>(); + + for (ExecutionVertex vertex : getAllExecutionVertices()) { + Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); + if (next != null) { + for (Map.Entry> otherEntry : next.entrySet()) { + List existing = largeUserAccumulatorRefs.get(otherEntry.getKey()); + if (existing == null) { + largeUserAccumulatorRefs.put(otherEntry.getKey(), otherEntry.getValue()); + } else { + existing.addAll(otherEntry.getValue()); + } + } + } + } + return largeUserAccumulatorRefs; } /** @@ -582,9 +618,9 @@ public Map> aggregateUserAccumulators() { * @return The accumulator map with serialized accumulator values. * @throws IOException */ - public Map> getAccumulatorsSerialized() throws IOException { + public Map> getSmallAccumulatorsSerialized() throws IOException { - Map> accumulatorMap = aggregateUserAccumulators(); + Map> accumulatorMap = aggregateSmallUserAccumulators(); Map> result = new HashMap>(); for (Map.Entry> entry : accumulatorMap.entrySet()) { @@ -599,14 +635,18 @@ public Map> getAccumulatorsSerialized() throws I * @return an Array containing the StringifiedAccumulatorResult objects */ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { + Map> smallAccumulatorMap = aggregateSmallUserAccumulators(); + Map> largeAccumulatorMap = aggregateLargeUserAccumulatorBlobKeys(); - Map> accumulatorMap = aggregateUserAccumulators(); + // get the total number of (unique) accumulators + Set uniqAccumulators = new HashSet(); + uniqAccumulators.addAll(smallAccumulatorMap.keySet()); + uniqAccumulators.addAll(largeAccumulatorMap.keySet()); + int num = uniqAccumulators.size(); - int num = accumulatorMap.size(); StringifiedAccumulatorResult[] resultStrings = new StringifiedAccumulatorResult[num]; - int i = 0; - for (Map.Entry> entry : accumulatorMap.entrySet()) { + for (Map.Entry> entry : smallAccumulatorMap.entrySet()) { StringifiedAccumulatorResult result; Accumulator value = entry.getValue(); @@ -619,6 +659,21 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { resultStrings[i++] = result; } + for (Map.Entry> entry : largeAccumulatorMap.entrySet()) { + + if(!smallAccumulatorMap.containsKey(entry.getKey())) { + StringBuilder str = new StringBuilder(); + str.append("BlobKeys=[ "); + for (BlobKey bk : entry.getValue()) { + str.append(bk + " "); + } + str.append("]"); + + StringifiedAccumulatorResult result = + new StringifiedAccumulatorResult(entry.getKey(), "Blob/Serialized", str.toString()); + resultStrings[i++] = result; + } + } return resultStrings; } @@ -941,18 +996,27 @@ public boolean updateState(TaskExecutionState state) { case RUNNING: return attempt.switchToRunning(); case FINISHED: + BaseAccumulatorSnapshot accumulatorSnapshot = state.getAccumulators(); + Map> flinkAccumulators = null; - Map> userAccumulators = null; + Map> smallUserAccumulators = null; + Map> largeUserAccumulators = null; try { - AccumulatorSnapshot accumulators = state.getAccumulators(); - flinkAccumulators = accumulators.deserializeFlinkAccumulators(); - userAccumulators = accumulators.deserializeUserAccumulators(userClassLoader); + flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); + + if(accumulatorSnapshot instanceof SmallAccumulatorSnapshot) { + smallUserAccumulators = ((SmallAccumulatorSnapshot) accumulatorSnapshot). + deserializeSmallUserAccumulators(userClassLoader); + } else if(accumulatorSnapshot instanceof LargeAccumulatorSnapshot) { + largeUserAccumulators = ((LargeAccumulatorSnapshot) accumulatorSnapshot). + getLargeAccumulatorBlobKeys(); + } } catch (Exception e) { // Exceptions would be thrown in the future here LOG.error("Failed to deserialize final accumulator results.", e); } - attempt.markFinished(flinkAccumulators, userAccumulators); + attempt.markFinished(flinkAccumulators, smallUserAccumulators, largeUserAccumulators); return true; case CANCELED: attempt.cancelingComplete(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index c4f62fbcb4c21..f02006c596912 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -59,8 +59,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; +import scala.runtime.AbstractFunction0; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -249,7 +251,6 @@ public Task(TaskDeploymentDescriptor tdd, this.memoryManager = checkNotNull(memManager); this.ioManager = checkNotNull(ioManager); this.broadcastVariableManager = checkNotNull(bcVarManager); - this.accumulatorRegistry = new AccumulatorRegistry(jobId, executionId); this.jobManager = checkNotNull(jobManagerActor); this.taskManager = checkNotNull(taskManagerActor); @@ -259,6 +260,10 @@ public Task(TaskDeploymentDescriptor tdd, this.fileCache = checkNotNull(fileCache); this.network = checkNotNull(networkEnvironment); + + this.accumulatorRegistry = new AccumulatorRegistry(tdd.getJobConfiguration(), + jobId, executionId, getBlobCacheServerAddress()); + this.executionListenerActors = new CopyOnWriteArrayList(); // create the reader and writer structures @@ -309,6 +314,27 @@ public Task(TaskDeploymentDescriptor tdd, executingThread = new Thread(TASK_THREADS_GROUP, this, taskNameWithSubtask); } + /** + * Gets the address where the blobCache is listening to. + * @return the address where the blobCache is listening to. + * */ + private InetSocketAddress getBlobCacheServerAddress() { + if(jobManager == null || libraryCache == null) { + throw new RuntimeException("TaskManager not associated to JobManager."); + } + + String jmHost = this.jobManager.actor().path().address().host().getOrElse( + new AbstractFunction0() { + @Override + public String apply() { + return "localhost"; + } + }); + + int blobPort = this.libraryCache.getBlobServerPort(); + return new InetSocketAddress(jmHost, blobPort); + } + // ------------------------------------------------------------------------ // Accessors // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java index 063701752c175..fac573aa861eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java @@ -20,7 +20,7 @@ import java.util.Arrays; -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.api.common.JobID; @@ -54,7 +54,7 @@ public class TaskExecutionState implements java.io.Serializable { private transient Throwable cachedError; /** Serialized flink and user-defined accumulators */ - private final AccumulatorSnapshot accumulators; + private final BaseAccumulatorSnapshot accumulators; /** * Creates a new task execution state update, with no attached exception and no accumulators. @@ -102,7 +102,7 @@ public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, */ public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState executionState, Throwable error, - AccumulatorSnapshot accumulators) { + BaseAccumulatorSnapshot accumulators) { if (jobID == null || executionId == null || executionState == null) { @@ -205,7 +205,7 @@ public JobID getJobID() { /** * Gets flink and user-defined accumulators in serialized form. */ - public AccumulatorSnapshot getAccumulators() { + public BaseAccumulatorSnapshot getAccumulators() { return accumulators; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java index 6a5468a5355ed..8fdf5351d1bc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java @@ -47,6 +47,9 @@ public SerializedValue(T value) throws IOException { this.serializedData = value == null ? null : InstantiationUtil.serializeObject(value); } + public SerializedValue(byte[] data) { + this.serializedData = data == null ? null : data; + } @SuppressWarnings("unchecked") public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundException { @@ -57,6 +60,17 @@ public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundE return serializedData == null ? null : (T) InstantiationUtil.deserializeObject(serializedData, loader); } + /** + * Gets the size of the serialized content. + * @return the size in bytes of the data stored in the blob. + * */ + public long getSizeInBytes() { + if(serializedData == null) { + throw new RuntimeException("No data in this Blob."); + } + return serializedData.length; + } + // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index b8cce41fe9109..1fcadadb175a6 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -421,4 +421,9 @@ object AkkaUtils { val duration = Duration(ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT) new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) } + + def getFramesize(config: Configuration): Long = config.getString( + ConfigConstants.AKKA_FRAMESIZE, + ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d.]", "").toLong + } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 7bf4447f6bc28..2de2d646e17d4 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -19,27 +19,33 @@ package org.apache.flink.runtime.jobmanager import java.io.{File, IOException} -import java.lang.reflect.{InvocationTargetException, Constructor} +import java.lang.reflect.{Constructor, InvocationTargetException} import java.net.InetSocketAddress -import java.util.{UUID, Collections} +import java.util.{Collections, UUID} +import _root_.akka.pattern.ask import akka.actor.Status.{Failure, Success} import akka.actor._ -import _root_.akka.pattern.ask - import grizzled.slf4j.Logger - import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot -import org.apache.flink.runtime.blob.BlobServer +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.blob.{BlobKey, BlobServer} import org.apache.flink.runtime.client._ +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex} +import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager} +import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID} +import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged +import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect} +import org.apache.flink.runtime.messages.RegistrationMessages._ +import org.apache.flink.runtime.messages.TaskManagerMessages.{Heartbeat, SendStackTrace} import org.apache.flink.runtime.messages.TaskMessages.{PartitionState, UpdateTaskExecutionState} import org.apache.flink.runtime.messages.accumulators._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint} @@ -48,28 +54,16 @@ import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner import org.apache.flink.runtime.taskmanager.TaskManager -import org.apache.flink.runtime.util.ZooKeeperUtil -import org.apache.flink.runtime.util.{SerializedValue, EnvironmentInformation} +import org.apache.flink.runtime.util.{EnvironmentInformation, SerializedValue, ZooKeeperUtil} import org.apache.flink.runtime.webmonitor.WebMonitor -import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessages} -import org.apache.flink.runtime.{LogMessages} -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager -import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager} -import org.apache.flink.runtime.jobgraph.{JobVertexID, JobGraph, JobStatus} -import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} -import org.apache.flink.runtime.messages.JobManagerMessages._ -import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.TaskManagerMessages.{SendStackTrace, Heartbeat} +import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.util.{ExceptionUtils, InstantiationUtil} -import _root_.akka.pattern.ask +import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps -import scala.collection.JavaConverters._ -import scala.collection.JavaConversions._ /** * The job manager is responsible for receiving Flink jobs, scheduling the tasks, gathering the @@ -338,16 +332,21 @@ class JobManager( // is the client waiting for the job result? newJobStatus match { case JobStatus.FINISHED => - val accumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { - executionGraph.getAccumulatorsSerialized + + val smallAccumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { + executionGraph.getSmallAccumulatorsSerialized } catch { case e: Exception => log.error(s"Cannot fetch serialized accumulators for job $jobID", e) Collections.emptyMap() } - val result = new SerializedJobExecutionResult(jobID, jobInfo.duration, - accumulatorResults) - jobInfo.client ! decorateMessage(JobResultSuccess(result)) + + val largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = + executionGraph.aggregateLargeUserAccumulatorBlobKeys() + + val result = new SerializedJobExecutionResult(jobID, + jobInfo.duration, smallAccumulatorResults, largeAccumulatorResults) + jobInfo.client ! JobResultSuccess(result) case JobStatus.CANCELED => jobInfo.client ! decorateMessage( @@ -770,8 +769,8 @@ class JobManager( try { currentJobs.get(jobID) match { case Some((graph, jobInfo)) => - val accumulatorValues = graph.getAccumulatorsSerialized() - sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) + val accumulatorValues = graph.getSmallAccumulatorsSerialized + sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) case None => archive.forward(message) } @@ -888,7 +887,7 @@ class JobManager( val finished = new java.util.ArrayList[JobID]() val canceled = new java.util.ArrayList[JobID]() val failed = new java.util.ArrayList[JobID]() - + currentJobs.values.foreach { case (graph, _) => graph.getState() match { case JobStatus.FINISHED => finished.add(graph.getJobID) @@ -932,7 +931,7 @@ class JobManager( * Updates the accumulators reported from a task manager via the Heartbeat message. * @param accumulators list of accumulator snapshots */ - private def updateAccumulators(accumulators : Seq[AccumulatorSnapshot]) = { + private def updateAccumulators(accumulators : Seq[BaseAccumulatorSnapshot]) = { accumulators foreach { case accumulatorEvent => currentJobs.get(accumulatorEvent.getJobID) match { @@ -1131,7 +1130,7 @@ object JobManager { // start the job manager web frontend if (configuration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) { LOG.info("Starting NEW JobManger web frontend") - + // start the new web frontend. we need to load this dynamically // because it is not in the same project/dependencies startWebRuntimeMonitor(configuration, jobManager, archiver) @@ -1565,10 +1564,10 @@ object JobManager { /** * Starts the web runtime monitor. Because the actual implementation of the * runtime monitor is in another project, we load the runtime monitor dynamically. - * + * * Because failure to start the web runtime monitor is not considered fatal, * this method does not throw any exceptions, but only logs them. - * + * * @param config The configuration for the runtime monitor. * @param jobManager The JobManager actor. * @param archiver The execution graph archive actor. @@ -1582,7 +1581,7 @@ object JobManager { val classname = "org.apache.flink.runtime.webmonitor.WebRuntimeMonitor" val clazz: Class[_ <: WebMonitor] = Class.forName(classname) .asSubclass(classOf[WebMonitor]) - + val ctor: Constructor[_ <: WebMonitor] = clazz.getConstructor(classOf[Configuration], classOf[ActorRef], classOf[ActorRef]) @@ -1601,13 +1600,13 @@ object JobManager { LOG.error("Failed to instantiate web runtime monitor.", t) null } - + if (monitor != null) { try { monitor.start() } catch { - case e: Exception => + case e: Exception => LOG.error("Failed to start web runtime monitor", e) } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala index 9f228ed4da833..5b6ba24616fda 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala @@ -132,7 +132,7 @@ class MemoryArchivist(private val max_entries: Int) try { graphs.get(jobID) match { case Some(graph) => - val accumulatorValues = graph.getAccumulatorsSerialized() + val accumulatorValues = graph.getSmallAccumulatorsSerialized() sender() ! AccumulatorResultsFound(jobID, accumulatorValues) case None => sender() ! AccumulatorResultsNotFound(jobID) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala index 6cb571ce7c63e..482cb6995d3de 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala @@ -18,7 +18,7 @@ package org.apache.flink.runtime.messages -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot import org.apache.flink.runtime.instance.InstanceID /** @@ -56,7 +56,7 @@ object TaskManagerMessages { * @param accumulators Accumulators of tasks serialized as Tuple2[internal, user-defined] */ case class Heartbeat(instanceID: InstanceID, metricsReport: Array[Byte], - accumulators: Seq[AccumulatorSnapshot]) + accumulators: Seq[BaseAccumulatorSnapshot]) // -------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index f974946df7d2f..364e7a8dea8a7 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -19,6 +19,8 @@ package org.apache.flink.runtime.taskmanager import java.io.{File, IOException} +import java.lang.management.{ManagementFactory, OperatingSystemMXBean} +import java.lang.reflect.Method import java.net.{InetAddress, InetSocketAddress} import java.util.UUID import java.util.concurrent.TimeUnit @@ -31,18 +33,18 @@ import _root_.akka.util.Timeout import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.codahale.metrics.json.MetricsModule -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} - +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration.{Configuration, ConfigConstants, GlobalConfiguration, IllegalConfigurationException} -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot +import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage} import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.blob.{BlobService, BlobCache} +import org.apache.flink.runtime.blob.{BlobCache, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor} import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} @@ -56,20 +58,25 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.io.network.netty.NettyConfig import org.apache.flink.runtime.jobgraph.IntermediateDataSetID import org.apache.flink.runtime.jobmanager.JobManager -import org.apache.flink.runtime.memorymanager.{MemoryManager, DefaultMemoryManager} +import org.apache.flink.runtime.memorymanager.{DefaultMemoryManager, MemoryManager} import org.apache.flink.runtime.messages.Messages._ import org.apache.flink.runtime.messages.RegistrationMessages._ import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ +import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} import org.apache.flink.runtime.net.NetUtils import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner -import org.apache.flink.runtime.util.{ZooKeeperUtil, MathUtils, EnvironmentInformation} +import org.apache.flink.runtime.util.{EnvironmentInformation, MathUtils, ZooKeeperUtil} +import org.apache.flink.runtime.StreamingMode +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool +import scala.language.postfixOps import scala.util.{Failure, Success} import scala.collection.JavaConverters._ import scala.collection.JavaConversions._ @@ -1046,7 +1053,7 @@ class TaskManager( val metricsReport: Array[Byte] = metricRegistryMapper.writeValueAsBytes(metricRegistry) val accumulatorEvents = - scala.collection.mutable.Buffer[AccumulatorSnapshot]() + scala.collection.mutable.Buffer[BaseAccumulatorSnapshot]() runningTasks foreach { case (execID, task) => diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java index 5c9ffa7c6747f..ee577fefff3c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java @@ -47,7 +47,7 @@ public void testSerialization() { origMap.put("name1", new SerializedValue(723L)); origMap.put("name2", new SerializedValue("peter")); - SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap); + SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap, null); // serialize and deserialize the object SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); @@ -84,12 +84,13 @@ public void testSerialization() { @Test public void testSerializationWithNullValues() { try { - SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null); + SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null, null); SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); assertNull(cloned.getJobId()); assertEquals(0L, cloned.getNetRuntime()); assertNull(cloned.getSerializedAccumulatorResults()); + assertNull(cloned.getBlobKeysToLargeAccumulators()); JobExecutionResult jResult = result.toJobExecutionResult(getClass().getClassLoader()); assertNull(jResult.getJobID()); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 8a7297b30370a..19467b8de1c04 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -181,7 +181,7 @@ trait TestingJobManager extends FlinkActor { currentJobs.get(jobID) match { case Some((graph, jobInfo)) => val flinkAccumulators = graph.getFlinkAccumulators - val userAccumulators = graph.aggregateUserAccumulators + val userAccumulators = graph.aggregateSmallUserAccumulators actors foreach { actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators) } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 01e6f624cda04..31f406d46bfd7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -44,7 +44,8 @@ * - when null records are passed through the system. * - when disjoint dataflows are executed * - when accumulators are used chained after a non-udf operator. - * + * - when an accumulator is bigger than the akka.framesize. + * * The tests are bundled into one class to reuse the same test cluster. This speeds * up test execution, as the majority of the test time goes usually into starting/stopping the * test cluster. @@ -61,6 +62,8 @@ public static void startCluster() { config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + config.setString(ConfigConstants.AKKA_FRAMESIZE, ConfigConstants.DEFAULT_AKKA_FRAMESIZE); + cluster = new ForkableFlinkMiniCluster(config, false); } catch (Exception e) { @@ -174,4 +177,28 @@ public void flatMap(Long value, Collector out) { fail(e.getMessage()); } } + + @Test + public void testOversizedAccumulators() { + try { + + ExecutionEnvironment env = + ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + + int noOfParallelism = 5; + int longsPerTask = 1200000; + + env.setParallelism(noOfParallelism); + env.getConfig().disableSysoutLogging(); + + DataSet bigEnough = env.generateSequence(1, noOfParallelism * longsPerTask); + long theCount = bigEnough.collect().size(); + + assertEquals(noOfParallelism * longsPerTask, theCount); + + }catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } } From 10e620ae03a5603ff45e8d3f9fa4d01acdf91eb0 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 24 Jul 2015 16:29:46 +0300 Subject: [PATCH 003/175] Fixed some tests. --- .../flink/runtime/taskmanager/Task.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index f02006c596912..4a3db94e8d352 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.taskmanager; +import akka.actor.ActorRef; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; @@ -323,14 +324,19 @@ private InetSocketAddress getBlobCacheServerAddress() { throw new RuntimeException("TaskManager not associated to JobManager."); } - String jmHost = this.jobManager.actor().path().address().host().getOrElse( - new AbstractFunction0() { - @Override - public String apply() { - return "localhost"; - } - }); - + final String jmHost; + ActorRef jobManagerActor = this.jobManager.actor(); + if (jobManagerActor == null) { + jmHost = "localhost"; + } else { + jmHost = jobManagerActor.path().address().host().getOrElse( + new AbstractFunction0() { + @Override + public String apply() { + return "localhost"; + } + }); + } int blobPort = this.libraryCache.getBlobServerPort(); return new InetSocketAddress(jmHost, blobPort); } From 31c0c2925c63c2999aa831602f52601d33cd6b47 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 24 Jul 2015 11:29:12 +0200 Subject: [PATCH 004/175] [hotfix] Fix warnings introduced by recent Watermark Commit --- .../BroadcastOutputSelectorWrapper.java | 2 +- .../DirectedOutputSelectorWrapper.java | 2 +- .../api/operators/StreamFlatMap.java | 4 ++-- .../windowing/StreamDiscretizer.java | 6 +++--- .../runtime/io/CollectorWrapper.java | 3 ++- .../runtime/io/RecordWriterOutput.java | 12 +++++------ .../runtime/io/StreamInputProcessor.java | 10 ++++----- .../runtime/io/StreamTwoInputProcessor.java | 8 +++---- .../runtime/streamrecord/StreamRecord.java | 2 +- .../streamrecord/StreamRecordSerializer.java | 2 +- .../runtime/tasks/OutputHandler.java | 21 +++++++++---------- .../streaming/util/keys/KeySelectorUtil.java | 1 + .../consumer/StreamTestSingleInputGate.java | 5 +++-- .../api/operators/StreamCounterTest.java | 3 +-- .../api/operators/StreamFilterTest.java | 2 +- .../api/operators/StreamFlatMapTest.java | 3 +-- .../api/operators/StreamGroupedFoldTest.java | 2 +- .../operators/StreamGroupedReduceTest.java | 2 +- .../api/operators/StreamMapTest.java | 2 +- .../api/operators/StreamProjectTest.java | 17 ++++++++------- .../api/operators/co/CoStreamFlatMapTest.java | 3 +-- .../api/operators/co/CoStreamMapTest.java | 11 +--------- .../runtime/tasks/OneInputStreamTaskTest.java | 11 ++++------ .../runtime/tasks/SourceStreamTaskTest.java | 5 +++-- .../runtime/tasks/StreamTaskTestHarness.java | 7 +++---- .../runtime/tasks/TwoInputStreamTaskTest.java | 8 +++---- .../streaming/timestamp/TimestampITCase.java | 1 + .../OneInputStreamOperatorTestHarness.java | 7 +++---- .../flink/streaming/util/TestHarnessUtil.java | 8 +++---- .../TwoInputStreamOperatorTestHarness.java | 6 +++--- 30 files changed, 83 insertions(+), 93 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java index 0fe84d8ecbe61..00c6f80d26420 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java @@ -33,7 +33,7 @@ public BroadcastOutputSelectorWrapper() { outputs = new ArrayList>>(); } - @SuppressWarnings("unchecked") + @SuppressWarnings("unchecked,rawtypes") @Override public void addCollector(Collector> output, StreamEdge edge) { Collector output1 = output; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java index 46b315de6a483..c6e33883fa5be 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java @@ -47,7 +47,7 @@ public DirectedOutputSelectorWrapper(List> outputSelectors) this.outputMap = new HashMap>>>(); } - @SuppressWarnings("unchecked") + @SuppressWarnings("unchecked,rawtypes") @Override public void addCollector(Collector> output, StreamEdge edge) { Collector output1 = output; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java index 5547c6ac46e31..ff7f66226b6fd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java @@ -28,7 +28,7 @@ public class StreamFlatMap private static final long serialVersionUID = 1L; - private TimestampedCollector collector; + private transient TimestampedCollector collector; public StreamFlatMap(FlatMapFunction flatMapper) { super(flatMapper); @@ -38,7 +38,7 @@ public StreamFlatMap(FlatMapFunction flatMapper) { @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - collector = new TimestampedCollector(output); + collector = new TimestampedCollector(output); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java index df84b623403c9..47c2323991681 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java @@ -132,7 +132,7 @@ protected synchronized void triggerOnFakeElement(Object input) { * if not empty */ protected void emitWindow() { - output.collect(new StreamRecord(windowEvent.setTrigger())); + output.collect(new StreamRecord>(windowEvent.setTrigger())); } private void activeEvict(Object input) { @@ -144,7 +144,7 @@ private void activeEvict(Object input) { } if (numToEvict > 0) { - output.collect(new StreamRecord(windowEvent.setEviction(numToEvict))); + output.collect(new StreamRecord>(windowEvent.setEviction(numToEvict))); bufferSize -= numToEvict; bufferSize = bufferSize >= 0 ? bufferSize : 0; } @@ -154,7 +154,7 @@ private void evict(IN input, boolean isTriggered) { int numToEvict = evictionPolicy.notifyEviction(input, isTriggered, bufferSize); if (numToEvict > 0) { - output.collect(new StreamRecord(windowEvent.setEviction(numToEvict))); + output.collect(new StreamRecord>(windowEvent.setEviction(numToEvict))); bufferSize -= numToEvict; bufferSize = bufferSize >= 0 ? bufferSize : 0; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java index 2f9d1d693052f..6bb44dd7a1e91 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java @@ -38,7 +38,8 @@ public CollectorWrapper(OutputSelectorWrapper outputSelectorWrapper) { allOutputs = new ArrayList>(); } - public void addCollector(Collector> output, StreamEdge edge) { + @SuppressWarnings("unchecked,rawtypes") + public void addCollector(Output> output, StreamEdge edge) { outputSelectorWrapper.addCollector(output, edge); allOutputs.add((Output) output); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index e9cbb7d6abdae..b656bb555e441 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -39,12 +39,12 @@ public class RecordWriterOutput implements Output> { private static final Logger LOG = LoggerFactory.getLogger(RecordWriterOutput.class); - private RecordWriter recordWriter; - private SerializationDelegate serializationDelegate; + private RecordWriter>> recordWriter; + private SerializationDelegate> serializationDelegate; @SuppressWarnings("unchecked") public RecordWriterOutput( - RecordWriter recordWriter, + RecordWriter>> recordWriter, TypeSerializer outSerializer, boolean enableWatermarkMultiplexing) { Preconditions.checkNotNull(recordWriter); @@ -79,9 +79,9 @@ public void collect(StreamRecord record) { } @Override - @SuppressWarnings("unchecked") + @SuppressWarnings("unchecked,rawtypes") public void emitWatermark(Watermark mark) { - serializationDelegate.setInstance(mark); + ((SerializationDelegate)serializationDelegate).setInstance(mark); try { recordWriter.broadcastEmit(serializationDelegate); } catch (Exception e) { @@ -95,7 +95,7 @@ public void emitWatermark(Watermark mark) { @Override public void close() { if (recordWriter instanceof StreamRecordWriter) { - ((StreamRecordWriter) recordWriter).close(); + ((StreamRecordWriter) recordWriter).close(); } else { try { recordWriter.flush(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index e665710c70bd6..4c40e5f334b02 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -57,9 +57,9 @@ public class StreamInputProcessor extends AbstractReader implements ReaderBa @SuppressWarnings("unused") private static final Logger LOG = LoggerFactory.getLogger(StreamInputProcessor.class); - private final RecordDeserializer[] recordDeserializers; + private final RecordDeserializer>[] recordDeserializers; - private RecordDeserializer currentRecordDeserializer; + private RecordDeserializer> currentRecordDeserializer; // We need to keep track of the channel from which a buffer came, so that we can // appropriately map the watermarks to input channels @@ -72,7 +72,7 @@ public class StreamInputProcessor extends AbstractReader implements ReaderBa private long[] watermarks; private long lastEmittedWatermark; - private DeserializationDelegate deserializationDelegate; + private DeserializationDelegate deserializationDelegate; @SuppressWarnings("unchecked") public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSerializer, boolean enableWatermarkMultiplexing) { @@ -86,12 +86,12 @@ public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSeri } else { inputRecordSerializer = new StreamRecordSerializer(inputSerializer); } - this.deserializationDelegate = new NonReusingDeserializationDelegate(inputRecordSerializer); + this.deserializationDelegate = new NonReusingDeserializationDelegate(inputRecordSerializer); // Initialize one deserializer per input channel this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; for (int i = 0; i < recordDeserializers.length; i++) { - recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(); + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer>(); } watermarks = new long[inputGate.getNumberOfInputChannels()]; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 1fe98bbd74f30..82e79360483f8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -58,9 +58,9 @@ public class StreamTwoInputProcessor extends AbstractReader implements @SuppressWarnings("unused") private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class); - private final RecordDeserializer[] recordDeserializers; + private final RecordDeserializer>[] recordDeserializers; - private RecordDeserializer currentRecordDeserializer; + private RecordDeserializer> currentRecordDeserializer; // We need to keep track of the channel from which a buffer came, so that we can // appropriately map the watermarks to input channels @@ -79,8 +79,8 @@ public class StreamTwoInputProcessor extends AbstractReader implements private int numInputChannels1; private int numInputChannels2; - private DeserializationDelegate deserializationDelegate1; - private DeserializationDelegate deserializationDelegate2; + private DeserializationDelegate deserializationDelegate1; + private DeserializationDelegate deserializationDelegate2; @SuppressWarnings("unchecked") public StreamTwoInputProcessor( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java index aff030ef67224..6521e7fcde725 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java @@ -103,7 +103,7 @@ public boolean equals(Object o) { return false; } - StreamRecord that = (StreamRecord) o; + StreamRecord that = (StreamRecord) o; return value.equals(that.value) && timestamp == that.timestamp; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java index b05eb36830bd8..2619891fbecd2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java @@ -58,7 +58,7 @@ public boolean isImmutableType() { @Override @SuppressWarnings("unchecked") - public TypeSerializer duplicate() { + public StreamRecordSerializer duplicate() { return this; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java index cf17b3e453d5b..aa55151e4c704 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java @@ -44,7 +44,6 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; -import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -146,18 +145,18 @@ private Output> createChainedCollector(StreamConfig chainedT // Create collectors for the network outputs for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(cl)) { - Collector outCollector = outputMap.get(outputEdge); + Output output = outputMap.get(outputEdge); - wrapper.addCollector(outCollector, outputEdge); + wrapper.addCollector(output, outputEdge); } // Create collectors for the chained outputs for (StreamEdge outputEdge : chainedTaskConfig.getChainedOutputs(cl)) { - Integer output = outputEdge.getTargetId(); + Integer outputId = outputEdge.getTargetId(); - Collector outCollector = createChainedCollector(chainedConfigs.get(output), accumulatorMap); + Output output = createChainedCollector(chainedConfigs.get(outputId), accumulatorMap); - wrapper.addCollector(outCollector, outputEdge); + wrapper.addCollector(output, outputEdge); } if (chainedTaskConfig.isChainStart()) { @@ -200,7 +199,7 @@ public Output> getOutput() { * the configuration of its source task * * @param outputVertex - * Name of the output to which the streamoutput will be set up + * Name of the output to which the stream output will be set up * @param upStreamConfig * The config of upStream task * @return The created StreamOutput @@ -222,7 +221,7 @@ private RecordWriterOutput createStreamOutput(StreamEdge edge, Integer ou output.setReporter(reporter); @SuppressWarnings("unchecked") - RecordWriterOutput streamOutput = new RecordWriterOutput((RecordWriter) output, outSerializer, vertex.getExecutionConfig().areTimestampsEnabled()); + RecordWriterOutput streamOutput = new RecordWriterOutput(output, outSerializer, vertex.getExecutionConfig().areTimestampsEnabled()); if (LOG.isTraceEnabled()) { LOG.trace("Partitioner set: {} with {} outputs for {}", outputPartitioner.getClass() @@ -245,9 +244,9 @@ public void clearWriters() { } private static class ChainingOutput implements Output> { - protected OneInputStreamOperator operator; + protected OneInputStreamOperator operator; - public ChainingOutput(OneInputStreamOperator operator) { + public ChainingOutput(OneInputStreamOperator operator) { this.operator = operator; } @@ -292,7 +291,7 @@ public void close() { private static class CopyingChainingOutput extends ChainingOutput { private final TypeSerializer> serializer; - public CopyingChainingOutput(OneInputStreamOperator operator, + public CopyingChainingOutput(OneInputStreamOperator operator, TypeSerializer> serializer) { super(operator); this.serializer = serializer; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java index 49f2fe0468519..89c61422329e3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java @@ -103,6 +103,7 @@ public OneKeySelector(TypeComparator comparator) { } @Override + @SuppressWarnings("unchecked") public K getKey(IN value) throws Exception { comparator.extractKeys(value, keyArray, 0); key = (K) keyArray[0]; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index a20436a0f40cb..c479f957e23af 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -62,6 +62,7 @@ public class StreamTestSingleInputGate extends TestSingleInputGate { private ConcurrentLinkedQueue>[] inputQueues; + @SuppressWarnings("unchecked") public StreamTestSingleInputGate( int numInputChannels, int bufferSize, @@ -84,8 +85,8 @@ private void setupInputChannels() throws IOException, InterruptedException { for (int i = 0; i < numInputChannels; i++) { final int channelIndex = i; - final RecordSerializer>> recordSerializer = new SpanningRecordSerializer>>(); - final SerializationDelegate delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer(serializer)); + final RecordSerializer> recordSerializer = new SpanningRecordSerializer>(); + final SerializationDelegate delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer(serializer)); inputQueues[channelIndex] = new ConcurrentLinkedQueue>(); inputChannels[channelIndex] = new TestInputChannel(inputGate, i); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java index 3e662badfc450..dc8024cb6e579 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java @@ -23,7 +23,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; -import org.joda.time.Instant; import org.junit.Test; /** @@ -43,7 +42,7 @@ public void testCount() throws Exception { OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java index f672a89663cd4..bf4fe40cb10a4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java @@ -58,7 +58,7 @@ public void testFilter() throws Exception { OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java index ac7caa7762333..e4e29c1d627fd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java @@ -27,7 +27,6 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.util.Collector; -import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; @@ -62,7 +61,7 @@ public void testFlatMap() throws Exception { OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java index 8499aa2aa9d0d..cb08e65ea4dab 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java @@ -76,7 +76,7 @@ public String getKey(Integer value) throws Exception { OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java index dca1cbb1a3854..9e35fa2bdc5d4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java @@ -70,7 +70,7 @@ public Integer getKey(Integer value) throws Exception { OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java index d5f2f62bdcd72..4d1249256a517 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java @@ -57,7 +57,7 @@ public void testMap() throws Exception { OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java index ede7db50e2d48..e8f0a036f0d52 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java @@ -42,7 +42,6 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TestStreamEnvironment; -import org.joda.time.Instant; import org.junit.Test; /** @@ -75,7 +74,7 @@ public void testProject() throws Exception { OneInputStreamOperatorTestHarness, Tuple3> testHarness = new OneInputStreamOperatorTestHarness, Tuple3>(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); @@ -110,13 +109,17 @@ public void APIWithoutTypesTest() { StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE); env.generateSequence(1, 10).map(new MapFunction>() { - @Override - public Tuple3 map(Long value) throws Exception { - return new Tuple3(value, 'c', value.doubleValue()); - } - }) + private static final long serialVersionUID = 1L; + + @Override + public Tuple3 map(Long value) throws Exception { + return new Tuple3(value, 'c', value.doubleValue()); + } + }) .project(0, 2) .addSink(new SinkFunction() { + private static final long serialVersionUID = 1L; + @Override @SuppressWarnings("unchecked") public void invoke(Tuple value) throws Exception { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java index 2c9ba5c3da77c..b8e9619e7eead 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java @@ -33,7 +33,6 @@ import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness; import org.apache.flink.util.Collector; -import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; @@ -72,7 +71,7 @@ public void testCoFlatMap() throws Exception { TwoInputStreamOperatorTestHarness testHarness = new TwoInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java index dcf49724bc787..28ae6642959fb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java @@ -18,27 +18,18 @@ package org.apache.flink.streaming.api.operators.co; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.streaming.api.functions.co.CoMapFunction; -import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoMapFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness; -import org.apache.flink.util.Collector; -import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; import java.io.Serializable; import java.util.concurrent.ConcurrentLinkedQueue; -import static org.junit.Assert.fail; - /** * Tests for {@link org.apache.flink.streaming.api.operators.co.CoStreamMap}. These test that: * @@ -73,7 +64,7 @@ public void testCoMap() throws Exception { TwoInputStreamOperatorTestHarness testHarness = new TwoInputStreamOperatorTestHarness(operator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index d623dd85be4e8..4399a104632ef 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -28,16 +28,13 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.TestHarnessUtil; -import org.joda.time.Instant; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import java.util.List; -import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -68,7 +65,7 @@ public void testOpenCloseAndTimestamps() throws Exception { streamConfig.setStreamOperator(mapOperator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.invoke(); @@ -103,7 +100,7 @@ public void testWatermarkForwarding() throws Exception { StreamMap mapOperator = new StreamMap(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; testHarness.invoke(); @@ -179,7 +176,7 @@ public void testCheckpointBarriers() throws Exception { StreamMap mapOperator = new StreamMap(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); - Queue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; testHarness.invoke(); @@ -237,7 +234,7 @@ public void testOvertakingCheckpointBarriers() throws Exception { StreamMap mapOperator = new StreamMap(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); - Queue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; testHarness.invoke(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index f34eafe88a4c5..0f6e5f1e11166 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -89,6 +89,7 @@ public void testOpenClose() throws Exception { * source kept emitting elements while the checkpoint was ongoing. */ @Test + @SuppressWarnings("unchecked") public void testCheckpointing() throws Exception { final int NUM_ELEMENTS = 100; final int NUM_CHECKPOINTS = 100; @@ -108,7 +109,7 @@ public void testCheckpointing() throws Exception { ExecutorService executor = Executors.newFixedThreadPool(10); - Future[] checkpointerResults = new Future[NUM_CHECKPOINTERS]; + Future[] checkpointerResults = new Future[NUM_CHECKPOINTERS]; for (int i = 0; i < NUM_CHECKPOINTERS; i++) { checkpointerResults[i] = executor.submit(new Checkpointer(NUM_CHECKPOINTS, CHECKPOINT_INTERVAL, sourceTask)); } @@ -131,7 +132,7 @@ public void testCheckpointing() throws Exception { Assert.assertEquals(NUM_ELEMENTS, resultElements.size()); } - private static class MockSource implements SourceFunction>, Checkpointed { + private static class MockSource implements SourceFunction>, Checkpointed { private static final long serialVersionUID = 1; private int maxElements; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index a4cc0d3fb7ec1..283243ecab9d5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -35,7 +35,6 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; @@ -43,7 +42,6 @@ import java.io.IOException; import java.util.LinkedList; import java.util.List; -import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; @@ -85,7 +83,7 @@ public class StreamTaskTestHarness { private TypeSerializer outputSerializer; private StreamRecordSerializer outputStreamRecordSerializer; - private ConcurrentLinkedQueue outputList; + private ConcurrentLinkedQueue outputList; protected Thread taskThread; @@ -94,6 +92,7 @@ public class StreamTaskTestHarness { // input related methods only need to be implemented once, in generic form protected int numInputGates; protected int numInputChannelsPerGate; + @SuppressWarnings("rawtypes") protected StreamTestSingleInputGate[] inputGates; public StreamTaskTestHarness(AbstractInvokable task, TypeInformation outputType) { @@ -198,7 +197,7 @@ public void waitForTaskCompletion() throws InterruptedException { * {@link org.apache.flink.streaming.util.TestHarnessUtil#getRawElementsFromOutput(java.util.Queue)}} * to extract only the StreamRecords. */ - public Queue getOutput() { + public ConcurrentLinkedQueue getOutput() { return outputList; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index 3b113ab07c7e7..3c7204de09e30 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -69,7 +69,7 @@ public void testOpenCloseAndTimestamps() throws Exception { streamConfig.setStreamOperator(coMapOperator); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); testHarness.invoke(); @@ -107,7 +107,7 @@ public void testWatermarkForwarding() throws Exception { CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); streamConfig.setStreamOperator(coMapOperator); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; testHarness.invoke(); @@ -186,7 +186,7 @@ public void testCheckpointBarriers() throws Exception { CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); streamConfig.setStreamOperator(coMapOperator); - Queue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; testHarness.invoke(); @@ -252,7 +252,7 @@ public void testOvertakingCheckpointBarriers() throws Exception { CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); streamConfig.setStreamOperator(coMapOperator); - Queue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; testHarness.invoke(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java index d3fde9e25bb21..52de8aacd1d14 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java @@ -235,6 +235,7 @@ public void testSourceEmitWatermark() throws Exception { env.execute(); } + @SuppressWarnings("unchecked") public static class CustomOperator extends AbstractStreamOperator implements OneInputStreamOperator { List watermarks; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index 133f1434a0429..6652fde09cced 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -34,7 +34,6 @@ import java.io.Serializable; import java.util.Collection; import java.util.HashMap; -import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -49,14 +48,14 @@ public class OneInputStreamOperatorTestHarness { OneInputStreamOperator operator; - ConcurrentLinkedQueue outputList; + ConcurrentLinkedQueue outputList; ExecutionConfig executionConfig; public OneInputStreamOperatorTestHarness(OneInputStreamOperator operator) { this.operator = operator; - outputList = new ConcurrentLinkedQueue(); + outputList = new ConcurrentLinkedQueue(); executionConfig = new ExecutionConfig(); @@ -77,7 +76,7 @@ public OneInputStreamOperatorTestHarness(OneInputStreamOperator operato * {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)} * to extract only the StreamRecords. */ - public Queue getOutput() { + public ConcurrentLinkedQueue getOutput() { return outputList; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java index a0a6c8d62498a..0732b641d5b73 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java @@ -32,7 +32,7 @@ public class TestHarnessUtil { * Extracts the StreamRecords from the given output list. */ @SuppressWarnings("unchecked") - public static List> getStreamRecordsFromOutput(List output) { + public static List> getStreamRecordsFromOutput(List output) { List> resultElements = new LinkedList>(); for (Object e: output) { if (e instanceof StreamRecord) { @@ -46,11 +46,11 @@ public static List> getStreamRecordsFromOutput(List outp * Extracts the raw elements from the given output list. */ @SuppressWarnings("unchecked") - public static List getRawElementsFromOutput(Queue output) { + public static List getRawElementsFromOutput(Queue output) { List resultElements = new LinkedList(); for (Object e: output) { if (e instanceof StreamRecord) { - resultElements.add((OUT) ((StreamRecord) e).getValue()); + resultElements.add(((StreamRecord) e).getValue()); } } return resultElements; @@ -59,7 +59,7 @@ public static List getRawElementsFromOutput(Queue output) { /** * Compare the two queues containing operator/task output by converting them to an array first. */ - public static void assertOutputEquals(String message, Queue expected, Queue actual) { + public static void assertOutputEquals(String message, Queue expected, Queue actual) { Assert.assertArrayEquals(message, expected.toArray(), actual.toArray()); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java index ea753f83af029..1e8b5c62dcc39 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java @@ -48,14 +48,14 @@ public class TwoInputStreamOperatorTestHarness { TwoInputStreamOperator operator; - ConcurrentLinkedQueue outputList; + ConcurrentLinkedQueue outputList; ExecutionConfig executionConfig; public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator operator) { this.operator = operator; - outputList = new ConcurrentLinkedQueue(); + outputList = new ConcurrentLinkedQueue(); executionConfig = new ExecutionConfig(); @@ -76,7 +76,7 @@ public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator o * {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)} * to extract only the StreamRecords. */ - public Queue getOutput() { + public ConcurrentLinkedQueue getOutput() { return outputList; } From a49593cbf18e1d4e2386e76282fc446263f98e04 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Mon, 27 Jul 2015 04:33:07 +0300 Subject: [PATCH 005/175] Now if the task accumulators are small, but become big after merging at the JobManager, the JobManager stores it also in the BlobCache and informs the client accordingly. --- .../accumulators/AccumulatorRegistry.java | 63 +-------- .../accumulators/BaseAccumulatorSnapshot.java | 4 - .../accumulators/LargeAccumulatorHelper.java | 125 ++++++++++++++++++ .../LargeAccumulatorSnapshot.java | 4 - .../SmallAccumulatorSnapshot.java | 4 - .../librarycache/BlobLibraryCacheManager.java | 13 +- .../FallbackLibraryCacheManager.java | 7 +- .../librarycache/LibraryCacheManager.java | 7 + .../executiongraph/ExecutionGraph.java | 85 +++++++++--- .../flink/runtime/util/SerializedValue.java | 8 ++ .../apache/flink/runtime/akka/AkkaUtils.scala | 7 + .../flink/runtime/jobmanager/JobManager.scala | 65 ++++++++- .../runtime/jobmanager/MemoryArchivist.scala | 2 +- .../test/misc/MiscellaneousIssuesITCase.java | 26 +++- 14 files changed, 322 insertions(+), 98 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java index 6d53f2045f697..555f860ddf7a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java @@ -23,17 +23,14 @@ import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.blob.BlobClient; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.util.SerializedValue; -import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.InetSocketAddress; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -100,73 +97,24 @@ public BaseAccumulatorSnapshot getSnapshot() { try { serializedAccumulators = new SerializedValue>>(userAccumulators); - if (serializedAccumulators.getSizeInBytes() > 0.8 * AkkaUtils.getFramesize(jobConfiguration)) { - largeAccumulatorBlobKeys = storeAccumulatorsToBlobCache(blobServerAddress, userAccumulators); + if (serializedAccumulators.getSizeInBytes() > AkkaUtils.getLargeAccumulatorThreshold(jobConfiguration)) { + largeAccumulatorBlobKeys = LargeAccumulatorHelper. + storeAccumulatorsToBlobCache(blobServerAddress, userAccumulators); + snapshot = new LargeAccumulatorSnapshot(jobID, taskID, flinkAccumulators, largeAccumulatorBlobKeys); - } else { snapshot = new SmallAccumulatorSnapshot(jobID, taskID, flinkAccumulators, serializedAccumulators); } return snapshot; } catch (IOException e) { - LOG.warn("Failed to serialize accumulators for task.", e); + LOG.warn("Error while creating a snapshot of current results: "+ e.getMessage()); return null; } } - /** - * Puts the blobs of the large accumulators on the BlobCache. - * @param blobServerAddress the address of the server to the blobCache. - * @param accumulators the accumulators to be stored in the cache. - * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. - * */ - private Map> storeAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, - Map> accumulators) { - if (blobServerAddress == null) { - throw new RuntimeException("Undefined Blob Server Address."); - } - if (accumulators.isEmpty()) { - return Collections.emptyMap(); - } - - Map> keys = new HashMap>(); - BlobClient bc = null; - try { - bc = new BlobClient(blobServerAddress); - - for (Map.Entry> entry : accumulators.entrySet()) { - - String accumulatorName = entry.getKey(); - Accumulator accumulator = entry.getValue(); - - byte[] serializedAccumulator = InstantiationUtil.serializeObject(accumulator); - BlobKey blobKey = bc.put(serializedAccumulator); - - List accKeys = keys.get(accumulatorName); - if (accKeys == null) { - accKeys = new ArrayList(); - } - accKeys.add(blobKey); - keys.put(accumulatorName, accKeys); - } - } catch (IOException e) { - LOG.error("Failed to send oversized accumulators to the BlobCache: ", e); - } finally { - try { - if(bc != null) { - bc.close(); - } - } catch (IOException e) { - LOG.error("Failed to close BlobClient: ", e); - } - - } - return keys; - } - /** * Gets the map for user-defined accumulators. */ @@ -228,5 +176,4 @@ public void reportNumBytesOut(long value) { numBytesOut.add(value); } } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java index 09bed90dfb763..39803814f14b8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java @@ -27,10 +27,6 @@ import java.io.Serializable; import java.util.Map; -/** - * LICENSE SHOULD GO HERE. - * Created by kkloudas on 7/23/15. - */ public class BaseAccumulatorSnapshot implements Serializable { private static final long serialVersionUID = 42L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java new file mode 100644 index 0000000000000..160a612dd265a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java @@ -0,0 +1,125 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.util.SerializedValue; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class LargeAccumulatorHelper { + + /** + * Puts the blobs of the large accumulators on the BlobCache. + * @param blobServerAddress the address of the server to the blobCache. + * @param accumulators the accumulators to be stored in the cache. + * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. + * */ + public static Map> storeAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, + Map> accumulators) throws IOException { + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> serializedAccumulators = serializeAccumulators(accumulators); + return storeSerializedAccumulatorsToBlobCache(blobServerAddress, serializedAccumulators); + } + + /** + * Puts the blobs of the large accumulators on the BlobCache. + * @param blobServerAddress the address of the server to the blobCache. + * @param accumulators the accumulators to be stored in the cache. + * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. + * */ + public static Map> storeSerializedAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, + Map> accumulators) throws IOException { + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> keys = new HashMap>(); + + BlobClient bc = null; + try { + bc = new BlobClient(blobServerAddress); + + for (Map.Entry> entry : accumulators.entrySet()) { + + String accumulatorName = entry.getKey(); + byte[] accumulatorPayload = entry.getValue().getSerializedData(); + if(accumulatorPayload == null) { + continue; + } + + BlobKey blobKey = bc.put(accumulatorPayload); + List accKeys = keys.get(accumulatorName); + if (accKeys == null) { + accKeys = new ArrayList(); + } + accKeys.add(blobKey); + keys.put(accumulatorName, accKeys); + } + } catch (IOException e) { + throw new IOException("Failed to send oversized accumulators to the BlobCache: ", e); + } finally { + try { + if(bc != null) { + bc.close(); + } + } catch (IOException e) { + throw new IOException("Failed to close BlobClient: ", e); + } + + } + return keys; + } + + private static Map> serializeAccumulators(Map> accumulators) throws IOException { + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> serializedAccumulators = + new HashMap>(); + try { + for (Map.Entry> entry : accumulators.entrySet()) { + String accumulatorName = entry.getKey(); + Accumulator accumulator = entry.getValue(); + serializedAccumulators.put(accumulatorName, new SerializedValue(accumulator)); + } + } catch (IOException e) { + throw new IOException("Failed to serialize accumulators.", e); + } + return serializedAccumulators; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java index a4283f165cc26..791796b9ead46 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java @@ -27,10 +27,6 @@ import java.util.List; import java.util.Map; -/** - * LICENSE SHOULD GO HERE. - * Created by kkloudas on 7/23/15. - */ public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java index 1911ebac6c321..6773ee49ff479 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java @@ -26,10 +26,6 @@ import java.io.IOException; import java.util.Map; -/** - * LICENSE SHOULD GO HERE. - * Created by kkloudas on 7/23/15. - */ public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index ad482cf4596fd..16262a42e32cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -138,7 +138,16 @@ public void registerTask(JobID jobId, ExecutionAttemptID task, Collection currentExecutions; + /** This is to guarantee that merging is going to happen only once. */ + private Map> mergedSmallUserAccumulators; + + /** This is to guarantee that merging is going to happen only once. */ + private Map> mergedLargeUserAccumulators; + /** * Updates the accumulators during the runtime of a job. Final accumulator results are transferred * through the UpdateTaskExecutionState message. @@ -573,58 +579,99 @@ public Map> /** * Merges all accumulator results from the tasks previously executed in the Executions. + * If the accumulators have already been merged, then the already computed result is returned. + * This is to avoid merging accumulators more than once. The later could lead to WRONG resutls, as + * accumulator merging happens in-place, thus merging more than once can lead to duplicate entries. * @return The accumulator map */ public Map> aggregateSmallUserAccumulators() { + if(mergedSmallUserAccumulators != null) { + return mergedSmallUserAccumulators; + } - Map> smallUserAccumulators = new HashMap>(); - + this.mergedSmallUserAccumulators = new HashMap>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { Map> next = vertex.getCurrentExecutionAttempt().getSmallUserAccumulators(); if (next != null) { - AccumulatorHelper.mergeInto(smallUserAccumulators, next); + AccumulatorHelper.mergeInto(mergedSmallUserAccumulators, next); } } - return smallUserAccumulators; + return mergedSmallUserAccumulators; } /** * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, - * thus had to be sent through the BlobCache. + * thus had to be sent through the BlobCache. If the lists of BlobKeys have already been merged, + * then the already computed result is returned. This is to avoid merging accumulators more than + * once. The later could lead to WRONG resutls, as accumulator merging happens in-place, thus + * merging more than once can lead to duplicate entries. * @return The accumulator map */ public Map> aggregateLargeUserAccumulatorBlobKeys() { - Map> largeUserAccumulatorRefs = new HashMap>(); + if(mergedLargeUserAccumulators != null) { + return mergedLargeUserAccumulators; + } + this.mergedLargeUserAccumulators = new HashMap>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { - Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); - if (next != null) { - for (Map.Entry> otherEntry : next.entrySet()) { - List existing = largeUserAccumulatorRefs.get(otherEntry.getKey()); - if (existing == null) { - largeUserAccumulatorRefs.put(otherEntry.getKey(), otherEntry.getValue()); - } else { - existing.addAll(otherEntry.getValue()); - } - } + Map> next = vertex.getCurrentExecutionAttempt(). + getLargeUserAccumulatorBlobKeys(); + mergeLargeUserAccumulatorBlobKeys(next); + } + return mergedLargeUserAccumulators; + } + + public Map> addLargeUserAccumulatorBlobKeys(Map> toMerge) { + if(mergedLargeUserAccumulators == null) { + this.mergedLargeUserAccumulators = new HashMap>();; + } + mergeLargeUserAccumulatorBlobKeys(toMerge); + return mergedLargeUserAccumulators; + } + + private void mergeLargeUserAccumulatorBlobKeys(Map> toMerge) { + if(toMerge == null || toMerge.isEmpty()) { + return; + } + + for (Map.Entry> otherEntry : toMerge.entrySet()) { + List existing = mergedLargeUserAccumulators.get(otherEntry.getKey()); + if (existing == null) { + mergedLargeUserAccumulators.put(otherEntry.getKey(), otherEntry.getValue()); + } else { + existing.addAll(otherEntry.getValue()); } } - return largeUserAccumulatorRefs; } /** - * Gets a serialized accumulator map. + * Gets a serialized map of the contents of the accumulators. * @return The accumulator map with serialized accumulator values. * @throws IOException */ + public Map> getSmallAccumulatorsContentSerialized() throws IOException { + return serializeAccumulators(true); + } + + /** + * Gets a serialized map of the objects of the accumulators. This means that the actual + * objects are serialized, thus merging can still be applied after deserialization. + * @return The accumulator map with serialized accumulator objects. + * @throws IOException + */ public Map> getSmallAccumulatorsSerialized() throws IOException { + return serializeAccumulators(false); + } + + private Map> serializeAccumulators(boolean onlyContent) throws IOException { Map> accumulatorMap = aggregateSmallUserAccumulators(); Map> result = new HashMap>(); for (Map.Entry> entry : accumulatorMap.entrySet()) { - result.put(entry.getKey(), new SerializedValue(entry.getValue().getLocalValue())); + Object toSerialize = onlyContent ? entry.getValue().getLocalValue() : entry.getValue(); + result.put(entry.getKey(), new SerializedValue(toSerialize)); } return result; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java index 8fdf5351d1bc4..08af378af0f8f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java @@ -60,6 +60,14 @@ public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundE return serializedData == null ? null : (T) InstantiationUtil.deserializeObject(serializedData, loader); } + /** + * Gets the content of the value in a byte array. + * @return the byte array backing the content of the object. + * */ + public byte[] getSerializedData() { + return serializedData; + } + /** * Gets the size of the serialized content. * @return the size in bytes of the data stored in the blob. diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 1fcadadb175a6..54c74eda087cb 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -426,4 +426,11 @@ object AkkaUtils { ConfigConstants.AKKA_FRAMESIZE, ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d.]", "").toLong + /** + * @return the threshold (in bytes) above which an object is considered too big + * to transfer using akka. For now this parameter is set to 80% of the + * akka.framesize. + * */ + def getLargeAccumulatorThreshold(config: Configuration): Long = + (0.8 * getFramesize(config)).toLong } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 2de2d646e17d4..f67de4c5ae3ed 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -30,7 +30,7 @@ import grizzled.slf4j.Logger import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot +import org.apache.flink.runtime.accumulators.{BaseAccumulatorSnapshot, LargeAccumulatorHelper} import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.blob.{BlobKey, BlobServer} import org.apache.flink.runtime.client._ @@ -333,17 +333,58 @@ class JobManager( newJobStatus match { case JobStatus.FINISHED => + val jobConfig = currentJobs.getOrElse(jobID, + throw new RuntimeException("Unknown Job: "+ jobID))._1.getJobConfiguration + val smallAccumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { - executionGraph.getSmallAccumulatorsSerialized + executionGraph.getSmallAccumulatorsContentSerialized } catch { case e: Exception => log.error(s"Cannot fetch serialized accumulators for job $jobID", e) Collections.emptyMap() } - val largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = + /* + * The following covers the case where partial accumulator results are small, but when + * aggregated, they become big. In this case, this happens at the JobManager, and this code + * is responsible for detecting it, storing the oversized result in the BlobCache, and + * informing the Client accordingly. + * */ + var largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = executionGraph.aggregateLargeUserAccumulatorBlobKeys() + val totalSize: Long = smallAccumulatorResults.asScala.map(_._2.getSizeInBytes).sum + if (totalSize > AkkaUtils.getLargeAccumulatorThreshold(jobConfig)) { + // given that the client is going to do the final merging, we serialize and + // store the accumulator objects, not only the content + val serializedSmallAccumulators = executionGraph.getSmallAccumulatorsSerialized + + // store the accumulators in the blobCache and get the keys. + val newBlobKeys = LargeAccumulatorHelper.storeSerializedAccumulatorsToBlobCache( + getBlobCacheServerAddress, serializedSmallAccumulators) + smallAccumulatorResults.clear() + + // and update the blobKeys to send to the client. + largeAccumulatorResults = executionGraph.addLargeUserAccumulatorBlobKeys(newBlobKeys) + } else { + // do nothing + java.util.Collections.emptyMap() + } +// +// +// +// val it = newBlobKeys.entrySet().iterator() +// while (it.hasNext) { +// val e = it.next() +// val existingKeys = largeAccumulatorResults.get(e.getKey) +// if(existingKeys == null) { +// largeAccumulatorResults.put(e.getKey, e.getValue) +// } else { +// existingKeys.addAll(e.getValue) +// largeAccumulatorResults.put(e.getKey, existingKeys) +// } +// } + val result = new SerializedJobExecutionResult(jobID, jobInfo.duration, smallAccumulatorResults, largeAccumulatorResults) jobInfo.client ! JobResultSuccess(result) @@ -514,6 +555,19 @@ class JobManager( sender() ! ResponseLeaderSessionID(leaderSessionID) } + /** + * Gets the address where the blobCache is listening to. + * @return the address where the blobCache is listening to. + **/ + private def getBlobCacheServerAddress: InetSocketAddress = { + if (libraryCacheManager == null) { + throw new RuntimeException("LibraryCacheManage is not initialized yet.") + } + val jmHost: String = "localhost" + val blobPort: Int = this.libraryCacheManager.getBlobServerPort + return new InetSocketAddress(jmHost, blobPort) + } + /** * Submits a job to the job manager. The job is registered at the libraryCacheManager which * creates the job's class loader. The job graph is appended to the corresponding execution @@ -769,7 +823,7 @@ class JobManager( try { currentJobs.get(jobID) match { case Some((graph, jobInfo)) => - val accumulatorValues = graph.getSmallAccumulatorsSerialized + val accumulatorValues = graph.getSmallAccumulatorsContentSerialized sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) case None => archive.forward(message) @@ -910,6 +964,8 @@ class JobManager( try { eg.prepareForArchiving() + // todo KOSTAS: handle also the large accumulators. + archive ! decorateMessage(ArchiveExecutionGraph(jobID, eg)) } catch { case t: Throwable => log.error(s"Could not prepare the execution graph $eg for " + @@ -921,6 +977,7 @@ class JobManager( try { libraryCacheManager.unregisterJob(jobID) + } catch { case t: Throwable => log.error(s"Could not properly unregister job $jobID form the library cache.", t) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala index 5b6ba24616fda..d623595ce5b9f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala @@ -132,7 +132,7 @@ class MemoryArchivist(private val max_entries: Int) try { graphs.get(jobID) match { case Some(graph) => - val accumulatorValues = graph.getSmallAccumulatorsSerialized() + val accumulatorValues = graph.getSmallAccumulatorsContentSerialized() sender() ! AccumulatorResultsFound(jobID, accumulatorValues) case None => sender() ! AccumulatorResultsNotFound(jobID) diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 31f406d46bfd7..218b257bd841a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -179,7 +179,7 @@ public void flatMap(Long value, Collector out) { } @Test - public void testOversizedAccumulators() { + public void testOversizedAccumulatorsAtTaskManagers() { try { ExecutionEnvironment env = @@ -201,4 +201,28 @@ public void testOversizedAccumulators() { fail(e.getMessage()); } } + + @Test + public void testOversizedAccumulatorsAtJobManager() { + try { + + ExecutionEnvironment env = + ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + + int noOfParallelism = 5; + int longsInTotal = 1200000; + + env.setParallelism(noOfParallelism); + env.getConfig().disableSysoutLogging(); + + DataSet bigEnough = env.generateSequence(1, longsInTotal); + long theCount = bigEnough.collect().size(); + + assertEquals(longsInTotal, theCount); + + }catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } } From f90f9201bbed97729d7958647730a85b9f9bdd2f Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Mon, 27 Jul 2015 04:48:46 +0300 Subject: [PATCH 006/175] Fixed some checkstyle violations. --- .../flink/runtime/jobmanager/JobManager.scala | 30 ++++++------------- 1 file changed, 9 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index f67de4c5ae3ed..a3ca4634d8775 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -334,7 +334,7 @@ class JobManager( case JobStatus.FINISHED => val jobConfig = currentJobs.getOrElse(jobID, - throw new RuntimeException("Unknown Job: "+ jobID))._1.getJobConfiguration + throw new RuntimeException("Unknown Job: " + jobID))._1.getJobConfiguration val smallAccumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { executionGraph.getSmallAccumulatorsContentSerialized @@ -345,10 +345,10 @@ class JobManager( } /* - * The following covers the case where partial accumulator results are small, but when - * aggregated, they become big. In this case, this happens at the JobManager, and this code - * is responsible for detecting it, storing the oversized result in the BlobCache, and - * informing the Client accordingly. + * The following covers the case where partial accumulator results are small, but + * when aggregated, they become big. In this case, this happens at the JobManager, + * and this code is responsible for detecting it, storing the oversized result in + * the BlobCache, and informing the Client accordingly. * */ var largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = executionGraph.aggregateLargeUserAccumulatorBlobKeys() @@ -365,26 +365,14 @@ class JobManager( smallAccumulatorResults.clear() // and update the blobKeys to send to the client. - largeAccumulatorResults = executionGraph.addLargeUserAccumulatorBlobKeys(newBlobKeys) + largeAccumulatorResults = executionGraph. + addLargeUserAccumulatorBlobKeys(newBlobKeys) + } else { // do nothing java.util.Collections.emptyMap() } -// -// -// -// val it = newBlobKeys.entrySet().iterator() -// while (it.hasNext) { -// val e = it.next() -// val existingKeys = largeAccumulatorResults.get(e.getKey) -// if(existingKeys == null) { -// largeAccumulatorResults.put(e.getKey, e.getValue) -// } else { -// existingKeys.addAll(e.getValue) -// largeAccumulatorResults.put(e.getKey, existingKeys) -// } -// } - + val result = new SerializedJobExecutionResult(jobID, jobInfo.duration, smallAccumulatorResults, largeAccumulatorResults) jobInfo.client ! JobResultSuccess(result) From 13acab525cb733b8ad064153b5a8b0da3cc7f56c Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Mon, 27 Jul 2015 13:48:43 +0300 Subject: [PATCH 007/175] Fixed a broken test. --- .../executiongraph/ExecutionGraph.java | 76 +++++++++++-------- .../flink/runtime/jobmanager/JobManager.scala | 9 ++- 2 files changed, 49 insertions(+), 36 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 6f4ddcfbfd071..0326ddb95d5c5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -141,12 +141,6 @@ public class ExecutionGraph implements Serializable { /** The currently executed tasks, for callbacks */ private final ConcurrentHashMap currentExecutions; - /** This is to guarantee that merging is going to happen only once. */ - private Map> mergedSmallUserAccumulators; - - /** This is to guarantee that merging is going to happen only once. */ - private Map> mergedLargeUserAccumulators; - /** * Updates the accumulators during the runtime of a job. Final accumulator results are transferred * through the UpdateTaskExecutionState message. @@ -577,19 +571,34 @@ public Map> return flinkAccumulators; } + /** + * This works as cache for already merged accumulators, as, in some cases, + * we do not want to remerge accumulators as this may lead to duplicate entries. + * */ + private Map> mergedSmallUserAccumulators; + /** * Merges all accumulator results from the tasks previously executed in the Executions. - * If the accumulators have already been merged, then the already computed result is returned. - * This is to avoid merging accumulators more than once. The later could lead to WRONG resutls, as - * accumulator merging happens in-place, thus merging more than once can lead to duplicate entries. * @return The accumulator map */ public Map> aggregateSmallUserAccumulators() { - if(mergedSmallUserAccumulators != null) { + return aggregateSmallUserAccumulators(true); + } + + /** + * Merges all accumulator results from the tasks previously executed in the Executions. + * If reaggregate is set to false, then no aggregation is performed, and + * the cache merge result is returned. Otherwise accumulators are merged. + * @param reaggregate true if we want to aggregate accumulators, + * false otherwise. + * @return The accumulator map + */ + public Map> aggregateSmallUserAccumulators(boolean reaggregate) { + if(!reaggregate) { return mergedSmallUserAccumulators; } - this.mergedSmallUserAccumulators = new HashMap>(); + for (ExecutionVertex vertex : getAllExecutionVertices()) { Map> next = vertex.getCurrentExecutionAttempt().getSmallUserAccumulators(); if (next != null) { @@ -602,43 +611,46 @@ public Map> aggregateSmallUserAccumulators() { /** * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, - * thus had to be sent through the BlobCache. If the lists of BlobKeys have already been merged, - * then the already computed result is returned. This is to avoid merging accumulators more than - * once. The later could lead to WRONG resutls, as accumulator merging happens in-place, thus - * merging more than once can lead to duplicate entries. + * thus had to be sent through the BlobCache. * @return The accumulator map */ public Map> aggregateLargeUserAccumulatorBlobKeys() { - if(mergedLargeUserAccumulators != null) { - return mergedLargeUserAccumulators; - } + Map> largeUserAccumulatorRefs = new HashMap>(); - this.mergedLargeUserAccumulators = new HashMap>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { - Map> next = vertex.getCurrentExecutionAttempt(). - getLargeUserAccumulatorBlobKeys(); - mergeLargeUserAccumulatorBlobKeys(next); + Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); + mergeLargeUserAccumulatorBlobKeys(largeUserAccumulatorRefs, next); } - return mergedLargeUserAccumulators; + return largeUserAccumulatorRefs; } - public Map> addLargeUserAccumulatorBlobKeys(Map> toMerge) { - if(mergedLargeUserAccumulators == null) { - this.mergedLargeUserAccumulators = new HashMap>();; + /** + * Adds new blobKeys referring to blobs of large accumulators to the already existing ones. + * These refer to blobs in the blobCache holding accumulators (results of tasks) that did not + * fit in an akka frame, thus had to be sent through the BlobCache. + * @param target the initial blobKey map + * @param toMerge the new keys to add to the initial map + * @return The resulting accumulator map + */ + public Map> addLargeUserAccumulatorBlobKeys(Map> target, + Map> toMerge) { + if(target == null) { + target = new HashMap>(); } - mergeLargeUserAccumulatorBlobKeys(toMerge); - return mergedLargeUserAccumulators; + mergeLargeUserAccumulatorBlobKeys(target, toMerge); + return target; } - private void mergeLargeUserAccumulatorBlobKeys(Map> toMerge) { + private void mergeLargeUserAccumulatorBlobKeys(Map> target, + Map> toMerge) { if(toMerge == null || toMerge.isEmpty()) { return; } for (Map.Entry> otherEntry : toMerge.entrySet()) { - List existing = mergedLargeUserAccumulators.get(otherEntry.getKey()); + List existing = target.get(otherEntry.getKey()); if (existing == null) { - mergedLargeUserAccumulators.put(otherEntry.getKey(), otherEntry.getValue()); + target.put(otherEntry.getKey(), otherEntry.getValue()); } else { existing.addAll(otherEntry.getValue()); } @@ -666,7 +678,7 @@ public Map> getSmallAccumulatorsSerialized() thr private Map> serializeAccumulators(boolean onlyContent) throws IOException { - Map> accumulatorMap = aggregateSmallUserAccumulators(); + Map> accumulatorMap = aggregateSmallUserAccumulators(onlyContent); Map> result = new HashMap>(); for (Map.Entry> entry : accumulatorMap.entrySet()) { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index a3ca4634d8775..3e3044befda0e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -344,15 +344,16 @@ class JobManager( Collections.emptyMap() } + var largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = + executionGraph.aggregateLargeUserAccumulatorBlobKeys() + /* * The following covers the case where partial accumulator results are small, but * when aggregated, they become big. In this case, this happens at the JobManager, * and this code is responsible for detecting it, storing the oversized result in * the BlobCache, and informing the Client accordingly. * */ - var largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = - executionGraph.aggregateLargeUserAccumulatorBlobKeys() - + val totalSize: Long = smallAccumulatorResults.asScala.map(_._2.getSizeInBytes).sum if (totalSize > AkkaUtils.getLargeAccumulatorThreshold(jobConfig)) { // given that the client is going to do the final merging, we serialize and @@ -366,7 +367,7 @@ class JobManager( // and update the blobKeys to send to the client. largeAccumulatorResults = executionGraph. - addLargeUserAccumulatorBlobKeys(newBlobKeys) + addLargeUserAccumulatorBlobKeys(largeAccumulatorResults, newBlobKeys) } else { // do nothing From 042b2e81f626621193aabc2a5cd0a6a9ca0e293d Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Mon, 27 Jul 2015 17:29:57 +0200 Subject: [PATCH 008/175] [FLINK-2405] [streaming] Added stateful functions to Scala DataStreams --- docs/apis/streaming_guide.md | 44 +++++++++ .../tasks/StreamingRuntimeContext.java | 2 +- .../streaming/api/scala/DataStream.scala | 89 ++++++++++++++++++- .../api/scala/function/StatefulFunction.scala | 47 ++++++++++ .../scala/api/StatefulFunctionITCase.java | 30 +++++++ .../streaming/api/scala/DataStreamTest.scala | 55 +++++++++--- .../api/scala/StateTestPrograms.scala | 85 ++++++++++++++++++ 7 files changed, 335 insertions(+), 17 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala create mode 100644 flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java create mode 100644 flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index 02da8cbb7c2f6..080272b52f842 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -620,6 +620,50 @@ dataStream.filter{ _ != 0 } + + MapWithState + +

Takes one element and produces one element using a stateful function. Note that the user state object needs to be serializable. +
+
+ A map that produces a rolling average per key:

+{% highlight scala %} +dataStream.keyBy(..).mapWithState((in, state: Option[(Long, Int)]) => state match { + case Some((sum, count)) => ((sum + in)/(count + 1), Some((sum + in, count + 1))) + case None => (in, Some((in, 1))) +}) +{% endhighlight %} + + + + + FlatMapWithState + +

Takes one element and produces zero, one, or more elements using a stateful function. Note that the user state object needs to be serializable.

+{% highlight scala %} +dataStream.flatMapWithState((I,Option[S]) => (Traversable[O], Option[S])) +{% endhighlight %} + + + + + FilterWithState + +

Evaluates a stateful boolean function for each element and retains those for which the function returns true. Note that the user state object needs to be serializable. +
+
+ A filter that only keeps the first 10 elements at each operator instance: +

+{% highlight scala %} +dataStream.filterWithState((in, count: Option[Int]) => count match { + case Some(c) => (c < 10, Some(c+1)) + case None => (true, Some(1)) +}) +{% endhighlight %} + + + + Reduce diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java index 29671f4b0ddc0..8c354bee4dcb4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java @@ -146,7 +146,7 @@ public StreamOperatorState createRawState(boolean partitioned) { return new PartitionedStreamOperatorState(provider, statePartitioner, cl); } else { throw new RuntimeException( - "A partitioning key must be provided for pastitioned state."); + "Partitioned state can only be used with KeyedDataStreams."); } } else { return new StreamOperatorState(provider); diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 2b0f60e44e372..82e88eab4c5fa 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -30,8 +30,6 @@ import org.apache.flink.api.scala.operators.ScalaCsvOutputFormat import org.apache.flink.core.fs.{FileSystem, Path} import org.apache.flink.streaming.api.collector.selector.OutputSelector import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, DataStreamSink, SingleOutputStreamOperator} -import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, DataStreamSink, - GroupedDataStream, SingleOutputStreamOperator} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} import org.apache.flink.streaming.api.functions.sink.SinkFunction @@ -40,6 +38,11 @@ import org.apache.flink.streaming.api.windowing.helper.WindowingHelper import org.apache.flink.streaming.api.windowing.policy.{EvictionPolicy, TriggerPolicy} import org.apache.flink.streaming.util.serialization.SerializationSchema import org.apache.flink.util.Collector +import org.apache.flink.api.common.state.OperatorState +import org.apache.flink.api.common.functions.{RichMapFunction, RichFlatMapFunction, RichFilterFunction} +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.datastream.KeyedDataStream +import org.apache.flink.streaming.api.scala.function.StatefulFunction class DataStream[T](javaStream: JavaStream[T]) { @@ -462,6 +465,31 @@ class DataStream[T](javaStream: JavaStream[T]) { javaStream.map(mapper).returns(outType).asInstanceOf[JavaStream[R]] } + /** + * Creates a new DataStream by applying the given stateful function to every element of this + * DataStream. To use state partitioning, a key must be defined using .keyBy(..), in which + * case an independent state will be kept per key. + * + * Note that the user state object needs to be serializable. + */ + def mapWithState[R: TypeInformation: ClassTag, S]( + fun: (T, Option[S]) => (R, Option[S])): DataStream[R] = { + if (fun == null) { + throw new NullPointerException("Map function must not be null.") + } + + val cleanFun = clean(fun) + val mapper = new RichMapFunction[T, R] with StatefulFunction[T, R, S] { + override def map(in: T): R = { + applyWithState(in, cleanFun) + } + } + + setStatePartitioning(mapper) + + map(mapper) + } + /** * Creates a new DataStream by applying the given function to every element and flattening * the results. @@ -505,7 +533,31 @@ class DataStream[T](javaStream: JavaStream[T]) { flatMap(flatMapper) } - + /** + * Creates a new DataStream by applying the given stateful function to every element and + * flattening the results. To use state partitioning, a key must be defined using .keyBy(..), + * in which case an independent state will be kept per key. + * + * Note that the user state object needs to be serializable. + */ + def flatMapWithState[R: TypeInformation: ClassTag, S]( + fun: (T, Option[S]) => (TraversableOnce[R], Option[S])): + DataStream[R] = { + if (fun == null) { + throw new NullPointerException("Flatmap function must not be null.") + } + + val cleanFun = clean(fun) + val flatMapper = new RichFlatMapFunction[T, R] with StatefulFunction[T,TraversableOnce[R],S]{ + override def flatMap(in: T, out: Collector[R]): Unit = { + applyWithState(in, cleanFun) foreach out.collect + } + } + + setStatePartitioning(flatMapper) + + flatMap(flatMapper) + } /** * Creates a new DataStream that contains only the elements satisfying the given filter predicate. @@ -530,6 +582,37 @@ class DataStream[T](javaStream: JavaStream[T]) { } this.filter(filter) } + + /** + * Creates a new DataStream that contains only the elements satisfying the given stateful filter + * predicate. To use state partitioning, a key must be defined using .keyBy(..), in which case + * an independent state will be kept per key. + * + * Note that the user state object needs to be serializable. + */ + def filterWithState[S]( + fun: (T, Option[S]) => (Boolean, Option[S])): DataStream[T] = { + if (fun == null) { + throw new NullPointerException("Filter function must not be null.") + } + + val cleanFun = clean(fun) + val filterFun = new RichFilterFunction[T] with StatefulFunction[T, Boolean, S] { + override def filter(in: T): Boolean = { + applyWithState(in, cleanFun) + } + } + + setStatePartitioning(filterFun) + + filter(filterFun) + } + + private[flink] def setStatePartitioning(fun: StatefulFunction[_, _, _]) = { + if (javaStream.isInstanceOf[KeyedDataStream[T]]) { + fun.partitionStateByKey + } + } /** * Create a WindowedDataStream that can be used to apply diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala new file mode 100644 index 0000000000000..cf685395c70a1 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala @@ -0,0 +1,47 @@ +/* + * 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.streaming.api.scala.function + +import org.apache.flink.api.common.functions.RichFunction +import org.apache.flink.configuration.Configuration +import org.apache.flink.api.common.state.OperatorState + +/** + * Trait implementing the functionality necessary to apply stateful functions in + * RichFunctions without exposing the OperatorStates to the user. The user should + * call the applyWithState method in his own RichFunction implementation. + */ +trait StatefulFunction[I, O, S] extends RichFunction { + + var state: OperatorState[Option[S]] = _ + var partitioned: Boolean = false + + def partitionStateByKey = { partitioned = true } + def isPartitioned = partitioned + + def applyWithState(in: I, fun: (I, Option[S]) => (O, Option[S])): O = { + val (o, s) = fun(in, state.value) + state.update(s) + o + } + + override def open(c: Configuration) = { + state = getRuntimeContext().getOperatorState("state", None, isPartitioned) + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java b/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java new file mode 100644 index 0000000000000..731222e73d89c --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java @@ -0,0 +1,30 @@ +/* + * 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.streaming.scala.api; + +import org.apache.flink.streaming.api.scala.StateTestPrograms; +import org.apache.flink.streaming.util.StreamingProgramTestBase; + +public class StatefulFunctionITCase extends StreamingProgramTestBase { + + @Override + protected void testProgram() throws Exception { + StateTestPrograms.testStatefulFunctions(); + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala index 5d44e6bfe3cb7..4a5cbf9c5d8ad 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala @@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.scala import java.lang - import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction, Partitioner, FoldFunction, Function} import org.apache.flink.api.java.typeutils.TypeExtractor @@ -32,6 +31,7 @@ import org.apache.flink.streaming.runtime.partitioner._ import org.apache.flink.util.Collector import org.junit.Assert.fail import org.junit.Test +import org.apache.flink.streaming.api.scala.function.StatefulFunction class DataStreamTest { @@ -310,9 +310,18 @@ class DataStreamTest { }; val map = src.map(mapFunction) assert(mapFunction == getFunctionForDataStream(map)) - assert(getFunctionForDataStream(map.map(x => 0)).isInstanceOf[MapFunction[Int, Int]]) - - + assert(getFunctionForDataStream(map.map(x => 0)).isInstanceOf[MapFunction[_, _]]) + + val statefulMap1 = src.mapWithState((in, state: Option[Long]) => (in, None)) + assert(getFunctionForDataStream(statefulMap1).isInstanceOf[MapFunction[_,_]]) + assert(!getFunctionForDataStream(statefulMap1). + asInstanceOf[StatefulFunction[_,_,_]].isPartitioned) + + val statefulMap2 = src.keyBy(x=>x).mapWithState( + (in, state: Option[Long]) => (in, None)) + assert(getFunctionForDataStream(statefulMap2). + asInstanceOf[StatefulFunction[_,_,_]].isPartitioned) + val flatMapFunction = new FlatMapFunction[Long, Int] { override def flatMap(value: Long, out: Collector[Int]): Unit = {} } @@ -321,8 +330,18 @@ class DataStreamTest { assert( getFunctionForDataStream(flatMap .flatMap((x: Int, out: Collector[Int]) => {})) - .isInstanceOf[FlatMapFunction[Int, Int]]) - + .isInstanceOf[FlatMapFunction[_, _]]) + + val statefulfMap1 = src.flatMapWithState((in, state: Option[Long]) => (List(in), None)) + assert(getFunctionForDataStream(statefulfMap1).isInstanceOf[FlatMapFunction[_, _]]) + assert(!getFunctionForDataStream(statefulfMap1). + asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + + val statefulfMap2 = src.keyBy(x=>x).flatMapWithState( + (in, state: Option[Long]) => (List(in), None)) + assert(getFunctionForDataStream(statefulfMap2). + asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + val filterFunction = new FilterFunction[Int] { override def filter(value: Int): Boolean = false } @@ -332,13 +351,23 @@ class DataStreamTest { assert( getFunctionForDataStream(map .filter((x: Int) => true)) - .isInstanceOf[FilterFunction[Int]]) - + .isInstanceOf[FilterFunction[_]]) + + val statefulFilter1 = src.filterWithState((in, state: Option[Long]) => (true, None)) + assert(getFunctionForDataStream(statefulFilter1).isInstanceOf[FilterFunction[_]]) + assert(!getFunctionForDataStream(statefulFilter1). + asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + + val statefulFilter2 = src.keyBy(x=>x).filterWithState( + (in, state: Option[Long]) => (false, None)) + assert(getFunctionForDataStream(statefulFilter2). + asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + try { streamGraph.getStreamEdge(map.getId, unionFilter.getId) } catch { - case e => { + case e: Throwable => { fail(e.getMessage) } } @@ -347,7 +376,7 @@ class DataStreamTest { streamGraph.getStreamEdge(flatMap.getId, unionFilter.getId) } catch { - case e => { + case e: Throwable => { fail(e.getMessage) } } @@ -378,7 +407,7 @@ class DataStreamTest { assert( getFunctionForDataStream(map.groupBy(x=>x) .fold("", (x: String, y: Int) => "")) - .isInstanceOf[FoldFunction[Int, String]]) + .isInstanceOf[FoldFunction[_, _]]) val connect = fold.connect(flatMap) @@ -395,7 +424,7 @@ class DataStreamTest { streamGraph.getStreamEdge(fold.getId, coMap.getId) } catch { - case e => { + case e: Throwable => { fail(e.getMessage) } } @@ -403,7 +432,7 @@ class DataStreamTest { streamGraph.getStreamEdge(flatMap.getId, coMap.getId) } catch { - case e => { + case e: Throwable => { fail(e.getMessage) } } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala new file mode 100644 index 0000000000000..5fa007ceaecf2 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala @@ -0,0 +1,85 @@ +/* + * 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.streaming.api.scala + +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction +import java.util.HashSet + +/** + * Test programs for stateful functions. + */ +object StateTestPrograms { + + def testStatefulFunctions(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + + // test stateful map + env.generateSequence(0, 10).setParallelism(1). + mapWithState((in, count: Option[Long]) => + count match { + case Some(c) => ((in - c), Some(c + 1)) + case None => (in, Some(1L)) + }).setParallelism(1). + addSink(new RichSinkFunction[Long]() { + var allZero = true + override def invoke(in: Long) = { + if (in != 0) allZero = false + } + override def close() = { + assert(allZero) + } + }) + + // test stateful flatmap + env.fromElements("Fir st-", "Hello world").flatMapWithState((w, s: Option[String]) => + s match { + case Some(s) => (w.split(" ").toList.map(s + _), Some(w)) + case None => (List(w), Some(w)) + }).setParallelism(1). + addSink(new RichSinkFunction[String]() { + val received = new HashSet[String]() + override def invoke(in: String) = { received.add(in) } + override def close() = { + assert(received.size() == 3) + assert(received.contains("Fir st-")) + assert(received.contains("Fir st-Hello")) + assert(received.contains("Fir st-world")) + } + }).setParallelism(1) + + // test stateful filter + env.generateSequence(1, 10).keyBy(_ % 2).filterWithState((in, state: Option[Int]) => + state match { + case Some(s) => (s < 2, Some(s + 1)) + case None => (true, Some(1)) + }).addSink(new RichSinkFunction[Long]() { + var numOdd = 0 + var numEven = 0 + override def invoke(in: Long) = { + if (in % 2 == 0) { numEven += 1 } else { numOdd += 1 } + } + override def close() = { + assert(numOdd == 2) + assert(numEven == 2) + } + }).setParallelism(1) + + env.execute("Stateful test") + } + +} From 4b44e02dd8c9f59271d5fb1df7093928effdc67d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 28 Jul 2015 10:38:28 +0200 Subject: [PATCH 009/175] [FLINK-2405] [streaming] Simplifies the way the partitioned field is set in StatefulFunction Closes #936 --- .../flink/streaming/api/scala/DataStream.scala | 18 ++++++++---------- .../api/scala/function/StatefulFunction.scala | 7 ++----- .../streaming/api/scala/DataStreamTest.scala | 12 ++++++------ 3 files changed, 16 insertions(+), 21 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 82e88eab4c5fa..5e62331a3dda5 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -483,10 +483,10 @@ class DataStream[T](javaStream: JavaStream[T]) { override def map(in: T): R = { applyWithState(in, cleanFun) } + + val partitioned = isStatePartitioned } - setStatePartitioning(mapper) - map(mapper) } @@ -552,10 +552,10 @@ class DataStream[T](javaStream: JavaStream[T]) { override def flatMap(in: T, out: Collector[R]): Unit = { applyWithState(in, cleanFun) foreach out.collect } + + val partitioned = isStatePartitioned } - setStatePartitioning(flatMapper) - flatMap(flatMapper) } @@ -601,17 +601,15 @@ class DataStream[T](javaStream: JavaStream[T]) { override def filter(in: T): Boolean = { applyWithState(in, cleanFun) } + + val partitioned = isStatePartitioned } - setStatePartitioning(filterFun) - filter(filterFun) } - private[flink] def setStatePartitioning(fun: StatefulFunction[_, _, _]) = { - if (javaStream.isInstanceOf[KeyedDataStream[T]]) { - fun.partitionStateByKey - } + private[flink] def isStatePartitioned: Boolean = { + javaStream.isInstanceOf[KeyedDataStream[T]] } /** diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala index cf685395c70a1..89c9d00056f40 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala @@ -30,10 +30,7 @@ import org.apache.flink.api.common.state.OperatorState trait StatefulFunction[I, O, S] extends RichFunction { var state: OperatorState[Option[S]] = _ - var partitioned: Boolean = false - - def partitionStateByKey = { partitioned = true } - def isPartitioned = partitioned + val partitioned: Boolean def applyWithState(in: I, fun: (I, Option[S]) => (O, Option[S])): O = { val (o, s) = fun(in, state.value) @@ -42,6 +39,6 @@ trait StatefulFunction[I, O, S] extends RichFunction { } override def open(c: Configuration) = { - state = getRuntimeContext().getOperatorState("state", None, isPartitioned) + state = getRuntimeContext().getOperatorState("state", None, partitioned) } } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala index 4a5cbf9c5d8ad..39a8fe6ffd059 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala @@ -315,12 +315,12 @@ class DataStreamTest { val statefulMap1 = src.mapWithState((in, state: Option[Long]) => (in, None)) assert(getFunctionForDataStream(statefulMap1).isInstanceOf[MapFunction[_,_]]) assert(!getFunctionForDataStream(statefulMap1). - asInstanceOf[StatefulFunction[_,_,_]].isPartitioned) + asInstanceOf[StatefulFunction[_,_,_]].partitioned) val statefulMap2 = src.keyBy(x=>x).mapWithState( (in, state: Option[Long]) => (in, None)) assert(getFunctionForDataStream(statefulMap2). - asInstanceOf[StatefulFunction[_,_,_]].isPartitioned) + asInstanceOf[StatefulFunction[_,_,_]].partitioned) val flatMapFunction = new FlatMapFunction[Long, Int] { override def flatMap(value: Long, out: Collector[Int]): Unit = {} @@ -335,12 +335,12 @@ class DataStreamTest { val statefulfMap1 = src.flatMapWithState((in, state: Option[Long]) => (List(in), None)) assert(getFunctionForDataStream(statefulfMap1).isInstanceOf[FlatMapFunction[_, _]]) assert(!getFunctionForDataStream(statefulfMap1). - asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + asInstanceOf[StatefulFunction[_, _, _]].partitioned) val statefulfMap2 = src.keyBy(x=>x).flatMapWithState( (in, state: Option[Long]) => (List(in), None)) assert(getFunctionForDataStream(statefulfMap2). - asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + asInstanceOf[StatefulFunction[_, _, _]].partitioned) val filterFunction = new FilterFunction[Int] { override def filter(value: Int): Boolean = false @@ -356,12 +356,12 @@ class DataStreamTest { val statefulFilter1 = src.filterWithState((in, state: Option[Long]) => (true, None)) assert(getFunctionForDataStream(statefulFilter1).isInstanceOf[FilterFunction[_]]) assert(!getFunctionForDataStream(statefulFilter1). - asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + asInstanceOf[StatefulFunction[_, _, _]].partitioned) val statefulFilter2 = src.keyBy(x=>x).filterWithState( (in, state: Option[Long]) => (false, None)) assert(getFunctionForDataStream(statefulFilter2). - asInstanceOf[StatefulFunction[_, _, _]].isPartitioned) + asInstanceOf[StatefulFunction[_, _, _]].partitioned) try { streamGraph.getStreamEdge(map.getId, unionFilter.getId) From 78fd2146dd00da1130910d9f23f09e2504854ef7 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Tue, 28 Jul 2015 15:47:30 +0200 Subject: [PATCH 010/175] [FLINK-2419] [hotfix] addSink now uses transform + remove double checkpoint commit at head operator --- .../flink/streaming/api/datastream/DataStream.java | 12 +----------- .../flink/streaming/runtime/tasks/StreamTask.java | 14 +++++--------- 2 files changed, 6 insertions(+), 20 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 7896169546048..5bd3fb8756585 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -1162,17 +1162,7 @@ protected void connectGraph(DataStream inputStream, Integer outputID, int * @return The closed DataStream. */ public DataStreamSink addSink(SinkFunction sinkFunction) { - - OneInputStreamOperator sinkOperator = new StreamSink(clean(sinkFunction)); - - DataStreamSink returnStream = new DataStreamSink(environment, "sink", getType(), - sinkOperator); - - streamGraph.addOperator(returnStream.getId(), sinkOperator, getType(), null, "Stream Sink"); - - this.connectGraph(this.copy(), returnStream.getId(), 0); - - return returnStream; + return new DataStreamSink((DataStream) transform("StreamSink", null, new StreamSink(clean(sinkFunction)))); } private void validateUnion(Integer id) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 286202ffe9e46..75bdd57e848fc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -279,22 +279,18 @@ public void triggerCheckpoint(long checkpointId, long timestamp) throws Exceptio } - @SuppressWarnings({ "unchecked", "rawtypes" }) + @SuppressWarnings("rawtypes") @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { // we do nothing here so far. this should call commit on the source function, for example synchronized (checkpointLock) { - if (streamOperator instanceof StatefulStreamOperator) { - ((StatefulStreamOperator) streamOperator).notifyCheckpointComplete(checkpointId); - } - if (hasChainedOperators) { - for (StreamOperator chainedOperator : outputHandler.getChainedOperators()) { - if (chainedOperator instanceof StatefulStreamOperator) { - ((StatefulStreamOperator) chainedOperator).notifyCheckpointComplete(checkpointId); - } + for (StreamOperator chainedOperator : outputHandler.getChainedOperators()) { + if (chainedOperator instanceof StatefulStreamOperator) { + ((StatefulStreamOperator) chainedOperator).notifyCheckpointComplete(checkpointId); } } + } } From 571084152f81f984fa4cdfc2122c3edcf5b62abd Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Mon, 27 Jul 2015 16:53:18 +0530 Subject: [PATCH 011/175] [FLINK-2404] add support for primitives to simple accumulator counters This closes #942. --- .../flink/api/common/accumulators/DoubleCounter.java | 7 +++++++ .../flink/api/common/accumulators/IntCounter.java | 7 +++++++ .../flink/api/common/accumulators/LongCounter.java | 11 +++++++++-- 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java index 85d187c59edcb..b7a4bfcd6d73b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java @@ -27,11 +27,18 @@ public class DoubleCounter implements SimpleAccumulator { private double localValue = 0; + /** + * Consider using {@link #add(double)} instead for primitive double values + */ @Override public void add(Double value) { localValue += value; } + public void add(double value){ + localValue += value; + } + @Override public Double getLocalValue() { return localValue; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java index c88c73d068c49..a97ba5124dcd4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java @@ -28,11 +28,18 @@ public class IntCounter implements SimpleAccumulator { private int localValue = 0; + /** + * Consider using {@link #add(int)} instead for primitive int values + */ @Override public void add(Integer value) { localValue += value; } + public void add(int value){ + localValue += value; + } + @Override public Integer getLocalValue() { return localValue; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java index 70f2417b0c31f..93b06d6cc6765 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java @@ -26,12 +26,19 @@ public class LongCounter implements SimpleAccumulator { private static final long serialVersionUID = 1L; private long localValue = 0; - + + /** + * Consider using {@link #add(long)} instead for primitive long values + */ @Override public void add(Long value) { this.localValue += value; } - + + public void add(long value){ + this.localValue += value; + } + @Override public Long getLocalValue() { return this.localValue; From a0556efb233f15c6985d17886372a8b4b00392b2 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 28 Jul 2015 15:07:43 +0200 Subject: [PATCH 012/175] [FLINK-2418] [streaming] Add an end-to-end exactly-once test for Checkpointed functions. --- .../checkpointing/StateCheckpoinedITCase.java | 433 ++++++++++++++++++ 1 file changed, 433 insertions(+) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java new file mode 100644 index 0000000000000..069366516be9f --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java @@ -0,0 +1,433 @@ +/* + * 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.test.checkpointing; + +import org.apache.flink.api.common.functions.RichFilterFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.state.OperatorState; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; + +import org.apache.flink.util.Collector; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * A simple test that runs a streaming topology with checkpointing enabled. + * + * The test triggers a failure after a while and verifies that, after completion, the + * state reflects the "exactly once" semantics. + */ +@SuppressWarnings("serial") +public class StateCheckpoinedITCase { + + private static final int NUM_TASK_MANAGERS = 2; + private static final int NUM_TASK_SLOTS = 3; + private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + + private static ForkableFlinkMiniCluster cluster; + + @BeforeClass + public static void startCluster() { + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); + config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + + cluster = new ForkableFlinkMiniCluster(config, false); + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to start test cluster: " + e.getMessage()); + } + } + + @AfterClass + public static void shutdownCluster() { + try { + cluster.shutdown(); + cluster = null; + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to stop test cluster: " + e.getMessage()); + } + } + + + /** + * Runs the following program: + * + *
+	 *     [ (source)->(filter)->(map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
+	 * 
+ */ + @Test + public void runCheckpointedProgram() { + + final long NUM_STRINGS = 10000000L; + assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); + + try { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(500); + env.getConfig().enableSysoutLogging(); + + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); + + stream + // -------------- first vertex, chained to the source ---------------- + .filter(new StringRichFilterFunction()) + + // -------------- seconds vertex - one-to-one connected ---------------- + .map(new StringPrefixCountRichMapFunction()) + .startNewChain() + .map(new StatefulCounterFunction()) + + // -------------- third vertex - reducer and the sink ---------------- + .partitionByHash("prefix") + .flatMap(new OnceFailingAggregator(NUM_STRINGS)) + .addSink(new ValidatingSink()); + + env.execute(); + + long filterSum = 0; + for (long l : StringRichFilterFunction.counts) { + filterSum += l; + } + + long mapSum = 0; + for (long l : StringPrefixCountRichMapFunction.counts) { + mapSum += l; + } + + long countSum = 0; + for (long l : StatefulCounterFunction.counts) { + countSum += l; + } + + // verify that we counted exactly right + assertEquals(NUM_STRINGS, filterSum); + assertEquals(NUM_STRINGS, mapSum); + assertEquals(NUM_STRINGS, countSum); + + for (Map map : ValidatingSink.maps) { + for (Long count : map.values()) { + assertEquals(NUM_STRINGS / 40, count.longValue()); + } + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + // Custom Functions + // -------------------------------------------------------------------------------------------- + + private static class StringGeneratingSourceFunction extends RichParallelSourceFunction + implements CheckpointedAsynchronously + { + private final long numElements; + + private int index; + + private volatile boolean isRunning = true; + + + StringGeneratingSourceFunction(long numElements) { + this.numElements = numElements; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final Object lockingObject = ctx.getCheckpointLock(); + + final Random rnd = new Random(); + final StringBuilder stringBuilder = new StringBuilder(); + + final int step = getRuntimeContext().getNumberOfParallelSubtasks(); + + if (index == 0) { + index = getRuntimeContext().getIndexOfThisSubtask(); + } + + while (isRunning && index < numElements) { + char first = (char) ((index % 40) + 40); + + stringBuilder.setLength(0); + stringBuilder.append(first); + + String result = randomString(stringBuilder, rnd); + + synchronized (lockingObject) { + index += step; + ctx.collect(result); + } + } + } + + @Override + public void cancel() { + isRunning = false; + } + + private static String randomString(StringBuilder bld, Random rnd) { + final int len = rnd.nextInt(10) + 5; + + for (int i = 0; i < len; i++) { + char next = (char) (rnd.nextInt(20000) + 33); + bld.append(next); + } + + return bld.toString(); + } + + @Override + public Integer snapshotState(long checkpointId, long checkpointTimestamp) { + return index; + } + + @Override + public void restoreState(Integer state) { + index = state; + } + } + + private static class StringRichFilterFunction extends RichFilterFunction implements Checkpointed { + + static final long[] counts = new long[PARALLELISM]; + + private long count; + + @Override + public boolean filter(String value) { + count++; + return value.length() < 100; // should be always true + } + + @Override + public void close() { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count; + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) { + return count; + } + + @Override + public void restoreState(Long state) { + count = state; + } + } + + private static class StringPrefixCountRichMapFunction extends RichMapFunction + implements CheckpointedAsynchronously { + + static final long[] counts = new long[PARALLELISM]; + + private long count; + + @Override + public PrefixCount map(String value) { + count++; + return new PrefixCount(value.substring(0, 1), value, 1L); + } + + @Override + public void close() { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count; + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) { + return count; + } + + @Override + public void restoreState(Long state) { + count = state; + } + } + + private static class StatefulCounterFunction extends RichMapFunction { + + static final long[] counts = new long[PARALLELISM]; + + private OperatorState count; + + @Override + public PrefixCount map(PrefixCount value) throws Exception { + count.update(count.value() + 1); + return value; + } + + @Override + public void open(Configuration conf) throws IOException { + count = getRuntimeContext().getOperatorState("count", 0L, false); + } + + @Override + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value(); + } + + } + + private static class OnceFailingAggregator extends RichFlatMapFunction + implements Checkpointed> { + + private static volatile boolean hasFailed = false; + + private final HashMap aggregationMap = new HashMap(); + + private final long numElements; + + private long failurePos; + private long count; + + + OnceFailingAggregator(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) { + long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); + long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); + + failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; + count = 0; + } + + @Override + public void flatMap(PrefixCount value, Collector out) throws Exception { + count++; + if (!hasFailed && count >= failurePos) { + hasFailed = true; + throw new Exception("Test Failure"); + } + + PrefixCount curr = aggregationMap.get(value.prefix); + if (curr == null) { + aggregationMap.put(value.prefix, value); + out.collect(value); + } + else { + curr.count += value.count; + out.collect(curr); + } + } + + @Override + public HashMap snapshotState(long checkpointId, long checkpointTimestamp) { + return aggregationMap; + } + + @Override + public void restoreState(HashMap state) { + aggregationMap.putAll(state); + } + } + + private static class ValidatingSink extends RichSinkFunction + implements Checkpointed> { + + @SuppressWarnings("unchecked") + private static Map[] maps = (Map[]) new Map[PARALLELISM]; + + private HashMap counts = new HashMap(); + + @Override + public void invoke(PrefixCount value) { + Character first = value.prefix.charAt(0); + Long previous = counts.get(first); + if (previous == null) { + counts.put(first, value.count); + } else { + counts.put(first, Math.max(previous, value.count)); + } + } + + @Override + public void close() throws Exception { + maps[getRuntimeContext().getIndexOfThisSubtask()] = counts; + } + + @Override + public HashMap snapshotState(long checkpointId, long checkpointTimestamp) { + return counts; + } + + @Override + public void restoreState(HashMap state) { + counts.putAll(state); + } + } + + // -------------------------------------------------------------------------------------------- + // Custom Type Classes + // -------------------------------------------------------------------------------------------- + + public static class PrefixCount implements Serializable { + + public String prefix; + public String value; + public long count; + + public PrefixCount() {} + + public PrefixCount(String prefix, String value, long count) { + this.prefix = prefix; + this.value = value; + this.count = count; + } + + @Override + public String toString() { + return prefix + " / " + value; + } + } +} From 8ba321332b994579f387add8bd0855bd29cb33ec Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 28 Jul 2015 15:38:20 +0200 Subject: [PATCH 013/175] [FLINK-2420] [streaming] StreamRecordWriter properly reports exceptions on flush. --- .../runtime/io/RecordWriterOutput.java | 13 +- .../runtime/io/StreamRecordWriter.java | 142 ++++++++++++++---- .../runtime/io/DummyBufferRecycler.java | 34 +++++ .../runtime/io/SpillingBufferOrEventTest.java | 4 +- .../runtime/io/StreamRecordWriterTest.java | 132 ++++++++++++++++ .../checkpointing/StateCheckpoinedITCase.java | 4 +- .../StreamCheckpointingITCase.java | 28 ++-- 7 files changed, 296 insertions(+), 61 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index b656bb555e441..f7d8d47b30aec 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -94,15 +94,16 @@ public void emitWatermark(Watermark mark) { @Override public void close() { - if (recordWriter instanceof StreamRecordWriter) { - ((StreamRecordWriter) recordWriter).close(); - } else { - try { + try { + if (recordWriter instanceof StreamRecordWriter) { + ((StreamRecordWriter) recordWriter).close(); + } else { recordWriter.flush(); - } catch (IOException e) { - e.printStackTrace(); } } + catch (IOException e) { + throw new RuntimeException("Failed to flush final output", e); + } } public void clearBuffers() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java index abae9a446663b..b0e25324a1d37 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java @@ -23,38 +23,61 @@ import org.apache.flink.runtime.io.network.api.writer.ChannelSelector; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector; -public class StreamRecordWriter extends RecordWriter { - - private long timeout; - private boolean flushAlways = false; +import static com.google.common.base.Preconditions.checkArgument; - private OutputFlusher outputFlusher; +/** + * This record writer keeps data in buffers at most for a certain timeout. It spawns a separate thread + * that flushes the outputs in a defined interval, to make sure data does not linger in the buffers for too long. + * + * @param The type of elements written. + */ +public class StreamRecordWriter extends RecordWriter { - public StreamRecordWriter(ResultPartitionWriter writer) { - this(writer, new RoundRobinChannelSelector(), 1000); - } + /** Default name for teh output flush thread, if no name with a task reference is given */ + private static final String DEFAULT_OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher"; + + + /** The thread that periodically flushes the output, to give an upper latency bound */ + private final OutputFlusher outputFlusher; + + /** Flag indicating whether the output should be flushed after every element */ + private final boolean flushAlways; - public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector channelSelector) { - this(writer, channelSelector, 1000); + /** The exception encountered in the flushing thread */ + private Throwable flusherException; + + + + public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector channelSelector, long timeout) { + this(writer, channelSelector, timeout, null); } - + public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector channelSelector, - long timeout) { + long timeout, String taskName) { + super(writer, channelSelector); - - this.timeout = timeout; + + checkArgument(timeout < 0); + if (timeout == 0) { flushAlways = true; - } else { - this.outputFlusher = new OutputFlusher(); + outputFlusher = null; + } + else { + flushAlways = false; + + String threadName = taskName == null ? + DEFAULT_OUTPUT_FLUSH_THREAD_NAME : "Output Timeout Flusher - " + taskName; + + outputFlusher = new OutputFlusher(threadName, timeout); outputFlusher.start(); } } @Override public void emit(T record) throws IOException, InterruptedException { + checkErroneous(); super.emit(record); if (flushAlways) { flush(); @@ -63,46 +86,101 @@ public void emit(T record) throws IOException, InterruptedException { @Override public void broadcastEmit(T record) throws IOException, InterruptedException { + checkErroneous(); super.broadcastEmit(record); if (flushAlways) { flush(); } } - public void close() { - try { - if (outputFlusher != null) { + /** + * Closes the writer. This stops the flushing thread (if there is one) and flushes all pending outputs. + * + * @throws IOException I/O errors may happen during the final flush of the buffers. + */ + public void close() throws IOException { + // propagate exceptions + flush(); + + if (outputFlusher != null) { + try { outputFlusher.terminate(); outputFlusher.join(); } + catch (InterruptedException e) { + // ignore on close + } + } - flush(); - } catch (IOException e) { - throw new RuntimeException(e); - } catch (InterruptedException e) { - // Do nothing here + // final check for asynchronous errors, before we exit with a green light + checkErroneous(); + } + + /** + * Notifies the writer that teh output flusher thread encountered an exception. + * + * @param t The exception to report. + */ + void notifyFlusherException(Throwable t) { + if (this.flusherException == null) { + this.flusherException = t; + } + } + + private void checkErroneous() throws IOException { + if (flusherException != null) { + throw new IOException("An exception happened while flushing the outputs", flusherException); } } + // ------------------------------------------------------------------------ + + /** + * A dedicated thread that periodically flushes the output buffers, to set upper latency bounds. + * + * The thread is daemonic, because it is only a utility thread. + */ private class OutputFlusher extends Thread { + + private final long timeout; + private volatile boolean running = true; + + OutputFlusher(String name, long timeout) { + super(name); + setDaemon(true); + this.timeout = timeout; + } + public void terminate() { running = false; + interrupt(); } @Override public void run() { - while (running) { - try { + try { + while (running) { + try { + Thread.sleep(timeout); + } + catch (InterruptedException e) { + // propagate this if we are still running, because it should not happen + // in that case + if (running) { + throw new Exception(e); + } + } + + // any errors here should let the thread come to a halt and be + // recognized by the writer flush(); - Thread.sleep(timeout); - } catch (InterruptedException e) { - // Do nothing here - } catch (IOException e) { - throw new RuntimeException(e); } } + catch (Throwable t) { + notifyFlusherException(t); + } } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java new file mode 100644 index 0000000000000..23ca86dfe6e42 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java @@ -0,0 +1,34 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; + +/** + * A BufferRecycler that does nothing. + */ +public class DummyBufferRecycler implements BufferRecycler { + + public static final BufferRecycler INSTANCE = new DummyBufferRecycler(); + + + @Override + public void recycle(MemorySegment memorySegment) {} +} \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java index e0fab17364e70..9934bd9d7d198 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java @@ -26,9 +26,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.streaming.runtime.io.BufferSpiller; -import org.apache.flink.streaming.runtime.io.SpillReader; -import org.apache.flink.streaming.runtime.io.SpillingBufferOrEvent; + import org.junit.Test; public class SpillingBufferOrEventTest { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java new file mode 100644 index 0000000000000..b5bece7e1b2ce --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java @@ -0,0 +1,132 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.api.writer.ChannelSelector; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.types.LongValue; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.io.IOException; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +/** + * This test uses the PowerMockRunner runner to work around the fact that the + * {@link ResultPartitionWriter} class is final. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(ResultPartitionWriter.class) +public class StreamRecordWriterTest { + + /** + * Verifies that exceptions during flush from the output flush thread are + * recognized in the writer. + */ + @Test + public void testPropagateAsyncFlushError() { + FailingWriter testWriter = null; + try { + ResultPartitionWriter mockResultPartitionWriter = getMockWriter(5); + + // test writer that flushes every 5ms and fails after 3 flushes + testWriter = new FailingWriter(mockResultPartitionWriter, + new RoundRobinChannelSelector(), 5, 3); + + try { + long deadline = System.currentTimeMillis() + 20000; // in max 20 seconds (conservative) + long l = 0L; + + while (System.currentTimeMillis() < deadline) { + testWriter.emit(new LongValue(l++)); + } + + fail("This should have failed with an exception"); + } + catch (IOException e) { + assertNotNull(e.getCause()); + assertTrue(e.getCause().getMessage().contains("Test Exception")); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + finally { + if (testWriter != null) { + try { + testWriter.close(); + } + catch (IOException e) { + // ignore in tests + } + } + } + } + + private static ResultPartitionWriter getMockWriter(int numPartitions) throws Exception { + BufferProvider mockProvider = mock(BufferProvider.class); + when(mockProvider.requestBufferBlocking()).thenAnswer(new Answer() { + @Override + public Buffer answer(InvocationOnMock invocation) { + return new Buffer(new MemorySegment(new byte[4096]), DummyBufferRecycler.INSTANCE); + } + }); + + ResultPartitionWriter mockWriter = mock(ResultPartitionWriter.class); + when(mockWriter.getBufferProvider()).thenReturn(mockProvider); + when(mockWriter.getNumberOfOutputChannels()).thenReturn(numPartitions); + + + return mockWriter; + } + + + // ------------------------------------------------------------------------ + + private static class FailingWriter extends StreamRecordWriter { + + private int flushesBeforeException; + + private FailingWriter(ResultPartitionWriter writer, ChannelSelector channelSelector, + long timeout, int flushesBeforeException) { + super(writer, channelSelector, timeout); + this.flushesBeforeException = flushesBeforeException; + } + + @Override + public void flush() throws IOException { + if (flushesBeforeException-- <= 0) { + throw new IOException("Test Exception"); + } + super.flush(); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java index 069366516be9f..39ff2e512d034 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java @@ -31,8 +31,8 @@ import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.test.util.ForkableFlinkMiniCluster; - import org.apache.flink.util.Collector; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -110,7 +110,7 @@ public void runCheckpointedProgram() { "localhost", cluster.getJobManagerRPCPort()); env.setParallelism(PARALLELISM); env.enableCheckpointing(500); - env.getConfig().enableSysoutLogging(); + env.getConfig().disableSysoutLogging(); DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java index 1730c63363412..438e980c2bb04 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java @@ -18,40 +18,32 @@ package org.apache.flink.test.checkpointing; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Random; - import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.state.OperatorState; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.StateHandle; -import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.test.util.ForkableFlinkMiniCluster; -import org.apache.flink.util.Collector; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * A simple test that runs a streaming topology with checkpointing enabled. From acae9ff2583384dada84b40a89d3a068e3b2a00c Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 28 Jul 2015 18:07:45 +0200 Subject: [PATCH 014/175] [hotfix] Fix generics for stream record and watermark multiplexing. --- .../plugable/SerializationDelegate.java | 7 +- .../runtime/io/RecordWriterOutput.java | 21 ++--- .../runtime/io/StreamInputProcessor.java | 25 +++--- .../runtime/io/StreamRecordWriter.java | 2 +- .../runtime/io/StreamTwoInputProcessor.java | 57 +++++++------ .../MultiplexingStreamRecordSerializer.java | 35 ++++++-- .../runtime/streamrecord/StreamRecord.java | 16 ++-- .../streamrecord/StreamRecordSerializer.java | 65 +++++++------- .../runtime/tasks/OutputHandler.java | 85 +++++++++++-------- .../runtime/tasks/StreamTaskTestHarness.java | 7 +- 10 files changed, 185 insertions(+), 135 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/SerializationDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/SerializationDelegate.java index 3cbaac3a98198..91b6dd934dd17 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/SerializationDelegate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/SerializationDelegate.java @@ -26,7 +26,12 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; - +/** + * The serialization delegate exposes an arbitrary element as a {@link IOReadableWritable} for + * serialization, with the help of a type serializer. + * + * @param The type to be represented as an IOReadableWritable. + */ public class SerializationDelegate implements IOReadableWritable { private T instance; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index f7d8d47b30aec..de8c205dd677e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -39,32 +39,33 @@ public class RecordWriterOutput implements Output> { private static final Logger LOG = LoggerFactory.getLogger(RecordWriterOutput.class); - private RecordWriter>> recordWriter; - private SerializationDelegate> serializationDelegate; + private RecordWriter> recordWriter; + + private SerializationDelegate serializationDelegate; @SuppressWarnings("unchecked") public RecordWriterOutput( - RecordWriter>> recordWriter, + RecordWriter recordWriter, TypeSerializer outSerializer, boolean enableWatermarkMultiplexing) { + Preconditions.checkNotNull(recordWriter); - this.recordWriter = recordWriter; + this.recordWriter = (RecordWriter>) recordWriter; - StreamRecordSerializer outRecordSerializer; + TypeSerializer outRecordSerializer; if (enableWatermarkMultiplexing) { outRecordSerializer = new MultiplexingStreamRecordSerializer(outSerializer); } else { - outRecordSerializer = new StreamRecordSerializer(outSerializer); + outRecordSerializer = (TypeSerializer) (TypeSerializer) new StreamRecordSerializer(outSerializer); } if (outSerializer != null) { - serializationDelegate = new SerializationDelegate(outRecordSerializer); + serializationDelegate = new SerializationDelegate(outRecordSerializer); } } @Override - @SuppressWarnings("unchecked") public void collect(StreamRecord record) { serializationDelegate.setInstance(record); @@ -79,9 +80,9 @@ public void collect(StreamRecord record) { } @Override - @SuppressWarnings("unchecked,rawtypes") public void emitWatermark(Watermark mark) { - ((SerializationDelegate)serializationDelegate).setInstance(mark); + serializationDelegate.setInstance(mark); + try { recordWriter.broadcastEmit(serializationDelegate); } catch (Exception e) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 4c40e5f334b02..9db017807d0a0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -46,9 +46,8 @@ /** * Input reader for {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}. * - *

- * This also keeps track of {@link Watermark} events and forwards them to event subscribers - * once the {@link Watermark} from all inputs advances. + *

This also keeps track of {@link Watermark} events and forwards them to event subscribers + * once the {@link Watermark} from all inputs advances.

* * @param The type of the record that can be read with this record reader. */ @@ -63,33 +62,35 @@ public class StreamInputProcessor extends AbstractReader implements ReaderBa // We need to keep track of the channel from which a buffer came, so that we can // appropriately map the watermarks to input channels - int currentChannel = -1; + private int currentChannel = -1; private boolean isFinished; private final BarrierBuffer barrierBuffer; - private long[] watermarks; + private final long[] watermarks; private long lastEmittedWatermark; - private DeserializationDelegate deserializationDelegate; + private final DeserializationDelegate deserializationDelegate; @SuppressWarnings("unchecked") public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSerializer, boolean enableWatermarkMultiplexing) { super(InputGateUtil.createInputGate(inputGates)); barrierBuffer = new BarrierBuffer(inputGate, this); - - StreamRecordSerializer inputRecordSerializer; + if (enableWatermarkMultiplexing) { - inputRecordSerializer = new MultiplexingStreamRecordSerializer(inputSerializer); + MultiplexingStreamRecordSerializer ser = new MultiplexingStreamRecordSerializer(inputSerializer); + this.deserializationDelegate = new NonReusingDeserializationDelegate(ser); } else { - inputRecordSerializer = new StreamRecordSerializer(inputSerializer); + StreamRecordSerializer ser = new StreamRecordSerializer(inputSerializer); + this.deserializationDelegate = (NonReusingDeserializationDelegate) + (NonReusingDeserializationDelegate) new NonReusingDeserializationDelegate>(ser); } - this.deserializationDelegate = new NonReusingDeserializationDelegate(inputRecordSerializer); - + // Initialize one deserializer per input channel this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; + for (int i = 0; i < recordDeserializers.length; i++) { recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer>(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java index b0e25324a1d37..321f3b4ec1fd8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java @@ -58,7 +58,7 @@ public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector chann super(writer, channelSelector); - checkArgument(timeout < 0); + checkArgument(timeout >= 0); if (timeout == 0) { flushAlways = true; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 82e79360483f8..e235ffeae3bb0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -64,64 +64,70 @@ public class StreamTwoInputProcessor extends AbstractReader implements // We need to keep track of the channel from which a buffer came, so that we can // appropriately map the watermarks to input channels - int currentChannel = -1; + private int currentChannel = -1; private boolean isFinished; private final BarrierBuffer barrierBuffer; - private long[] watermarks1; + private final long[] watermarks1; private long lastEmittedWatermark1; - private long[] watermarks2; + private final long[] watermarks2; private long lastEmittedWatermark2; - private int numInputChannels1; - private int numInputChannels2; + private final int numInputChannels1; - private DeserializationDelegate deserializationDelegate1; - private DeserializationDelegate deserializationDelegate2; + private final DeserializationDelegate deserializationDelegate1; + private final DeserializationDelegate deserializationDelegate2; - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "rawtypes"}) public StreamTwoInputProcessor( Collection inputGates1, Collection inputGates2, TypeSerializer inputSerializer1, TypeSerializer inputSerializer2, boolean enableWatermarkMultiplexing) { + super(InputGateUtil.createInputGate(inputGates1, inputGates2)); barrierBuffer = new BarrierBuffer(inputGate, this); - - StreamRecordSerializer inputRecordSerializer1; + if (enableWatermarkMultiplexing) { - inputRecordSerializer1 = new MultiplexingStreamRecordSerializer(inputSerializer1); - } else { - inputRecordSerializer1 = new StreamRecordSerializer(inputSerializer1); + MultiplexingStreamRecordSerializer ser = new MultiplexingStreamRecordSerializer(inputSerializer1); + this.deserializationDelegate1 = new NonReusingDeserializationDelegate(ser); } - this.deserializationDelegate1 = new NonReusingDeserializationDelegate(inputRecordSerializer1); - - StreamRecordSerializer inputRecordSerializer2; + else { + StreamRecordSerializer ser = new StreamRecordSerializer(inputSerializer1); + this.deserializationDelegate1 = (DeserializationDelegate) + (DeserializationDelegate) new NonReusingDeserializationDelegate>(ser); + } + if (enableWatermarkMultiplexing) { - inputRecordSerializer2 = new MultiplexingStreamRecordSerializer(inputSerializer2); - } else { - inputRecordSerializer2 = new StreamRecordSerializer(inputSerializer2); + MultiplexingStreamRecordSerializer ser = new MultiplexingStreamRecordSerializer(inputSerializer2); + this.deserializationDelegate2 = new NonReusingDeserializationDelegate(ser); + } + else { + StreamRecordSerializer ser = new StreamRecordSerializer(inputSerializer2); + this.deserializationDelegate2 = (DeserializationDelegate) + (DeserializationDelegate) new NonReusingDeserializationDelegate>(ser); } - this.deserializationDelegate2 = new NonReusingDeserializationDelegate(inputRecordSerializer2); // Initialize one deserializer per input channel - this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate - .getNumberOfInputChannels()]; + this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; + for (int i = 0; i < recordDeserializers.length; i++) { - recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(); + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer>(); } // determine which unioned channels belong to input 1 and which belong to input 2 - numInputChannels1 = 0; + int numInputChannels1 = 0; for (InputGate gate: inputGates1) { numInputChannels1 += gate.getNumberOfInputChannels(); } - numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1; + + this.numInputChannels1 = numInputChannels1; + int numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1; watermarks1 = new long[numInputChannels1]; for (int i = 0; i < numInputChannels1; i++) { @@ -262,6 +268,7 @@ public void clearBuffers() { } } + @Override public void cleanup() throws IOException { barrierBuffer.cleanup(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java index 715f0d28bdecb..075c4fcdc6b49 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.streamrecord; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -35,17 +36,36 @@ * * @param The type of value in the {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord} */ -public final class MultiplexingStreamRecordSerializer extends StreamRecordSerializer { - - private final long IS_WATERMARK = Long.MIN_VALUE; +public final class MultiplexingStreamRecordSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; + private static final long IS_WATERMARK = Long.MIN_VALUE; + + protected final TypeSerializer typeSerializer; + + public MultiplexingStreamRecordSerializer(TypeSerializer serializer) { - super(serializer); - if (serializer instanceof MultiplexingStreamRecordSerializer) { + if (serializer instanceof MultiplexingStreamRecordSerializer || serializer instanceof StreamRecordSerializer) { throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer); } + this.typeSerializer = Preconditions.checkNotNull(serializer); + } + + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return this; + } + + @Override + public Object createInstance() { + return new StreamRecord(typeSerializer.createInstance(), 0L); } @Override @@ -80,6 +100,11 @@ public Object copy(Object from, Object reuse) { } } + @Override + public int getLength() { + return 0; + } + @Override @SuppressWarnings("unchecked") public void serialize(Object value, DataOutputView target) throws IOException { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java index 6521e7fcde725..92ce66f5226e3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java @@ -19,12 +19,13 @@ /** * One value in a data stream. This stores the value and the associated timestamp. + * + * @param The type encapsulated with the stream record. */ public class StreamRecord { - - // We store it as Object so that we can reuse a StreamElement for emitting - // elements of a different type while still reusing the timestamp. - private Object value; + + private T value; + private long timestamp; /** @@ -52,9 +53,8 @@ public StreamRecord(T value, long timestamp) { /** * Returns the value wrapped in this stream value. */ - @SuppressWarnings("unchecked") public T getValue() { - return (T) value; + return value; } /** @@ -74,7 +74,7 @@ public long getTimestamp() { */ @SuppressWarnings("unchecked") public StreamRecord replace(X element) { - this.value = element; + this.value = (T) element; return (StreamRecord) this; } @@ -90,7 +90,7 @@ public StreamRecord replace(X element) { @SuppressWarnings("unchecked") public StreamRecord replace(X value, long timestamp) { this.timestamp = timestamp; - this.value = value; + this.value = (T) value; return (StreamRecord) this; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java index 2619891fbecd2..e58d3c89f1875 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java @@ -38,11 +38,12 @@ * * @param The type of value in the {@link StreamRecord} */ -public class StreamRecordSerializer extends TypeSerializer { +public final class StreamRecordSerializer extends TypeSerializer> { private static final long serialVersionUID = 1L; - protected final TypeSerializer typeSerializer; + private final TypeSerializer typeSerializer; + public StreamRecordSerializer(TypeSerializer serializer) { if (serializer instanceof StreamRecordSerializer) { @@ -51,19 +52,36 @@ public StreamRecordSerializer(TypeSerializer serializer) { this.typeSerializer = Preconditions.checkNotNull(serializer); } + public TypeSerializer getContainedTypeSerializer() { + return this.typeSerializer; + } + + // ------------------------------------------------------------------------ + // General serializer and type utils + // ------------------------------------------------------------------------ + + @Override + public StreamRecordSerializer duplicate() { + TypeSerializer serializerCopy = typeSerializer.duplicate(); + return serializerCopy == typeSerializer ? this : new StreamRecordSerializer(serializerCopy); + } + @Override public boolean isImmutableType() { return false; } @Override - @SuppressWarnings("unchecked") - public StreamRecordSerializer duplicate() { - return this; + public int getLength() { + return typeSerializer.getLength(); } + // ------------------------------------------------------------------------ + // Type serialization, copying, instantiation + // ------------------------------------------------------------------------ + @Override - public Object createInstance() { + public StreamRecord createInstance() { try { return new StreamRecord(typeSerializer.createInstance()); } catch (Exception e) { @@ -72,46 +90,31 @@ public Object createInstance() { } @Override - @SuppressWarnings("unchecked") - public Object copy(Object from) { - StreamRecord fromRecord = (StreamRecord) from; - return new StreamRecord(typeSerializer.copy(fromRecord.getValue()), fromRecord.getTimestamp()); + public StreamRecord copy(StreamRecord from) { + return new StreamRecord(typeSerializer.copy(from.getValue()), from.getTimestamp()); } @Override - @SuppressWarnings("unchecked") - public Object copy(Object from, Object reuse) { - StreamRecord fromRecord = (StreamRecord) from; - StreamRecord reuseRecord = (StreamRecord) reuse; - - reuseRecord.replace(typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue()), 0); + public StreamRecord copy(StreamRecord from, StreamRecord reuse) { + reuse.replace(typeSerializer.copy(from.getValue(), reuse.getValue()), 0); return reuse; } @Override - public int getLength() { - return -1; - } - - @Override - @SuppressWarnings("unchecked") - public void serialize(Object value, DataOutputView target) throws IOException { - StreamRecord record = (StreamRecord) value; - typeSerializer.serialize(record.getValue(), target); + public void serialize(StreamRecord value, DataOutputView target) throws IOException { + typeSerializer.serialize(value.getValue(), target); } @Override - public Object deserialize(DataInputView source) throws IOException { + public StreamRecord deserialize(DataInputView source) throws IOException { T element = typeSerializer.deserialize(source); return new StreamRecord(element, 0); } @Override - @SuppressWarnings("unchecked") - public Object deserialize(Object reuse, DataInputView source) throws IOException { - StreamRecord reuseRecord = (StreamRecord) reuse; - T element = typeSerializer.deserialize(reuseRecord.getValue(), source); - reuseRecord.replace(element, 0); + public StreamRecord deserialize(StreamRecord reuse, DataInputView source) throws IOException { + T element = typeSerializer.deserialize(reuse.getValue(), source); + reuse.replace(element, 0); return reuse; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java index aa55151e4c704..84614bf8c2953 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java @@ -44,28 +44,30 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class OutputHandler { + private static final Logger LOG = LoggerFactory.getLogger(OutputHandler.class); - private StreamTask vertex; - private StreamConfig configuration; - private ClassLoader cl; - private Output> outerOutput; + private final StreamTask vertex; + + /** The classloader used to access all user code */ + private final ClassLoader userCodeClassloader; + + + private final Output> outerOutput; - public List> chainedOperators; + public final List> chainedOperators; - private Map> outputMap; + private final Map> outputMap; - private Map chainedConfigs; - private List outEdgesInOrder; + private final Map chainedConfigs; - /** - * Counters for the number of records emitted and bytes written. - */ - protected AccumulatorRegistry.Reporter reporter; + /** Counters for the number of records emitted and bytes written. */ + protected final AccumulatorRegistry.Reporter reporter; public OutputHandler(StreamTask vertex, Map> accumulatorMap, @@ -73,17 +75,17 @@ public OutputHandler(StreamTask vertex, Map> ac // Initialize some fields this.vertex = vertex; - this.configuration = new StreamConfig(vertex.getTaskConfiguration()); + StreamConfig configuration = new StreamConfig(vertex.getTaskConfiguration()); this.chainedOperators = new ArrayList>(); this.outputMap = new HashMap>(); - this.cl = vertex.getUserCodeClassLoader(); + this.userCodeClassloader = vertex.getUserCodeClassLoader(); // We read the chained configs, and the order of record writer - // registrations by outputname - this.chainedConfigs = configuration.getTransitiveChainedTaskConfigs(cl); + // registrations by output name + this.chainedConfigs = configuration.getTransitiveChainedTaskConfigs(userCodeClassloader); this.chainedConfigs.put(configuration.getVertexID(), configuration); - this.outEdgesInOrder = configuration.getOutEdgesInOrder(cl); + List outEdgesInOrder = configuration.getOutEdgesInOrder(userCodeClassloader); this.reporter = reporter; @@ -133,25 +135,24 @@ public List> getChainedOperators(){ * @return Returns the output for the chain starting from the given * config */ - @SuppressWarnings({"unchecked", "rawtypes"}) + @SuppressWarnings("unchecked") private Output> createChainedCollector(StreamConfig chainedTaskConfig, Map> accumulatorMap) { - - + // We create a wrapper that will encapsulate the chained operators and // network outputs - OutputSelectorWrapper outputSelectorWrapper = chainedTaskConfig.getOutputSelectorWrapper(cl); + OutputSelectorWrapper outputSelectorWrapper = chainedTaskConfig.getOutputSelectorWrapper(userCodeClassloader); CollectorWrapper wrapper = new CollectorWrapper(outputSelectorWrapper); // Create collectors for the network outputs - for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(cl)) { + for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(userCodeClassloader)) { Output output = outputMap.get(outputEdge); wrapper.addCollector(output, outputEdge); } // Create collectors for the chained outputs - for (StreamEdge outputEdge : chainedTaskConfig.getChainedOutputs(cl)) { + for (StreamEdge outputEdge : chainedTaskConfig.getChainedOutputs(userCodeClassloader)) { Integer outputId = outputEdge.getTargetId(); Output output = createChainedCollector(chainedConfigs.get(outputId), accumulatorMap); @@ -163,11 +164,12 @@ private Output> createChainedCollector(StreamConfig chainedT // The current task is the first chained task at this vertex so we // return the wrapper return (Output>) wrapper; - } else { + } + else { // The current task is a part of the chain so we get the chainable // operator which will be returned and set it up using the wrapper OneInputStreamOperator chainableOperator = - chainedTaskConfig.getStreamOperator(vertex.getUserCodeClassLoader()); + chainedTaskConfig.getStreamOperator(userCodeClassloader); StreamingRuntimeContext chainedContext = vertex.createRuntimeContext(chainedTaskConfig, accumulatorMap); vertex.contexts.add(chainedContext); @@ -177,14 +179,20 @@ private Output> createChainedCollector(StreamConfig chainedT chainedOperators.add(chainableOperator); if (vertex.getExecutionConfig().isObjectReuseEnabled() || chainableOperator.isInputCopyingDisabled()) { return new ChainingOutput(chainableOperator); - } else { - StreamRecordSerializer serializerIn1; + } + else { + TypeSerializer typeSer = chainedTaskConfig.getTypeSerializerIn1(userCodeClassloader); + TypeSerializer> inSerializer; + if (vertex.getExecutionConfig().areTimestampsEnabled()) { - serializerIn1 = new MultiplexingStreamRecordSerializer(chainedTaskConfig.getTypeSerializerIn1(vertex.getUserCodeClassLoader())); - } else { - serializerIn1 = new StreamRecordSerializer(chainedTaskConfig.getTypeSerializerIn1(vertex.getUserCodeClassLoader())); + inSerializer = (TypeSerializer>) + (TypeSerializer) new MultiplexingStreamRecordSerializer(typeSer); } - return new CopyingChainingOutput(chainableOperator, (TypeSerializer>) serializerIn1); + else { + inSerializer = new StreamRecordSerializer(typeSer); + } + + return new CopyingChainingOutput(chainableOperator, inSerializer); } } @@ -244,14 +252,14 @@ public void clearWriters() { } private static class ChainingOutput implements Output> { - protected OneInputStreamOperator operator; + + protected final OneInputStreamOperator operator; public ChainingOutput(OneInputStreamOperator operator) { this.operator = operator; } @Override - @SuppressWarnings("unchecked") public void collect(StreamRecord record) { try { operator.getRuntimeContext().setNextInput(record); @@ -268,7 +276,8 @@ public void collect(StreamRecord record) { public void emitWatermark(Watermark mark) { try { operator.processWatermark(mark); - } catch (Exception e) { + } + catch (Exception e) { if (LOG.isErrorEnabled()) { LOG.error("Could not forward element to operator: {}", e); } @@ -280,10 +289,12 @@ public void emitWatermark(Watermark mark) { public void close() { try { operator.close(); - } catch (Exception e) { + } + catch (Exception e) { if (LOG.isErrorEnabled()) { LOG.error("Could not forward close call to operator.", e); } + throw new RuntimeException(e); } } } @@ -298,12 +309,12 @@ public CopyingChainingOutput(OneInputStreamOperator operator, } @Override - @SuppressWarnings("unchecked") public void collect(StreamRecord record) { try { operator.getRuntimeContext().setNextInput(record); operator.processElement(serializer.copy(record)); - } catch (Exception e) { + } + catch (Exception e) { if (LOG.isErrorEnabled()) { LOG.error("Could not forward element to operator.", e); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index 283243ecab9d5..435831f017721 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -35,7 +35,6 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; @@ -79,9 +78,8 @@ public class StreamTaskTestHarness { private AbstractInvokable task; - private TypeInformation outputType; private TypeSerializer outputSerializer; - private StreamRecordSerializer outputStreamRecordSerializer; + private TypeSerializer outputStreamRecordSerializer; private ConcurrentLinkedQueue outputList; @@ -114,7 +112,6 @@ public StreamTaskTestHarness(AbstractInvokable task, TypeInformation output streamConfig.setChainStart(); streamConfig.setBufferTimeout(0); - this.outputType = outputType; outputSerializer = outputType.createSerializer(executionConfig); outputStreamRecordSerializer = new MultiplexingStreamRecordSerializer(outputSerializer); } @@ -127,7 +124,7 @@ protected void initializeInputs() throws IOException, InterruptedException { @SuppressWarnings("unchecked") private void initializeOutput() { - outputList = new ConcurrentLinkedQueue(); + outputList = new ConcurrentLinkedQueue(); mockEnv.addOutput(outputList, outputStreamRecordSerializer); From 2d237e18a2f7cf21721340933c505bb518c4fc66 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 28 Jul 2015 18:08:19 +0200 Subject: [PATCH 015/175] [FLINK-2421] [streaming] Add tests for basic utility behavior of StreamRecordSerializer (fixed in previous commit). --- .../StreamRecordSerializerTest.java | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java new file mode 100644 index 0000000000000..d48f7f45aeda2 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java @@ -0,0 +1,68 @@ +/* + * 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.streaming.runtime.streamrecord; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +public class StreamRecordSerializerTest { + + @Test + public void testDeepDuplication() { + try { + @SuppressWarnings("unchecked") + TypeSerializer serializer1 = (TypeSerializer) mock(TypeSerializer.class); + @SuppressWarnings("unchecked") + TypeSerializer serializer2 = (TypeSerializer) mock(TypeSerializer.class); + + when(serializer1.duplicate()).thenReturn(serializer2); + + StreamRecordSerializer streamRecSer = new StreamRecordSerializer(serializer1); + assertEquals(serializer1, streamRecSer.getContainedTypeSerializer()); + + StreamRecordSerializer copy = streamRecSer.duplicate(); + assertNotEquals(copy, streamRecSer); + assertNotEquals(copy.getContainedTypeSerializer(), streamRecSer.getContainedTypeSerializer()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testBasicProperties() { + try { + StreamRecordSerializer streamRecSer = new StreamRecordSerializer(LongSerializer.INSTANCE); + + assertFalse(streamRecSer.isImmutableType()); + assertEquals(Long.class, streamRecSer.createInstance().getValue().getClass()); + assertEquals(LongSerializer.INSTANCE.getLength(), streamRecSer.getLength()); + + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +} From ed30ff4de46deb524d9449768d7199d99e3cc0f0 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 12 Jul 2015 19:33:38 +0200 Subject: [PATCH 016/175] [tests] Add a manual test to evaluate impact of checkpointing on latency --- .../StreamingScalabilityAndLatency.java | 154 ++++++++++++++++++ .../flink/test/manual/package-info.java | 4 +- 2 files changed, 156 insertions(+), 2 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java new file mode 100644 index 0000000000000..a34ec1576f710 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java @@ -0,0 +1,154 @@ +/* + * 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.test.manual; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; + +import static org.junit.Assert.fail; + +public class StreamingScalabilityAndLatency { + + public static void main(String[] args) throws Exception { + if ((Runtime.getRuntime().maxMemory() >>> 20) < 5000) { + throw new RuntimeException("This test program needs to run with at least 5GB of heap space."); + } + + final int TASK_MANAGERS = 1; + final int SLOTS_PER_TASK_MANAGER = 80; + final int PARALLELISM = TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; + + LocalFlinkMiniCluster cluster = null; + + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, TASK_MANAGERS); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 80); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TASK_MANAGER); + config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 20000); + + config.setInteger("taskmanager.net.server.numThreads", 1); + config.setInteger("taskmanager.net.client.numThreads", 1); + + cluster = new LocalFlinkMiniCluster(config, false, StreamingMode.STREAMING); + + runPartitioningProgram(cluster.getJobManagerRPCPort(), PARALLELISM); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + private static void runPartitioningProgram(int jobManagerPort, int parallelism) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort); + env.setParallelism(parallelism); + env.getConfig().enableObjectReuse(); + + env.setBufferTimeout(5L); +// env.enableCheckpointing(1000); + + env + .addSource(new TimeStampingSource()) + .map(new IdMapper>()) + .partitionByHash(0) + .addSink(new TimestampingSink()); + + env.execute("Partitioning Program"); + } + + public static class TimeStampingSource implements ParallelSourceFunction> { + + private static final long serialVersionUID = -151782334777482511L; + + private volatile boolean running = true; + + + @Override + public void run(SourceContext> ctx) throws Exception { + + long num = 100; + long counter = (long) (Math.random() * 4096); + + while (running) { + if (num < 100) { + num++; + ctx.collect(new Tuple2(counter++, 0L)); + } + else { + num = 0; + ctx.collect(new Tuple2(counter++, System.currentTimeMillis())); + } + + Thread.sleep(1); + } + } + + @Override + public void cancel() { + running = false; + } + } + + public static class TimestampingSink implements SinkFunction> { + + private static final long serialVersionUID = 1876986644706201196L; + + private long maxLatency; + private long count; + + @Override + public void invoke(Tuple2 value) { + long ts = value.f1; + if (ts != 0L) { + long diff = System.currentTimeMillis() - ts; + maxLatency = Math.max(diff, maxLatency); + } + + count++; + if (count == 5000) { + System.out.println("Max latency: " + maxLatency); + count = 0; + maxLatency = 0; + } + } + } + + public static class IdMapper implements MapFunction { + + private static final long serialVersionUID = -6543809409233225099L; + + @Override + public T map(T value) { + return value; + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java b/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java index 893f3cc90469d..1c5744d5379b7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java @@ -18,7 +18,7 @@ /** * This package contains various tests that are not automatically executed, but - * need to be manually invoked, because they are extremely heavy of require larger-than-usual - * JVMs. + * need to be manually invoked, because they are extremely heavy, time intensive, + * of require larger-than-usual JVMs. */ package org.apache.flink.test.manual; \ No newline at end of file From 0579f90bab165a7df336163eb9d6337267020029 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 26 Jul 2015 18:58:37 +0200 Subject: [PATCH 017/175] [FLINK-2406] [streaming] Abstract and improve stream alignment via the BarrierBuffer - Add an interface for the functionaliy of the barrier buffer (for later addition of other implementatiions) - Add broader tests for the BarrierBuffer, inluding trailing data and barrier races. - Checkpoint barriers are handled by the buffer directly, rather than being returned and re-injected. - Simplify logic in the BarrierBuffer and fix certain corner cases. - Give access to spill directories properly via I/O manager, rather than via GlobalConfiguration singleton. - Rename the "BarrierBufferIOTest" to "BarrierBufferMassiveRandomTest" - A lot of code style/robustness fixes (proplery define constants, visibility, exception signatures) --- .../runtime/io/disk/iomanager/IOManager.java | 12 +- .../io/network/api/EndOfPartitionEvent.java | 23 +- .../partition/consumer/SingleInputGate.java | 2 +- .../streaming/runtime/io/BarrierBuffer.java | 413 +++++----- .../streaming/runtime/io/BufferSpiller.java | 54 +- .../runtime/io/CheckpointBarrierHandler.java | 55 ++ .../runtime/io/StreamInputProcessor.java | 69 +- .../runtime/io/StreamTwoInputProcessor.java | 56 +- .../runtime/tasks/OneInputStreamTask.java | 37 +- .../streaming/runtime/tasks/StreamTask.java | 16 +- .../runtime/tasks/TwoInputStreamTask.java | 74 +- .../consumer/StreamTestSingleInputGate.java | 13 - ...va => BarrierBufferMassiveRandomTest.java} | 112 +-- .../runtime/io/BarrierBufferTest.java | 775 ++++++++++++++---- .../runtime/io/DummyBufferRecycler.java | 8 +- .../runtime/io/SpillingBufferOrEventTest.java | 20 +- 16 files changed, 1175 insertions(+), 564 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java rename flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/{BarrierBufferIOTest.java => BarrierBufferMassiveRandomTest.java} (57%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java index c0bd3604d598c..45d9b9eaf1cb5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java @@ -301,9 +301,19 @@ public abstract BulkBlockChannelReader createBulkBlockChannelReader(FileIOChanne * * @return The number of temporary file directories. */ - public int getNumberOfTempDirs() { + public int getNumberOfSpillingDirectories() { return this.paths.length; } + + /** + * Gets the directories that the I/O manager spills to. + * + * @return The directories that the I/O manager spills to. + */ + public File[] getSpillingDirectories() { + return this.paths; + } + protected int getNextPathNum() { final int next = this.nextPath; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java index 49d7958efc430..3ecdb945841a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java @@ -22,19 +22,34 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.event.task.RuntimeEvent; -import java.io.IOException; public class EndOfPartitionEvent extends RuntimeEvent { public static final EndOfPartitionEvent INSTANCE = new EndOfPartitionEvent(); - + + @Override - public void read(DataInputView in) throws IOException { + public void read(DataInputView in) { // Nothing to do here } @Override - public void write(DataOutputView out) throws IOException { + public void write(DataOutputView out) { // Nothing to do here } + + @Override + public int hashCode() { + return 1965146673; + } + + @Override + public boolean equals(Object obj) { + return obj != null && obj.getClass() == EndOfPartitionEvent.class; + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 78aa6f75fd84a..0aebcae64ffd9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -408,7 +408,7 @@ public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedExcep // Sanity check that notifications only happen when data is available if (buffer == null) { - throw new IllegalStateException("Bug in input gate/channel logic: input gate got" + + throw new IllegalStateException("Bug in input gate/channel logic: input gate got " + "notified by channel about available data, but none was available."); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index 40e84fcdf69dd..466b8f75f9287 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -19,263 +19,262 @@ import java.io.File; import java.io.IOException; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.Queue; -import java.util.Set; +import java.util.ArrayDeque; -import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.reader.AbstractReader; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * The barrier buffer is responsible for implementing the blocking behaviour described - * here: {@link CheckpointBarrier}. - * - *

- * To avoid back-pressuring the - * readers, we buffer up the new data received from the blocked channels until - * the blocks are released. + * The barrier buffer is {@link CheckpointBarrierHandler} that blocks inputs with barriers until + * all inputs have received the barrier for a given checkpoint. + * + *

To avoid back-pressuring the input streams (which may cause distributed deadlocks), the + * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until + * the blocks are released.

*/ -public class BarrierBuffer { +public class BarrierBuffer implements CheckpointBarrierHandler { private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class); + + /** The gate that the buffer draws its input from */ + private final InputGate inputGate; + + /** Flags that indicate whether a channel is currently blocked/buffered */ + private final boolean[] blockedChannels; + + /** The total number of channels that this buffer handles data from */ + private final int totalNumberOfInputChannels; + + private final SpillReader spillReader; + private final BufferSpiller bufferSpiller; + + private ArrayDeque nonProcessed; + private ArrayDeque blockedNonProcessed; + + /** Handler that receives the checkpoint notifications */ + private EventListener checkpointHandler; + + /** The ID of the checkpoint for which we expect barriers */ + private long currentCheckpointId = -1L; + + /** The number of received barriers (= number of blocked/buffered channels) */ + private long numReceivedBarriers; + + /** Flag to indicate whether we have drawn all available input */ + private boolean endOfStream; + + + public BarrierBuffer(InputGate inputGate, IOManager ioManager) throws IOException { + this.inputGate = inputGate; + this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); + this.blockedChannels = new boolean[this.totalNumberOfInputChannels]; + + this.nonProcessed = new ArrayDeque(); + this.blockedNonProcessed = new ArrayDeque(); + + this.bufferSpiller = new BufferSpiller(ioManager); + this.spillReader = new SpillReader(); + } - private Queue nonProcessed = new LinkedList(); - private Queue blockedNonProcessed = new LinkedList(); - - private Set blockedChannels = new HashSet(); - private int totalNumberOfInputChannels; - - private CheckpointBarrier currentBarrier; - - private AbstractReader reader; - - private InputGate inputGate; - - private SpillReader spillReader; - private BufferSpiller bufferSpiller; - - private boolean inputFinished = false; + // ------------------------------------------------------------------------ + // Buffer and barrier handling + // ------------------------------------------------------------------------ - private BufferOrEvent endOfStreamEvent = null; + @Override + public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException { + while (true) { + // process buffered BufferOrEvents before grabbing new ones + final SpillingBufferOrEvent nextBuffered = nonProcessed.pollFirst(); + final BufferOrEvent next = nextBuffered == null ? + inputGate.getNextBufferOrEvent() : + nextBuffered.getBufferOrEvent(); + + if (next != null) { + if (isBlocked(next.getChannelIndex())) { + // if the channel is blocked we, we just store the BufferOrEvent + blockedNonProcessed.add(new SpillingBufferOrEvent(next, bufferSpiller, spillReader)); + } + else if (next.isBuffer() || next.getEvent().getClass() != CheckpointBarrier.class) { + return next; + } + else if (!endOfStream) { + // process barriers only if there is a chance of the checkpoint completing + processBarrier((CheckpointBarrier) next.getEvent(), next.getChannelIndex()); + } + } + else if (!endOfStream) { + // end of stream. we feed the data that is still buffered + endOfStream = true; + releaseBlocks(); + return getNextNonBlocked(); + } + else { + return null; + } + } + } + + private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws IOException { + final long barrierId = receivedBarrier.getId(); + + if (numReceivedBarriers > 0) { + // subsequent barrier of a checkpoint. + if (barrierId == currentCheckpointId) { + // regular case + onBarrier(channelIndex); + } + else if (barrierId > currentCheckpointId) { + // we did not complete the current checkpoint + LOG.warn("Received checkpoint barrier for checkpoint {} before completing current checkpoint {}. " + + "Skipping current checkpoint.", barrierId, currentCheckpointId); + + releaseBlocks(); + currentCheckpointId = barrierId; + onBarrier(channelIndex); + } + else { + // ignore trailing barrier from aborted checkpoint + return; + } + + } + else if (barrierId > currentCheckpointId) { + // first barrier of a new checkpoint + currentCheckpointId = barrierId; + onBarrier(channelIndex); + } + else { + // trailing barrier from previous (skipped) checkpoint + return; + } - private long lastCheckpointId = Long.MIN_VALUE; + // check if we have all barriers + if (numReceivedBarriers == totalNumberOfInputChannels) { + if (LOG.isDebugEnabled()) { + LOG.debug("Received all barrier, triggering checkpoint {} at {}", + receivedBarrier.getId(), receivedBarrier.getTimestamp()); + } - public BarrierBuffer(InputGate inputGate, AbstractReader reader) { - this.inputGate = inputGate; - totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); - this.reader = reader; - try { - this.bufferSpiller = new BufferSpiller(); - this.spillReader = new SpillReader(); - } catch (IOException e) { - throw new RuntimeException(e); + if (checkpointHandler != null) { + checkpointHandler.onEvent(receivedBarrier); + } + + releaseBlocks(); } - + } + + @Override + public void registerCheckpointEventHandler(EventListener checkpointHandler) { + if (this.checkpointHandler == null) { + this.checkpointHandler = checkpointHandler; + } + else { + throw new IllegalStateException("BarrierBuffer already has a registered checkpoint handler"); + } + } + + @Override + public boolean isEmpty() { + return nonProcessed.isEmpty() && blockedNonProcessed.isEmpty(); } - /** - * Get then next non-blocked non-processed {@link BufferOrEvent}. Returns null if - * none available. - * - * @throws IOException - */ - private BufferOrEvent getNonProcessed() throws IOException { - SpillingBufferOrEvent nextNonProcessed; - - while ((nextNonProcessed = nonProcessed.poll()) != null) { - BufferOrEvent boe = nextNonProcessed.getBufferOrEvent(); - if (isBlocked(boe.getChannelIndex())) { - blockedNonProcessed.add(new SpillingBufferOrEvent(boe, bufferSpiller, spillReader)); - } else { - return boe; + @Override + public void cleanup() throws IOException { + bufferSpiller.close(); + File spillfile1 = bufferSpiller.getSpillFile(); + if (spillfile1 != null) { + if (!spillfile1.delete()) { + LOG.warn("Cannot remove barrier buffer spill file: " + spillfile1.getAbsolutePath()); } } - return null; + spillReader.close(); + File spillfile2 = spillReader.getSpillFile(); + if (spillfile2 != null) { + if (!spillfile2.delete()) { + LOG.warn("Cannot remove barrier buffer spill file: " + spillfile2.getAbsolutePath()); + } + } } - + /** * Checks whether the channel with the given index is blocked. * - * @param channelIndex The channel index to check + * @param channelIndex The channel index to check. + * @return True if the channel is blocked, false if not. */ private boolean isBlocked(int channelIndex) { - return blockedChannels.contains(channelIndex); + return blockedChannels[channelIndex]; } - - /** - * Checks whether all channels are blocked meaning that barriers have been - * received from all channels - */ - private boolean isAllBlocked() { - return blockedChannels.size() == totalNumberOfInputChannels; - } - - /** - * Returns the next non-blocked {@link BufferOrEvent}. This is a blocking operator. - */ - public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException { - // If there are non-processed buffers from the previously blocked ones, - // we get the next - BufferOrEvent bufferOrEvent = getNonProcessed(); - - if (bufferOrEvent != null) { - return bufferOrEvent; - } else if (blockedNonProcessed.isEmpty() && inputFinished) { - return endOfStreamEvent; - } else { - // If no non-processed, get new from input - while (true) { - if (!inputFinished) { - // We read the next buffer from the inputgate - bufferOrEvent = inputGate.getNextBufferOrEvent(); - - if (!bufferOrEvent.isBuffer() - && bufferOrEvent.getEvent() instanceof EndOfPartitionEvent) { - if (inputGate.isFinished()) { - // store the event for later if the channel is - // closed - endOfStreamEvent = bufferOrEvent; - inputFinished = true; - } - - } else { - if (isBlocked(bufferOrEvent.getChannelIndex())) { - // If channel blocked we just store it - blockedNonProcessed.add(new SpillingBufferOrEvent(bufferOrEvent, - bufferSpiller, spillReader)); - } else { - return bufferOrEvent; - } - } - } else { - actOnAllBlocked(); - return getNextNonBlocked(); - } - } - } - } - + /** * Blocks the given channel index, from which a barrier has been received. * - * @param channelIndex - * The channel index to block. + * @param channelIndex The channel index to block. */ - private void blockChannel(int channelIndex) { - if (!blockedChannels.contains(channelIndex)) { - blockedChannels.add(channelIndex); + private void onBarrier(int channelIndex) throws IOException { + if (!blockedChannels[channelIndex]) { + blockedChannels[channelIndex] = true; + numReceivedBarriers++; + if (LOG.isDebugEnabled()) { - LOG.debug("Channel blocked with index: " + channelIndex); - } - if (isAllBlocked()) { - actOnAllBlocked(); + LOG.debug("Received barrier from channel " + channelIndex); } - - } else { - throw new RuntimeException("Tried to block an already blocked channel"); + } + else { + throw new IOException("Stream corrupt: Repeated barrier for same checkpoint and input stream"); } } /** * Releases the blocks on all channels. */ - private void releaseBlocks() { - if (!nonProcessed.isEmpty()) { - // sanity check - throw new RuntimeException("Error in barrier buffer logic"); - } - nonProcessed = blockedNonProcessed; - blockedNonProcessed = new LinkedList(); - - try { - spillReader.setSpillFile(bufferSpiller.getSpillFile()); - bufferSpiller.resetSpillFile(); - } catch (IOException e) { - throw new RuntimeException(e); - } - - blockedChannels.clear(); - currentBarrier = null; + private void releaseBlocks() throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug("All barriers received, blocks released"); + LOG.debug("Releasing blocks"); } - } - /** - * Method that is executed once the barrier has been received from all - * channels. - */ - private void actOnAllBlocked() { - if (LOG.isDebugEnabled()) { - LOG.debug("Publishing barrier to the vertex"); + for (int i = 0; i < blockedChannels.length; i++) { + blockedChannels[i] = false; } - - if (currentBarrier != null && !inputFinished) { - reader.publish(currentBarrier); - lastCheckpointId = currentBarrier.getId(); + numReceivedBarriers = 0; + + if (nonProcessed.isEmpty()) { + // swap the queues + ArrayDeque empty = nonProcessed; + nonProcessed = blockedNonProcessed; + blockedNonProcessed = empty; } - - releaseBlocks(); - } - - /** - * Processes one {@link org.apache.flink.streaming.runtime.tasks.CheckpointBarrier} - * - * @param bufferOrEvent The {@link BufferOrEvent} containing the checkpoint barrier - */ - public void processBarrier(BufferOrEvent bufferOrEvent) { - CheckpointBarrier receivedBarrier = (CheckpointBarrier) bufferOrEvent.getEvent(); - - if (receivedBarrier.getId() < lastCheckpointId) { - // a barrier from an old checkpoint, ignore these - return; - } - - if (currentBarrier == null) { - this.currentBarrier = receivedBarrier; - if (LOG.isDebugEnabled()) { - LOG.debug("Checkpoint barrier received start waiting for checkpoint: {}", receivedBarrier); - } - } else if (receivedBarrier.getId() > currentBarrier.getId()) { - // we have a barrier from a more recent checkpoint, free all locks and start with - // this newer checkpoint - if (LOG.isDebugEnabled()) { - LOG.debug("Checkpoint barrier received while waiting on checkpoint {}. Restarting waiting with checkpoint {}: ", currentBarrier, receivedBarrier); - } - releaseBlocks(); - currentBarrier = receivedBarrier; - + else { + throw new IllegalStateException("Unconsumed data from previous checkpoint alignment " + + "when starting next checkpoint alignment"); } - blockChannel(bufferOrEvent.getChannelIndex()); + + // roll over the spill files + spillReader.setSpillFile(bufferSpiller.getSpillFile()); + bufferSpiller.resetSpillFile(); } - public void cleanup() throws IOException { - bufferSpiller.close(); - File spillfile1 = bufferSpiller.getSpillFile(); - if (spillfile1 != null) { - spillfile1.delete(); - } + // ------------------------------------------------------------------------ + // For Testing + // ------------------------------------------------------------------------ - spillReader.close(); - File spillfile2 = spillReader.getSpillFile(); - if (spillfile2 != null) { - spillfile2.delete(); - } + public long getCurrentCheckpointId() { + return this.currentCheckpointId; } - + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + @Override public String toString() { - return nonProcessed.toString() + blockedNonProcessed.toString(); - } - - public boolean isEmpty() { - return nonProcessed.isEmpty() && blockedNonProcessed.isEmpty(); + return "Non-Processed: " + nonProcessed + " | Blocked: " + blockedNonProcessed; } - } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java index 0d57d056e6f4b..fda612e7247d0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java @@ -22,28 +22,33 @@ import java.io.RandomAccessFile; import java.nio.channels.FileChannel; import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.util.StringUtils; public class BufferSpiller { + + /** The random number generator for temp file names */ + private static final Random RND = new Random(); - protected static Random rnd = new Random(); + /** The counter that selects the next directory to spill into */ + private static final AtomicInteger DIRECTORY_INDEX = new AtomicInteger(0); + + + /** The directories to spill to */ + private final File tempDir; private File spillFile; - protected FileChannel spillingChannel; - private String tempDir; - - public BufferSpiller() throws IOException { - String tempDirString = GlobalConfiguration.getString( - ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH); - String[] tempDirs = tempDirString.split(",|" + File.pathSeparator); - - tempDir = tempDirs[rnd.nextInt(tempDirs.length)]; - + + private FileChannel spillingChannel; + + + + public BufferSpiller(IOManager ioManager) throws IOException { + File[] tempDirs = ioManager.getSpillingDirectories(); + this.tempDir = tempDirs[DIRECTORY_INDEX.getAndIncrement() % tempDirs.length]; createSpillingChannel(); } @@ -54,24 +59,20 @@ public void spill(Buffer buffer) throws IOException { try { spillingChannel.write(buffer.getNioBuffer()); buffer.recycle(); - } catch (IOException e) { + } + catch (IOException e) { close(); - throw new IOException(e); + throw e; } - } @SuppressWarnings("resource") private void createSpillingChannel() throws IOException { - this.spillFile = new File(tempDir, randomString(rnd) + ".buffer"); + this.spillFile = new File(tempDir, randomString(RND) + ".buffer"); this.spillingChannel = new RandomAccessFile(spillFile, "rw").getChannel(); } - private static String randomString(Random random) { - final byte[] bytes = new byte[20]; - random.nextBytes(bytes); - return StringUtils.byteToHexString(bytes); - } + public void close() throws IOException { if (spillingChannel != null && spillingChannel.isOpen()) { @@ -87,5 +88,12 @@ public void resetSpillFile() throws IOException { public File getSpillFile() { return spillFile; } + + // ------------------------------------------------------------------------ + private static String randomString(Random random) { + final byte[] bytes = new byte[20]; + random.nextBytes(bytes); + return StringUtils.byteToHexString(bytes); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java new file mode 100644 index 0000000000000..02dd33d5da066 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java @@ -0,0 +1,55 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; + +import java.io.IOException; + +/** + * The CheckpointBarrierHandler reacts to checkpoint barrier arriving from the input channels. + * Different implementations may either simply track barriers, or block certain inputs on + * barriers. + */ +public interface CheckpointBarrierHandler { + + /** + * Returns the next {@link BufferOrEvent} that the operator may consume. + * This call blocks until the next BufferOrEvent is available, ir until the stream + * has been determined to be finished. + * + * @return The next BufferOrEvent, or {@code null}, if the stream is finished. + * @throws java.io.IOException Thrown, if the network or local disk I/O fails. + * @throws java.lang.InterruptedException Thrown, if the thread is interrupted while blocking during + * waiting for the next BufferOrEvent to become available. + */ + BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException; + + void registerCheckpointEventHandler(EventListener checkpointHandler); + + void cleanup() throws IOException; + + /** + * Checks if the barrier handler has buffered any data internally. + * @return True, if no data is buffered internally, false otherwise. + */ + boolean isEmpty(); +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 9db017807d0a0..4d60375173ac3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.reader.AbstractReader; import org.apache.flink.runtime.io.network.api.reader.ReaderBase; import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; @@ -33,6 +34,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; @@ -40,6 +42,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +69,7 @@ public class StreamInputProcessor extends AbstractReader implements ReaderBa private boolean isFinished; - private final BarrierBuffer barrierBuffer; + private final CheckpointBarrierHandler barrierHandler; private final long[] watermarks; private long lastEmittedWatermark; @@ -74,10 +77,17 @@ public class StreamInputProcessor extends AbstractReader implements ReaderBa private final DeserializationDelegate deserializationDelegate; @SuppressWarnings("unchecked") - public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSerializer, boolean enableWatermarkMultiplexing) { + public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSerializer, + EventListener checkpointListener, + IOManager ioManager, + boolean enableWatermarkMultiplexing) throws IOException { + super(InputGateUtil.createInputGate(inputGates)); - barrierBuffer = new BarrierBuffer(inputGate, this); + this.barrierHandler = new BarrierBuffer(inputGate, ioManager); + if (checkpointListener != null) { + this.barrierHandler.registerCheckpointEventHandler(checkpointListener); + } if (enableWatermarkMultiplexing) { MultiplexingStreamRecordSerializer ser = new MultiplexingStreamRecordSerializer(inputSerializer); @@ -101,8 +111,8 @@ public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSeri } lastEmittedWatermark = Long.MIN_VALUE; } - - @SuppressWarnings("unchecked") + + public boolean processInput(OneInputStreamOperator streamOperator) throws Exception { if (isFinished) { return false; @@ -137,8 +147,10 @@ public boolean processInput(OneInputStreamOperator streamOperator) throws } } continue; - } else { + } + else { // now we can do the actual processing + @SuppressWarnings("unchecked") StreamRecord record = (StreamRecord) deserializationDelegate.getInstance(); StreamingRuntimeContext ctx = streamOperator.getRuntimeContext(); if (ctx != null) { @@ -150,32 +162,26 @@ public boolean processInput(OneInputStreamOperator streamOperator) throws } } - final BufferOrEvent bufferOrEvent = barrierBuffer.getNextNonBlocked(); - - if (bufferOrEvent.isBuffer()) { - currentChannel = bufferOrEvent.getChannelIndex(); - currentRecordDeserializer = recordDeserializers[currentChannel]; - currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); - } else { - // Event received - final AbstractEvent event = bufferOrEvent.getEvent(); - - if (event instanceof CheckpointBarrier) { - barrierBuffer.processBarrier(bufferOrEvent); - } else { - if (handleEvent(event)) { - if (inputGate.isFinished()) { - if (!barrierBuffer.isEmpty()) { - throw new RuntimeException("BarrierBuffer should be empty at this point"); - } - isFinished = true; - return false; - } else if (hasReachedEndOfSuperstep()) { - return false; - } // else: More data is coming... - } + final BufferOrEvent bufferOrEvent = barrierHandler.getNextNonBlocked(); + if (bufferOrEvent != null) { + if (bufferOrEvent.isBuffer()) { + currentChannel = bufferOrEvent.getChannelIndex(); + currentRecordDeserializer = recordDeserializers[currentChannel]; + currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); + } + else { + // Event received + final AbstractEvent event = bufferOrEvent.getEvent(); + handleEvent(event); } } + else { + isFinished = true; + if (!barrierHandler.isEmpty()) { + throw new IllegalStateException("Trailing data in checkpoint barrier handler."); + } + return false; + } } } @@ -195,7 +201,8 @@ public void clearBuffers() { } } + @Override public void cleanup() throws IOException { - barrierBuffer.cleanup(); + barrierHandler.cleanup(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index e235ffeae3bb0..9668c7f310dca 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.reader.AbstractReader; import org.apache.flink.runtime.io.network.api.reader.ReaderBase; import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; @@ -31,6 +32,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; @@ -68,7 +70,7 @@ public class StreamTwoInputProcessor extends AbstractReader implements private boolean isFinished; - private final BarrierBuffer barrierBuffer; + private final CheckpointBarrierHandler barrierHandler; private final long[] watermarks1; private long lastEmittedWatermark1; @@ -87,11 +89,17 @@ public StreamTwoInputProcessor( Collection inputGates2, TypeSerializer inputSerializer1, TypeSerializer inputSerializer2, - boolean enableWatermarkMultiplexing) { + EventListener checkpointListener, + IOManager ioManager, + boolean enableWatermarkMultiplexing) throws IOException { super(InputGateUtil.createInputGate(inputGates1, inputGates2)); - barrierBuffer = new BarrierBuffer(inputGate, this); + this.barrierHandler = new BarrierBuffer(inputGate, ioManager); + if (checkpointListener != null) { + this.barrierHandler.registerCheckpointEventHandler(checkpointListener); + } + if (enableWatermarkMultiplexing) { MultiplexingStreamRecordSerializer ser = new MultiplexingStreamRecordSerializer(inputSerializer1); @@ -186,32 +194,26 @@ public boolean processInput(TwoInputStreamOperator streamOperator) } } - final BufferOrEvent bufferOrEvent = barrierBuffer.getNextNonBlocked(); - - if (bufferOrEvent.isBuffer()) { - currentChannel = bufferOrEvent.getChannelIndex(); - currentRecordDeserializer = recordDeserializers[currentChannel]; - currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); - - } else { - // Event received - final AbstractEvent event = bufferOrEvent.getEvent(); + final BufferOrEvent bufferOrEvent = barrierHandler.getNextNonBlocked(); + if (bufferOrEvent != null) { - if (event instanceof CheckpointBarrier) { - barrierBuffer.processBarrier(bufferOrEvent); + if (bufferOrEvent.isBuffer()) { + currentChannel = bufferOrEvent.getChannelIndex(); + currentRecordDeserializer = recordDeserializers[currentChannel]; + currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); + } else { - if (handleEvent(event)) { - if (inputGate.isFinished()) { - if (!barrierBuffer.isEmpty()) { - throw new RuntimeException("BarrierBuffer should be empty at this point"); - } - isFinished = true; - return false; - } else if (hasReachedEndOfSuperstep()) { - return false; - } // else: More data is coming... - } + // Event received + final AbstractEvent event = bufferOrEvent.getEvent(); + handleEvent(event); + } + } + else { + isFinished = true; + if (!barrierHandler.isEmpty()) { + throw new IllegalStateException("Trailing data in checkpoint barrier handler."); } + return false; } } } @@ -270,6 +272,6 @@ public void clearBuffers() { @Override public void cleanup() throws IOException { - barrierBuffer.cleanup(); + barrierHandler.cleanup(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index 9d6e88e375e8f..d0783206feb03 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -34,22 +34,27 @@ public class OneInputStreamTask extends StreamTask inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); - - int numberOfInputs = configuration.getNumberOfInputs(); - - if (numberOfInputs > 0) { - InputGate[] inputGates = getEnvironment().getAllInputGates(); - inputProcessor = new StreamInputProcessor(inputGates, inSerializer, getExecutionConfig().areTimestampsEnabled()); - - inputProcessor.registerTaskEventListener(getCheckpointBarrierListener(), CheckpointBarrier.class); - - AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); - AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); - - inputProcessor.setReporter(reporter); + try { + super.registerInputOutput(); + + TypeSerializer inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); + int numberOfInputs = configuration.getNumberOfInputs(); + + if (numberOfInputs > 0) { + InputGate[] inputGates = getEnvironment().getAllInputGates(); + inputProcessor = new StreamInputProcessor(inputGates, inSerializer, + getCheckpointBarrierListener(), + getEnvironment().getIOManager(), + getExecutionConfig().areTimestampsEnabled()); + + // make sure that stream tasks report their I/O statistics + AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); + AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); + inputProcessor.setReporter(reporter); + } + } + catch (Exception e) { + throw new RuntimeException("Failed to initialize stream operator: " + e.getMessage(), e); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 75bdd57e848fc..d8298337d49bc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -31,7 +31,6 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.TaskEvent; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.CheckpointNotificationOperator; @@ -74,7 +73,7 @@ public abstract class StreamTask> extends Abs protected ClassLoader userClassLoader; - private EventListener checkpointBarrierListener; + private EventListener checkpointBarrierListener; public StreamTask() { streamOperator = null; @@ -106,7 +105,7 @@ public void registerInputOutput() { streamOperator.setup(outputHandler.getOutput(), headContext); } - hasChainedOperators = !(outputHandler.getChainedOperators().size() == 1); + hasChainedOperators = outputHandler.getChainedOperators().size() != 1; } public String getName() { @@ -199,7 +198,7 @@ public void cancel() { this.isRunning = false; } - public EventListener getCheckpointBarrierListener() { + public EventListener getCheckpointBarrierListener() { return this.checkpointBarrierListener; } @@ -211,7 +210,7 @@ public EventListener getCheckpointBarrierListener() { @Override public void setInitialState(StateHandle stateHandle) throws Exception { - // We retrieve end restore the states for the chained oeprators. + // We retrieve end restore the states for the chained operators. List, Map>> chainedStates = (List, Map>>) stateHandle.getState(); // We restore all stateful chained operators @@ -306,13 +305,12 @@ public String toString() { // ------------------------------------------------------------------------ - private class CheckpointBarrierListener implements EventListener { + private class CheckpointBarrierListener implements EventListener { @Override - public void onEvent(TaskEvent event) { + public void onEvent(CheckpointBarrier barrier) { try { - CheckpointBarrier sStep = (CheckpointBarrier) event; - triggerCheckpoint(sStep.getId(), sStep.getTimestamp()); + triggerCheckpoint(barrier.getId(), barrier.getTimestamp()); } catch (Exception e) { throw new RuntimeException("Error triggering a checkpoint as the result of receiving checkpoint barrier", e); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index f981cd578ea71..b4667b2b90307 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -34,44 +34,52 @@ public class TwoInputStreamTask extends StreamTask inputProcessor; + private StreamTwoInputProcessor inputProcessor; @Override public void registerInputOutput() { - super.registerInputOutput(); - - TypeSerializer inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); - TypeSerializer inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); - - int numberOfInputs = configuration.getNumberOfInputs(); - - ArrayList inputList1 = new ArrayList(); - ArrayList inputList2 = new ArrayList(); - - List inEdges = configuration.getInPhysicalEdges(userClassLoader); - - for (int i = 0; i < numberOfInputs; i++) { - int inputType = inEdges.get(i).getTypeNumber(); - InputGate reader = getEnvironment().getInputGate(i); - switch (inputType) { - case 1: - inputList1.add(reader); - break; - case 2: - inputList2.add(reader); - break; - default: - throw new RuntimeException("Invalid input type number: " + inputType); + try { + super.registerInputOutput(); + + TypeSerializer inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); + TypeSerializer inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); + + int numberOfInputs = configuration.getNumberOfInputs(); + + ArrayList inputList1 = new ArrayList(); + ArrayList inputList2 = new ArrayList(); + + List inEdges = configuration.getInPhysicalEdges(userClassLoader); + + for (int i = 0; i < numberOfInputs; i++) { + int inputType = inEdges.get(i).getTypeNumber(); + InputGate reader = getEnvironment().getInputGate(i); + switch (inputType) { + case 1: + inputList1.add(reader); + break; + case 2: + inputList2.add(reader); + break; + default: + throw new RuntimeException("Invalid input type number: " + inputType); + } } + + this.inputProcessor = new StreamTwoInputProcessor(inputList1, inputList2, + inputDeserializer1, inputDeserializer2, + getCheckpointBarrierListener(), + getEnvironment().getIOManager(), + getExecutionConfig().areTimestampsEnabled()); + + // make sure that stream tasks report their I/O statistics + AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); + AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); + this.inputProcessor.setReporter(reporter); + } + catch (Exception e) { + throw new RuntimeException("Failed to initialize stream operator: " + e.getMessage(), e); } - - inputProcessor = new StreamTwoInputProcessor(inputList1, inputList2, inputDeserializer1, inputDeserializer2, getExecutionConfig().areTimestampsEnabled()); - - AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); - AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); - inputProcessor.setReporter(reporter); - - inputProcessor.registerTaskEventListener(getCheckpointBarrierListener(), CheckpointBarrier.class); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index c479f957e23af..b59ad191ec5e8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -20,11 +20,8 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.event.task.AbstractEvent; -import org.apache.flink.runtime.event.task.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; @@ -220,14 +217,4 @@ public boolean isEvent() { return isEvent; } } - - public static class DummyEvent extends TaskEvent { - @Override - public void write(DataOutputView out) throws IOException { - } - - @Override - public void read(DataInputView in) throws IOException { - } - } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java similarity index 57% rename from flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java rename to flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java index d8a36967fbcd0..c2df4d83e2bc4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java @@ -23,53 +23,71 @@ import java.util.Random; import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; + import org.junit.Test; -public class BarrierBufferIOTest { +/** + * The test generates two random streams (input channels) which independently + * and randomly generate checkpoint barriers. The two streams are very + * unaligned, putting heavy work on the BarrierBuffer. + */ +public class BarrierBufferMassiveRandomTest { @Test - public void IOTest() throws IOException, InterruptedException { - - BufferPool pool1 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); - BufferPool pool2 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); - - MockInputGate myIG = new MockInputGate(new BufferPool[] { pool1, pool2 }, - new BarrierGenerator[] { new CountBarrier(100000), new RandomBarrier(100000) }); - // new BarrierSimulator[] { new CountBarrier(1000), new - // CountBarrier(1000) }); - - BarrierBuffer barrierBuffer = new BarrierBuffer(myIG, - new BarrierBufferTest.MockReader(myIG)); - + public void testWithTwoChannelsAndRandomBarriers() { + IOManager ioMan = null; try { - // long time = System.currentTimeMillis(); + ioMan = new IOManagerAsync(); + + BufferPool pool1 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); + BufferPool pool2 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); + + RandomGeneratingInputGate myIG = new RandomGeneratingInputGate( + new BufferPool[] { pool1, pool2 }, + new BarrierGenerator[] { new CountBarrier(100000), new RandomBarrier(100000) }); + + BarrierBuffer barrierBuffer = new BarrierBuffer(myIG, ioMan); + for (int i = 0; i < 2000000; i++) { BufferOrEvent boe = barrierBuffer.getNextNonBlocked(); if (boe.isBuffer()) { boe.getBuffer().recycle(); - } else { - barrierBuffer.processBarrier(boe); } } - // System.out.println("Ran for " + (System.currentTimeMillis() - - // time)); - } catch (Exception e) { - fail(); - } finally { - barrierBuffer.cleanup(); } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + finally { + if (ioMan != null) { + ioMan.shutdown(); + } + } + } + + // ------------------------------------------------------------------------ + // Mocks and Generators + // ------------------------------------------------------------------------ + + protected interface BarrierGenerator { + public boolean isNextBarrier(); } - private static class RandomBarrier implements BarrierGenerator { - private static Random rnd = new Random(); + protected static class RandomBarrier implements BarrierGenerator { + + private static final Random rnd = new Random(); - double threshold; + private final double threshold; public RandomBarrier(double expectedEvery) { threshold = 1 / expectedEvery; @@ -83,8 +101,8 @@ public boolean isNextBarrier() { private static class CountBarrier implements BarrierGenerator { - long every; - long c = 0; + private final long every; + private long c = 0; public CountBarrier(long every) { this.every = every; @@ -96,16 +114,16 @@ public boolean isNextBarrier() { } } - protected static class MockInputGate implements InputGate { + protected static class RandomGeneratingInputGate implements InputGate { - private int numChannels; - private BufferPool[] bufferPools; - private int[] currentBarriers; - BarrierGenerator[] barrierGens; - int currentChannel = 0; - long c = 0; + private final int numChannels; + private final BufferPool[] bufferPools; + private final int[] currentBarriers; + private final BarrierGenerator[] barrierGens; + private int currentChannel = 0; + private long c = 0; - public MockInputGate(BufferPool[] bufferPools, BarrierGenerator[] barrierGens) { + public RandomGeneratingInputGate(BufferPool[] bufferPools, BarrierGenerator[] barrierGens) { this.numChannels = bufferPools.length; this.currentBarriers = new int[numChannels]; this.bufferPools = bufferPools; @@ -123,37 +141,27 @@ public boolean isFinished() { } @Override - public void requestPartitions() throws IOException, InterruptedException { - } + public void requestPartitions() {} @Override public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException { currentChannel = (currentChannel + 1) % numChannels; if (barrierGens[currentChannel].isNextBarrier()) { - return BarrierBufferTest.createBarrier(++currentBarriers[currentChannel], - currentChannel); + return new BufferOrEvent( + new CheckpointBarrier(++currentBarriers[currentChannel], System.currentTimeMillis()), + currentChannel); } else { Buffer buffer = bufferPools[currentChannel].requestBuffer(); buffer.getMemorySegment().putLong(0, c++); - return new BufferOrEvent(buffer, currentChannel); } - } @Override - public void sendTaskEvent(TaskEvent event) throws IOException { - } + public void sendTaskEvent(TaskEvent event) {} @Override - public void registerListener(EventListener listener) { - } - - } - - protected interface BarrierGenerator { - public boolean isNextBarrier(); + public void registerListener(EventListener listener) {} } - } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index cb5e046e49f90..ad61c6fb49b43 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, @@ -17,152 +18,652 @@ package org.apache.flink.streaming.runtime.io; -import static org.junit.Assert.assertEquals; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; - -import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.reader.AbstractReader; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; -public class BarrierBufferTest { - - @Test - public void testWithoutBarriers() throws IOException, InterruptedException { +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.List; +import java.util.Queue; - List input = new LinkedList(); - input.add(createBuffer(0)); - input.add(createBuffer(0)); - input.add(createBuffer(0)); +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; - InputGate mockIG = new MockInputGate(1, input); - AbstractReader mockAR = new MockReader(mockIG); +/** + * Tests for the behavior of the {@link BarrierBuffer}. + */ +public class BarrierBufferTest { - BarrierBuffer bb = new BarrierBuffer(mockIG, mockAR); + private static int SIZE_COUNTER = 0; + + private static IOManager IO_MANAGER; - assertEquals(input.get(0), bb.getNextNonBlocked()); - assertEquals(input.get(1), bb.getNextNonBlocked()); - assertEquals(input.get(2), bb.getNextNonBlocked()); + @BeforeClass + public static void setup() { + IO_MANAGER = new IOManagerAsync(); + SIZE_COUNTER = 1; + } - bb.cleanup(); + @AfterClass + public static void shutdownIOManager() { + IO_MANAGER.shutdown(); } + // ------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------ + + /** + * Validates that the buffer behaves correctly if no checkpoint barriers come, + * for a single input channel. + */ @Test - public void testOneChannelBarrier() throws IOException, InterruptedException { + public void testSingleChannelNoBarriers() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(0), createBuffer(0), + createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + for (BufferOrEvent boe : sequence) { + assertEquals(boe, buffer.getNextNonBlocked()); + } + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - List input = new LinkedList(); - input.add(createBuffer(0)); - input.add(createBuffer(0)); - input.add(createBarrier(1, 0)); - input.add(createBuffer(0)); - input.add(createBuffer(0)); - input.add(createBarrier(2, 0)); - input.add(createBuffer(0)); + /** + * Validates that the buffer behaves correctly if no checkpoint barriers come, + * for an input with multiple input channels. + */ + @Test + public void testMultiChannelNoBarriers() { + try { + BufferOrEvent[] sequence = { createBuffer(2), createBuffer(2), createBuffer(0), + createBuffer(1), createBuffer(0), createEndOfPartition(0), + createBuffer(3), createBuffer(1), createEndOfPartition(3), + createBuffer(1), createEndOfPartition(1), createBuffer(2), createEndOfPartition(2) + }; + + MockInputGate gate = new MockInputGate(4, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + for (BufferOrEvent boe : sequence) { + assertEquals(boe, buffer.getNextNonBlocked()); + } + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - InputGate mockIG = new MockInputGate(1, input); - AbstractReader mockAR = new MockReader(mockIG); + /** + * Validates that the buffer preserved the order of elements for a + * input with a single input channel, and checkpoint events. + */ + @Test + public void testSingleChannelWithBarriers() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(0), createBuffer(0), + createBarrier(1, 0), + createBuffer(0), createBuffer(0), createBuffer(0), createBuffer(0), + createBarrier(2, 0), createBarrier(3, 0), + createBuffer(0), createBuffer(0), + createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0), + createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + for (BufferOrEvent boe : sequence) { + if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) { + assertEquals(boe, buffer.getNextNonBlocked()); + } + } + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - BarrierBuffer bb = new BarrierBuffer(mockIG, mockAR); - BufferOrEvent nextBoe; + /** + * Validates that the buffer correctly aligns the streams for inputs with + * multiple input channels, by buffering and blocking certain inputs. + */ + @Test + public void testMultiChannelWithBarriers() { + try { + BufferOrEvent[] sequence = { + // checkpoint with blocked data + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(1, 0), + + // checkpoint without blocked data + createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2), + createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2), + + // checkpoint with data only from one channel + createBuffer(2), createBuffer(2), + createBarrier(3, 2), + createBuffer(2), createBuffer(2), + createBarrier(3, 0), createBarrier(3, 1), + + // empty checkpoint + createBarrier(4, 1), createBarrier(4, 2), createBarrier(4, 0), + + // checkpoint with blocked data in mixed order + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(5, 1), + createBuffer(2), createBuffer(0), createBuffer(2), createBuffer(1), + createBarrier(5, 2), + createBuffer(1), createBuffer(0), createBuffer(2), createBuffer(1), + createBarrier(5, 0), + + // some trailing data + createBuffer(0), + createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + // pre checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + assertEquals(1L, handler.getNextExpectedCheckpointId()); + + // blocking while aligning for checkpoint 1 + check(sequence[7], buffer.getNextNonBlocked()); + assertEquals(1L, handler.getNextExpectedCheckpointId()); + + // checkpoint 1 done, returning buffered data + check(sequence[5], buffer.getNextNonBlocked()); + assertEquals(2L, handler.getNextExpectedCheckpointId()); + check(sequence[6], buffer.getNextNonBlocked()); + + // pre checkpoint 2 + check(sequence[9], buffer.getNextNonBlocked()); + check(sequence[10], buffer.getNextNonBlocked()); + check(sequence[11], buffer.getNextNonBlocked()); + check(sequence[12], buffer.getNextNonBlocked()); + check(sequence[13], buffer.getNextNonBlocked()); + assertEquals(2L, handler.getNextExpectedCheckpointId()); + + // checkpoint 2 barriers come together + check(sequence[17], buffer.getNextNonBlocked()); + assertEquals(3L, handler.getNextExpectedCheckpointId()); + check(sequence[18], buffer.getNextNonBlocked()); + + // checkpoint 3 starts, data buffered + check(sequence[20], buffer.getNextNonBlocked()); + assertEquals(4L, handler.getNextExpectedCheckpointId()); + check(sequence[21], buffer.getNextNonBlocked()); + + // checkpoint 4 happens without extra data + + // pre checkpoint 5 + check(sequence[27], buffer.getNextNonBlocked()); + assertEquals(5L, handler.getNextExpectedCheckpointId()); + check(sequence[28], buffer.getNextNonBlocked()); + check(sequence[29], buffer.getNextNonBlocked()); + + // checkpoint 5 aligning + check(sequence[31], buffer.getNextNonBlocked()); + check(sequence[32], buffer.getNextNonBlocked()); + check(sequence[33], buffer.getNextNonBlocked()); + check(sequence[37], buffer.getNextNonBlocked()); + + // buffered data from checkpoint 5 alignment + check(sequence[34], buffer.getNextNonBlocked()); + check(sequence[36], buffer.getNextNonBlocked()); + check(sequence[38], buffer.getNextNonBlocked()); + check(sequence[39], buffer.getNextNonBlocked()); + + // remaining data + check(sequence[41], buffer.getNextNonBlocked()); + check(sequence[42], buffer.getNextNonBlocked()); + check(sequence[43], buffer.getNextNonBlocked()); + check(sequence[44], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - assertEquals(input.get(0), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(1), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(2), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - assertEquals(input.get(3), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(4), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(5), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - assertEquals(input.get(6), nextBoe = bb.getNextNonBlocked()); + @Test + public void testMultiChannelTrailingBlockedData() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(1), createBuffer(2), + createBarrier(1, 1), createBarrier(1, 2), createBarrier(1, 0), + + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(2, 1), + createBuffer(1), createBuffer(1), createEndOfPartition(1), createBuffer(0), createBuffer(2), + createBarrier(2, 2), + createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + // pre-checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + assertEquals(1L, handler.getNextExpectedCheckpointId()); + + // pre-checkpoint 2 + check(sequence[6], buffer.getNextNonBlocked()); + assertEquals(2L, handler.getNextExpectedCheckpointId()); + check(sequence[7], buffer.getNextNonBlocked()); + check(sequence[8], buffer.getNextNonBlocked()); + + // checkpoint 2 alignment + check(sequence[13], buffer.getNextNonBlocked()); + check(sequence[14], buffer.getNextNonBlocked()); + check(sequence[18], buffer.getNextNonBlocked()); + check(sequence[19], buffer.getNextNonBlocked()); + + // end of stream: remaining buffered contents + check(sequence[10], buffer.getNextNonBlocked()); + check(sequence[11], buffer.getNextNonBlocked()); + check(sequence[12], buffer.getNextNonBlocked()); + check(sequence[16], buffer.getNextNonBlocked()); + check(sequence[17], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - bb.cleanup(); + /** + * Validates that the buffer correctly aligns the streams in cases + * where some channels receive barriers from multiple successive checkpoints + * before the pending checkpoint is complete. + */ + @Test + public void testMultiChannelWithQueuedFutureBarriers() { + try { + BufferOrEvent[] sequence = { + // checkpoint 1 - with blocked data + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(1, 0), + createBuffer(1), createBuffer(0), + + // checkpoint 2 - where future checkpoint barriers come before + // the current checkpoint is complete + createBarrier(2, 1), + createBuffer(1), createBuffer(2), createBarrier(2, 0), + createBarrier(3, 0), createBuffer(0), + createBarrier(3, 1), createBuffer(0), createBuffer(1), createBuffer(2), + createBarrier(4, 1), createBuffer(1), createBuffer(2), + + // complete checkpoint 2, send a barrier for checkpoints 4 and 5 + createBarrier(2, 2), + createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0), + createBarrier(4, 0), + createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0), + createBarrier(5, 1), + + // complete checkpoint 3 + createBarrier(3, 2), + createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0), + createBarrier(6, 1), + + // complete checkpoint 4, checkpoint 5 remains not fully triggered + createBarrier(4, 2), + createBuffer(2), + createBuffer(1), createEndOfPartition(1), + createBuffer(2), createEndOfPartition(2), + createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + // around checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + check(sequence[7], buffer.getNextNonBlocked()); + + check(sequence[5], buffer.getNextNonBlocked()); + assertEquals(2L, handler.getNextExpectedCheckpointId()); + check(sequence[6], buffer.getNextNonBlocked()); + check(sequence[9], buffer.getNextNonBlocked()); + check(sequence[10], buffer.getNextNonBlocked()); + + // alignment of checkpoint 2 - buffering also some barriers for + // checkpoints 3 and 4 + check(sequence[13], buffer.getNextNonBlocked()); + check(sequence[20], buffer.getNextNonBlocked()); + check(sequence[23], buffer.getNextNonBlocked()); + + // checkpoint 2 completed + check(sequence[12], buffer.getNextNonBlocked()); + check(sequence[25], buffer.getNextNonBlocked()); + check(sequence[27], buffer.getNextNonBlocked()); + check(sequence[30], buffer.getNextNonBlocked()); + check(sequence[32], buffer.getNextNonBlocked()); + + // checkpoint 3 completed (emit buffered) + check(sequence[16], buffer.getNextNonBlocked()); + check(sequence[18], buffer.getNextNonBlocked()); + check(sequence[19], buffer.getNextNonBlocked()); + check(sequence[28], buffer.getNextNonBlocked()); + + // past checkpoint 3 + check(sequence[36], buffer.getNextNonBlocked()); + check(sequence[38], buffer.getNextNonBlocked()); + + // checkpoint 4 completed (emit buffered) + check(sequence[22], buffer.getNextNonBlocked()); + check(sequence[26], buffer.getNextNonBlocked()); + check(sequence[31], buffer.getNextNonBlocked()); + check(sequence[33], buffer.getNextNonBlocked()); + check(sequence[39], buffer.getNextNonBlocked()); + + // past checkpoint 4, alignment for checkpoint 5 + check(sequence[42], buffer.getNextNonBlocked()); + check(sequence[45], buffer.getNextNonBlocked()); + check(sequence[46], buffer.getNextNonBlocked()); + check(sequence[47], buffer.getNextNonBlocked()); + check(sequence[48], buffer.getNextNonBlocked()); + + // end of input, emit remainder + check(sequence[37], buffer.getNextNonBlocked()); + check(sequence[43], buffer.getNextNonBlocked()); + check(sequence[44], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } + /** + * Validates that the buffer skips over the current checkpoint if it + * receives a barrier from a later checkpoint on a non-blocked input. + */ @Test - public void testMultiChannelBarrier() throws IOException, InterruptedException { - - List input = new LinkedList(); - input.add(createBuffer(0)); - input.add(createBuffer(1)); - input.add(createBarrier(1, 0)); - input.add(createBarrier(2, 0)); - input.add(createBuffer(0)); - input.add(createBarrier(3, 0)); - input.add(createBuffer(0)); - input.add(createBuffer(1)); - input.add(createBarrier(1, 1)); - input.add(createBuffer(0)); - input.add(createBuffer(1)); - input.add(createBarrier(2, 1)); - input.add(createBarrier(3, 1)); - input.add(createBarrier(4, 0)); - input.add(createBuffer(0)); - input.add(new BufferOrEvent(new EndOfPartitionEvent(), 1)); - + public void testMultiChannelSkippingCheckpoints() { + try { + BufferOrEvent[] sequence = { + // checkpoint 1 - with blocked data + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(1, 0), + createBuffer(1), createBuffer(0), + + // checkpoint 2 will not complete: pre-mature barrier from checkpoint 3 + createBarrier(2, 1), + createBuffer(1), createBuffer(2), + createBarrier(2, 0), + createBuffer(2), createBuffer(0), + createBarrier(3, 2), + + createBuffer(2), + createBuffer(1), createEndOfPartition(1), + createBuffer(2), createEndOfPartition(2), + createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + // checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + check(sequence[7], buffer.getNextNonBlocked()); + assertEquals(1L, buffer.getCurrentCheckpointId()); + + check(sequence[5], buffer.getNextNonBlocked()); + check(sequence[6], buffer.getNextNonBlocked()); + check(sequence[9], buffer.getNextNonBlocked()); + check(sequence[10], buffer.getNextNonBlocked()); + + // alignment of checkpoint 2 + check(sequence[13], buffer.getNextNonBlocked()); + assertEquals(2L, buffer.getCurrentCheckpointId()); + check(sequence[15], buffer.getNextNonBlocked()); + + // checkpoint 2 aborted, checkpoint 3 started + check(sequence[12], buffer.getNextNonBlocked()); + assertEquals(3L, buffer.getCurrentCheckpointId()); + check(sequence[16], buffer.getNextNonBlocked()); + check(sequence[19], buffer.getNextNonBlocked()); + check(sequence[20], buffer.getNextNonBlocked()); + check(sequence[23], buffer.getNextNonBlocked()); + check(sequence[24], buffer.getNextNonBlocked()); + + // end of input, emit remainder + check(sequence[18], buffer.getNextNonBlocked()); + check(sequence[21], buffer.getNextNonBlocked()); + check(sequence[22], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - InputGate mockIG1 = new MockInputGate(2, input); - AbstractReader mockAR1 = new MockReader(mockIG1); + /** + * Validates that the buffer skips over a later checkpoint if it + * receives a barrier from an even later checkpoint on a blocked input. + * + * NOTE: This test currently fails, because the barrier buffer does not support + * to unblock inputs before all previously unblocked data is consumed. + * + * Since this test checks only that the buffer behaves "failsafe" in cases of + * corrupt checkpoint barrier propagation (a situation that does not occur + * under the current model), we ignore it for the moment. + */ +// @Test + public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { + try { + BufferOrEvent[] sequence = { + // checkpoint 1 - with blocked data + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(1, 0), + createBuffer(1), createBuffer(0), + + // checkpoint 2 will not complete: pre-mature barrier from checkpoint 3 + createBarrier(2, 1), + createBuffer(1), createBuffer(2), + createBarrier(2, 0), + createBuffer(1), createBuffer(0), + + createBarrier(4, 1), // pre-mature barrier on blocked input + createBarrier(3, 0), // queued barrier, ignored on replay + + // complete checkpoint 2 + createBarrier(2, 0), + createBuffer(0), + + createBarrier(3, 2), // should be ignored + createBuffer(2), + createBarrier(4, 0), + createBuffer(0), createBuffer(1), createBuffer(2), + createBarrier(4, 1), + + createBuffer(1), createEndOfPartition(1), + createBuffer(2), createEndOfPartition(2), + createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + // checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + check(sequence[7], buffer.getNextNonBlocked()); + assertEquals(1L, buffer.getCurrentCheckpointId()); + check(sequence[5], buffer.getNextNonBlocked()); + check(sequence[6], buffer.getNextNonBlocked()); + check(sequence[9], buffer.getNextNonBlocked()); + check(sequence[10], buffer.getNextNonBlocked()); + + // alignment of checkpoint 2 + check(sequence[13], buffer.getNextNonBlocked()); + assertEquals(2L, buffer.getCurrentCheckpointId()); + + // checkpoint 2 completed + check(sequence[12], buffer.getNextNonBlocked()); + check(sequence[15], buffer.getNextNonBlocked()); + check(sequence[16], buffer.getNextNonBlocked()); + + // checkpoint 3 skipped, alignment for 4 started + check(sequence[20], buffer.getNextNonBlocked()); + assertEquals(4L, buffer.getCurrentCheckpointId()); + check(sequence[22], buffer.getNextNonBlocked()); + check(sequence[26], buffer.getNextNonBlocked()); + + // checkpoint 4 completed + check(sequence[24], buffer.getNextNonBlocked()); + check(sequence[25], buffer.getNextNonBlocked()); + + check(sequence[28], buffer.getNextNonBlocked()); + check(sequence[29], buffer.getNextNonBlocked()); + check(sequence[30], buffer.getNextNonBlocked()); + check(sequence[31], buffer.getNextNonBlocked()); + check(sequence[32], buffer.getNextNonBlocked()); + check(sequence[33], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } - BarrierBuffer bb = new BarrierBuffer(mockIG1, mockAR1); - BufferOrEvent nextBoe; + // ------------------------------------------------------------------------ + // Utils + // ------------------------------------------------------------------------ - check(input.get(0), nextBoe = bb.getNextNonBlocked()); - check(input.get(1), nextBoe = bb.getNextNonBlocked()); - check(input.get(2), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(7), nextBoe = bb.getNextNonBlocked()); - check(input.get(8), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(3), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(10), nextBoe = bb.getNextNonBlocked()); - check(input.get(11), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(4), nextBoe = bb.getNextNonBlocked()); - check(input.get(5), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(12), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(6), nextBoe = bb.getNextNonBlocked()); - check(input.get(9), nextBoe = bb.getNextNonBlocked()); - check(input.get(13), nextBoe = bb.getNextNonBlocked()); - bb.processBarrier(nextBoe); - check(input.get(14), nextBoe = bb.getNextNonBlocked()); - check(input.get(15), nextBoe = bb.getNextNonBlocked()); + private static BufferOrEvent createBarrier(long id, int channel) { + return new BufferOrEvent(new CheckpointBarrier(id, System.currentTimeMillis()), channel); + } - bb.cleanup(); + private static BufferOrEvent createBuffer(int channel) { + // since we have no access to the contents, we need to use the size as an + // identifier to validate correctness here + return new BufferOrEvent( + new Buffer(new MemorySegment(new byte[SIZE_COUNTER++]), DummyBufferRecycler.INSTANCE), channel); } - private static void check(BufferOrEvent expected, BufferOrEvent actual) { - assertEquals(expected.isBuffer(), actual.isBuffer()); - assertEquals(expected.getChannelIndex(), actual.getChannelIndex()); - if (expected.isEvent()) { - assertEquals(expected.getEvent(), actual.getEvent()); + private static BufferOrEvent createEndOfPartition(int channel) { + return new BufferOrEvent(EndOfPartitionEvent.INSTANCE, channel); + } + + private static void check(BufferOrEvent expected, BufferOrEvent present) { + assertNotNull(expected); + assertNotNull(present); + assertEquals(expected.isBuffer(), present.isBuffer()); + + if (expected.isBuffer()) { + // since we have no access to the contents, we need to use the size as an + // identifier to validate correctness here + assertEquals(expected.getBuffer().getSize(), present.getBuffer().getSize()); + } + else { + assertEquals(expected.getEvent(), present.getEvent()); } } + + // ------------------------------------------------------------------------ + // Testing Mocks + // ------------------------------------------------------------------------ - protected static class MockInputGate implements InputGate { + private static class MockInputGate implements InputGate { - private int numChannels; - private Queue boes; + private final int numChannels; + private final Queue boes; public MockInputGate(int numChannels, List boes) { this.numChannels = numChannels; - this.boes = new LinkedList(boes); + this.boes = new ArrayDeque(boes); } @Override @@ -176,48 +677,38 @@ public boolean isFinished() { } @Override - public void requestPartitions() throws IOException, InterruptedException { - } + public void requestPartitions() {} @Override - public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException { - return boes.remove(); + public BufferOrEvent getNextBufferOrEvent() { + return boes.poll(); } @Override - public void sendTaskEvent(TaskEvent event) throws IOException { - } + public void sendTaskEvent(TaskEvent event) {} @Override - public void registerListener(EventListener listener) { - } - + public void registerListener(EventListener listener) {} } - protected static class MockReader extends AbstractReader { + private static class ValidatingCheckpointHandler implements EventListener { + + private long nextExpectedCheckpointId = -1L; - protected MockReader(InputGate inputGate) { - super(inputGate); + public void setNextExpectedCheckpointId(long nextExpectedCheckpointId) { + this.nextExpectedCheckpointId = nextExpectedCheckpointId; } - @Override - public void setReporter(AccumulatorRegistry.Reporter reporter) { - + public long getNextExpectedCheckpointId() { + return nextExpectedCheckpointId; } - } - - protected static BufferOrEvent createBarrier(long id, int channel) { - return new BufferOrEvent(new CheckpointBarrier(id, System.currentTimeMillis()), channel); - } - protected static BufferOrEvent createBuffer(int channel) { - return new BufferOrEvent(new Buffer(new MemorySegment(new byte[] { 1 }), - new BufferRecycler() { - - @Override - public void recycle(MemorySegment memorySegment) { - } - }), channel); + @Override + public void onEvent(CheckpointBarrier barrier) { + assertNotNull(barrier); + assertTrue("wrong checkpoint id", nextExpectedCheckpointId == -1L || nextExpectedCheckpointId == barrier.getId()); + assertTrue(barrier.getTimestamp() > 0); + nextExpectedCheckpointId++; + } } - } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java index 23ca86dfe6e42..3f815ef937ef4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java @@ -25,10 +25,10 @@ * A BufferRecycler that does nothing. */ public class DummyBufferRecycler implements BufferRecycler { - + public static final BufferRecycler INSTANCE = new DummyBufferRecycler(); - - + + @Override public void recycle(MemorySegment memorySegment) {} -} \ No newline at end of file +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java index 9934bd9d7d198..b6cd656ccc116 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java @@ -22,18 +22,36 @@ import java.io.IOException; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; public class SpillingBufferOrEventTest { + + private static IOManager IO_MANAGER; + + @BeforeClass + public static void createIOManager() { + IO_MANAGER = new IOManagerAsync(); + } + + @AfterClass + public static void shutdownIOManager() { + IO_MANAGER.shutdown(); + } + // ------------------------------------------------------------------------ + @Test public void testSpilling() throws IOException, InterruptedException { - BufferSpiller bsp = new BufferSpiller(); + BufferSpiller bsp = new BufferSpiller(IO_MANAGER); SpillReader spr = new SpillReader(); BufferPool pool1 = new NetworkBufferPool(10, 256).createBufferPool(2, true); From 8f87b7164b644ea8f1708f7eb76567e58341b224 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 26 Jul 2015 19:05:30 +0200 Subject: [PATCH 018/175] [FLINK-2402] [streaming] Add a stream checkpoint barrier tracker. The BarrierTracker is non-blocking and only counts barriers. That way, it does not increase latency of records in the stream, but can only be used to obtain "at least once" processing guarantees. --- .../streaming/runtime/io/BarrierTracker.java | 194 +++++++++ .../runtime/io/BarrierTrackerTest.java | 404 ++++++++++++++++++ 2 files changed, 598 insertions(+) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java new file mode 100644 index 0000000000000..6b24556e6eae4 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java @@ -0,0 +1,194 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; + +import java.io.IOException; +import java.util.ArrayDeque; + +/** + * The BarrierTracker keeps track of what checkpoint barriers have been received from + * which input channels. + * + *

Unlike the {@link BarrierBuffer}, the BarrierTracker does not block the input + * channels that have sent barriers, so it cannot be used to gain "exactly-once" processing + * guarantees. It can, however, be used to gain "at least once" processing guarantees.

+ */ +public class BarrierTracker implements CheckpointBarrierHandler { + + private static final int MAX_CHECKPOINTS_TO_TRACK = 50; + + private final InputGate inputGate; + + private final int totalNumberOfInputChannels; + + private final ArrayDeque pendingCheckpoints; + + private EventListener checkpointHandler; + + private long latestPendingCheckpointID = -1; + + public BarrierTracker(InputGate inputGate) { + this.inputGate = inputGate; + this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); + this.pendingCheckpoints = new ArrayDeque(); + } + + @Override + public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException { + while (true) { + BufferOrEvent next = inputGate.getNextBufferOrEvent(); + if (next == null) { + return null; + } + else if (next.isBuffer() || next.getEvent().getClass() != CheckpointBarrier.class) { + return next; + } + else { + processBarrier((CheckpointBarrier) next.getEvent()); + } + } + } + + @Override + public void registerCheckpointEventHandler(EventListener checkpointHandler) { + if (this.checkpointHandler == null) { + this.checkpointHandler = checkpointHandler; + } + else { + throw new IllegalStateException("BarrierTracker already has a registered checkpoint handler"); + } + } + + @Override + public void cleanup() { + pendingCheckpoints.clear(); + } + + @Override + public boolean isEmpty() { + return pendingCheckpoints.isEmpty(); + } + + private void processBarrier(CheckpointBarrier receivedBarrier) { + // fast path for single channel trackers + if (totalNumberOfInputChannels == 1) { + if (checkpointHandler != null) { + checkpointHandler.onEvent(receivedBarrier); + } + return; + } + + // general path for multiple input channels + final long barrierId = receivedBarrier.getId(); + + // find the checkpoint barrier in the queue of bending barriers + CheckpointBarrierCount cbc = null; + int pos = 0; + + for (CheckpointBarrierCount next : pendingCheckpoints) { + if (next.checkpointId == barrierId) { + cbc = next; + break; + } + pos++; + } + + if (cbc != null) { + // add one to the count to that barrier and check for completion + int numBarriersNew = cbc.incrementBarrierCount(); + if (numBarriersNew == totalNumberOfInputChannels) { + // checkpoint can be triggered + // first, remove this checkpoint and all all prior pending + // checkpoints (which are now subsumed) + for (int i = 0; i <= pos; i++) { + pendingCheckpoints.pollFirst(); + } + + // notify the listener + if (checkpointHandler != null) { + checkpointHandler.onEvent(receivedBarrier); + } + } + } + else { + // first barrier for that checkpoint ID + // add it only if it is newer than the latest checkpoint. + // if it is not newer than the latest checkpoint ID, then there cannot be a + // successful checkpoint for that ID anyways + if (barrierId > latestPendingCheckpointID) { + latestPendingCheckpointID = barrierId; + pendingCheckpoints.addLast(new CheckpointBarrierCount(barrierId)); + + // make sure we do not track too many checkpoints + if (pendingCheckpoints.size() > MAX_CHECKPOINTS_TO_TRACK) { + pendingCheckpoints.pollFirst(); + } + } + } + } + + // ------------------------------------------------------------------------ + // + // ------------------------------------------------------------------------ + + /** + * Simple class for a checkpoint ID with a barrier counter. + */ + private static final class CheckpointBarrierCount { + + private final long checkpointId; + + private int barrierCount; + + private CheckpointBarrierCount(long checkpointId) { + this.checkpointId = checkpointId; + this.barrierCount = 1; + } + + public int incrementBarrierCount() { + return ++barrierCount; + } + + @Override + public int hashCode() { + return (int) ((checkpointId >>> 32) ^ checkpointId) + 17 * barrierCount; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof CheckpointBarrierCount) { + CheckpointBarrierCount that = (CheckpointBarrierCount) obj; + return this.checkpointId == that.checkpointId && this.barrierCount == that.barrierCount; + } + else { + return false; + } + } + + @Override + public String toString() { + return String.format("checkpointID=%d, count=%d", checkpointId, barrierCount); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java new file mode 100644 index 0000000000000..b2c570e409070 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java @@ -0,0 +1,404 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; + +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.List; +import java.util.Queue; + +import static org.junit.Assert.*; + +/** + * Tests for the behavior of the barrier tracker. + */ +public class BarrierTrackerTest { + + @Test + public void testSingleChannelNoBarriers() { + try { + BufferOrEvent[] sequence = { createBuffer(0), createBuffer(0), createBuffer(0) }; + + MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + for (BufferOrEvent boe : sequence) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testMultiChannelNoBarriers() { + try { + BufferOrEvent[] sequence = { createBuffer(2), createBuffer(2), createBuffer(0), + createBuffer(1), createBuffer(0), createBuffer(3), + createBuffer(1), createBuffer(1), createBuffer(2) + }; + + MockInputGate gate = new MockInputGate(4, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + for (BufferOrEvent boe : sequence) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testSingleChannelWithBarriers() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(0), createBuffer(0), + createBarrier(1, 0), + createBuffer(0), createBuffer(0), createBuffer(0), createBuffer(0), + createBarrier(2, 0), createBarrier(3, 0), + createBuffer(0), createBuffer(0), + createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0), + createBuffer(0) + }; + + MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + CheckpointSequenceValidator validator = + new CheckpointSequenceValidator(1, 2, 3, 4, 5, 6); + tracker.registerCheckpointEventHandler(validator); + + for (BufferOrEvent boe : sequence) { + if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testSingleChannelWithSkippedBarriers() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), + createBarrier(1, 0), + createBuffer(0), createBuffer(0), + createBarrier(3, 0), createBuffer(0), + createBarrier(4, 0), createBarrier(6, 0), createBuffer(0), + createBarrier(7, 0), createBuffer(0), createBarrier(10, 0), + createBuffer(0) + }; + + MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + CheckpointSequenceValidator validator = + new CheckpointSequenceValidator(1, 3, 4, 6, 7, 10); + tracker.registerCheckpointEventHandler(validator); + + for (BufferOrEvent boe : sequence) { + if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testMultiChannelWithBarriers() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), + createBarrier(1, 0), + + createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2), + createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2), + + createBuffer(2), createBuffer(2), + createBarrier(3, 2), + createBuffer(2), createBuffer(2), + createBarrier(3, 0), createBarrier(3, 1), + + createBarrier(4, 1), createBarrier(4, 2), createBarrier(4, 0), + + createBuffer(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + CheckpointSequenceValidator validator = + new CheckpointSequenceValidator(1, 2, 3, 4); + tracker.registerCheckpointEventHandler(validator); + + for (BufferOrEvent boe : sequence) { + if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testMultiChannelSkippingCheckpoints() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), + createBarrier(1, 0), + + createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2), + createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2), + + createBuffer(2), createBuffer(2), + createBarrier(3, 2), + createBuffer(2), createBuffer(2), + + // jump to checkpoint 4 + createBarrier(4, 0), + createBuffer(0), createBuffer(1), createBuffer(2), + createBarrier(4, 1), + createBuffer(1), + createBarrier(4, 2), + + createBuffer(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + CheckpointSequenceValidator validator = + new CheckpointSequenceValidator(1, 2, 4); + tracker.registerCheckpointEventHandler(validator); + + for (BufferOrEvent boe : sequence) { + if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + /** + * This test validates that the barrier tracker does not immediately + * discard a pending checkpoint as soon as it sees a barrier from a + * later checkpoint from some channel. + * + * This behavior is crucial, otherwise topologies where different inputs + * have different latency (and that latency is close to or higher than the + * checkpoint interval) may skip many checkpoints, or fail to complete a + * checkpoint all together. + */ + @Test + public void testCompleteCheckpointsOnLateBarriers() { + try { + BufferOrEvent[] sequence = { + // checkpoint 2 + createBuffer(1), createBuffer(1), createBuffer(0), createBuffer(2), + createBarrier(2, 1), createBarrier(2, 0), createBarrier(2, 2), + + // incomplete checkpoint 3 + createBuffer(1), createBuffer(0), + createBarrier(3, 1), createBarrier(3, 2), + + // some barriers from checkpoint 4 + createBuffer(1), createBuffer(0), + createBarrier(4, 2), createBarrier(4, 1), + createBuffer(1), createBuffer(2), + + // last barrier from checkpoint 3 + createBarrier(3, 0), + + // complete checkpoint 4 + createBuffer(0), createBarrier(4, 0), + + // regular checkpoint 5 + createBuffer(1), createBuffer(2), createBarrier(5, 1), + createBuffer(0), createBarrier(5, 0), + createBuffer(1), createBarrier(5, 2), + + // checkpoint 6 (incomplete), + createBuffer(1), createBarrier(6, 1), + createBuffer(0), createBarrier(6, 0), + + // checkpoint 7, with early barriers for checkpoints 8 and 9 + createBuffer(1), createBarrier(7, 1), + createBuffer(0), createBarrier(7, 2), + createBuffer(2), createBarrier(8, 2), + createBuffer(0), createBarrier(8, 1), + createBuffer(1), createBarrier(9, 1), + + // complete checkpoint 7, first barriers from checkpoint 10 + createBarrier(7, 0), + createBuffer(0), createBarrier(9, 2), + createBuffer(2), createBarrier(10, 2), + + // complete checkpoint 8 and 9 + createBarrier(8, 0), + createBuffer(1), createBuffer(2), createBarrier(9, 0), + + // trailing data + createBuffer(1), createBuffer(0), createBuffer(2) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierTracker tracker = new BarrierTracker(gate); + + CheckpointSequenceValidator validator = + new CheckpointSequenceValidator(2, 3, 4, 5, 7, 8, 9); + tracker.registerCheckpointEventHandler(validator); + + for (BufferOrEvent boe : sequence) { + if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) { + assertEquals(boe, tracker.getNextNonBlocked()); + } + } + + assertNull(tracker.getNextNonBlocked()); + assertNull(tracker.getNextNonBlocked()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // ------------------------------------------------------------------------ + // Utils + // ------------------------------------------------------------------------ + + private static BufferOrEvent createBarrier(long id, int channel) { + return new BufferOrEvent(new CheckpointBarrier(id, System.currentTimeMillis()), channel); + } + + private static BufferOrEvent createBuffer(int channel) { + return new BufferOrEvent( + new Buffer(new MemorySegment(new byte[] { 1 }), DummyBufferRecycler.INSTANCE), channel); + } + + // ------------------------------------------------------------------------ + // Testing Mocks + // ------------------------------------------------------------------------ + + private static class MockInputGate implements InputGate { + + private final int numChannels; + private final Queue boes; + + public MockInputGate(int numChannels, List boes) { + this.numChannels = numChannels; + this.boes = new ArrayDeque(boes); + } + + @Override + public int getNumberOfInputChannels() { + return numChannels; + } + + @Override + public boolean isFinished() { + return boes.isEmpty(); + } + + @Override + public void requestPartitions() {} + + @Override + public BufferOrEvent getNextBufferOrEvent() { + return boes.poll(); + } + + @Override + public void sendTaskEvent(TaskEvent event) {} + + @Override + public void registerListener(EventListener listener) {} + } + + private static class CheckpointSequenceValidator implements EventListener { + + private final long[] checkpointIDs; + + private int i = 0; + + private CheckpointSequenceValidator(long... checkpointIDs) { + this.checkpointIDs = checkpointIDs; + } + + @Override + public void onEvent(CheckpointBarrier barrier) { + assertTrue("More checkpoints than expected", i < checkpointIDs.length); + assertNotNull(barrier); + assertEquals("wrong checkpoint id", checkpointIDs[i++], barrier.getId()); + assertTrue(barrier.getTimestamp() > 0); + } + } +} From 22473eaeec9edf64d2215ac1c0c02c0fb648c4b8 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 28 Jul 2015 17:58:14 +0300 Subject: [PATCH 019/175] Integrating Pull Request Comments. --- .../flink/configuration/ConfigConstants.java | 8 +++++++ .../accumulators/BaseAccumulatorSnapshot.java | 9 +++++--- .../accumulators/LargeAccumulatorHelper.java | 18 +++++++-------- .../LargeAccumulatorSnapshot.java | 13 +++++++++-- .../SmallAccumulatorSnapshot.java | 4 ++++ .../executiongraph/ExecutionGraph.java | 23 +++++++++++++++---- .../flink/runtime/taskmanager/Task.java | 18 ++++----------- .../flink/runtime/util/SerializedValue.java | 4 ++-- .../apache/flink/runtime/akka/AkkaUtils.scala | 16 ++++++------- .../flink/runtime/jobmanager/JobManager.scala | 11 ++------- .../runtime/taskmanager/TaskManager.scala | 19 +++------------ 11 files changed, 74 insertions(+), 69 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index c76741b2de8ee..3ebe320bda825 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -414,6 +414,12 @@ public final class ConfigConstants { * Exit JVM on fatal Akka errors */ public static final String AKKA_JVM_EXIT_ON_FATAL_ERROR = "akka.jvm-exit-on-fatal-error"; + + /** + * The threshold (in bytes) above which an object is considered too big + * to transfer using akka + */ + public static final String AKKA_FRAMESIZE_OVERSIZED_THRESHOLD = "akka.framesize.oversized.threshold"; // ----------------------------- Streaming -------------------------------- @@ -662,6 +668,8 @@ public final class ConfigConstants { public static String DEFAULT_AKKA_ASK_TIMEOUT = "100 s"; public static String DEFAULT_AKKA_LOOKUP_TIMEOUT = "10 s"; + + public static double DEFAULT_AKKA_FRAMESIZE_OVERSIZED_THRESHOLD = 0.8; // ----------------------------- Streaming Values -------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java index 39803814f14b8..8c4faf03bc358 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java @@ -34,11 +34,14 @@ public class BaseAccumulatorSnapshot implements Serializable { private final JobID jobID; private final ExecutionAttemptID executionAttemptID; - /** - * Flink internal accumulators which can be deserialized using the system class loader. - */ + /** Flink internal accumulators which can be deserialized using the system class loader. */ private final SerializedValue>> flinkAccumulators; + /** + * This class and its subclasses encapsulate a map of accumulators for a single + * task. It is used for the transfer from TaskManagers to the JobManager and from + * the JobManager to the Client. + */ public BaseAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, Map> flinkAccumulators) throws IOException { this.jobID = jobID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java index 160a612dd265a..7d9ca8af0c8ef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java @@ -77,17 +77,15 @@ public static Map> storeSerializedAccumulatorsToBlobCache( String accumulatorName = entry.getKey(); byte[] accumulatorPayload = entry.getValue().getSerializedData(); - if(accumulatorPayload == null) { - continue; + if(accumulatorPayload != null) { + BlobKey blobKey = bc.put(accumulatorPayload); + List accKeys = keys.get(accumulatorName); + if (accKeys == null) { + accKeys = new ArrayList(); + } + accKeys.add(blobKey); + keys.put(accumulatorName, accKeys); } - - BlobKey blobKey = bc.put(accumulatorPayload); - List accKeys = keys.get(accumulatorName); - if (accKeys == null) { - accKeys = new ArrayList(); - } - accKeys.add(blobKey); - keys.put(accumulatorName, accKeys); } } catch (IOException e) { throw new IOException("Failed to send oversized accumulators to the BlobCache: ", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java index 791796b9ead46..823a03c2e3a08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java @@ -36,6 +36,14 @@ public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { * */ private final Map> largeUserAccumulatorBlobs; + /** + * In case some user-defined accumulators do not fit in an Akka message payload, we store them in the + * blobCache, and put in the snapshot only the mapping between the name of the accumulator, + * and its blobKey in the cache. This clase is a subclass of the BaseAccumulatorSnapshot + * and holds the (potential) references to blobs stored in the BlobCache and containing + * these oversized accumulators. It is used for the transfer from TaskManagers to the + * JobManager and from the JobManager to the Client. + */ public LargeAccumulatorSnapshot( JobID jobID, ExecutionAttemptID executionAttemptID, Map> flinkAccumulators, @@ -45,8 +53,9 @@ public LargeAccumulatorSnapshot( } /** - * Gets the Flink (internal) accumulators values. - * @return the serialized map + * Gets the BlobKeys of the oversized accumulators that were too big to be sent through akka, and + * had to be stored in the BlobCache. + * @return the maping between accumulator and its blobKeys. */ public Map> getLargeAccumulatorBlobKeys() { return largeUserAccumulatorBlobs; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java index 6773ee49ff479..f2a381ce72565 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java @@ -33,6 +33,10 @@ public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { */ private final SerializedValue>> userAccumulators; + /** + * This is a subclass of the BaseAccumulatorSnapshot that serves at storing the task user-defined + * accumulators that are small enough to be sent to the JobManager using akka. + * */ public SmallAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, Map> flinkAccumulators, SerializedValue>> userAccumulators) throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 0326ddb95d5c5..808b11f7549a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -592,7 +592,7 @@ public Map> aggregateSmallUserAccumulators() { * @param reaggregate true if we want to aggregate accumulators, * false otherwise. * @return The accumulator map - */ + */ public Map> aggregateSmallUserAccumulators(boolean reaggregate) { if(!reaggregate) { return mergedSmallUserAccumulators; @@ -676,6 +676,19 @@ public Map> getSmallAccumulatorsSerialized() thr return serializeAccumulators(false); } + /** + * Merges and serializes all accumulator results from the tasks previously executed in + * the Executions. If onlyContent is set to true, then the Accumulators are + * merged and the content of the resulting Accumulator is serialized and returned. In other + * case, the result is assumed to be merged, so no additional merging is performed (as this + * could lead to duplicate entries), and the whole accumulator object is serialized and + * returned. + * @param onlyContent true if we want to aggregate accumulators and serialize just + * the content of the result, false if (partial) accumulators + * are already merged (so no additional merging is required), and we want the + * whole object serialized. + * @return The accumulator map + */ private Map> serializeAccumulators(boolean onlyContent) throws IOException { Map> accumulatorMap = aggregateSmallUserAccumulators(onlyContent); @@ -698,10 +711,10 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { Map> largeAccumulatorMap = aggregateLargeUserAccumulatorBlobKeys(); // get the total number of (unique) accumulators - Set uniqAccumulators = new HashSet(); - uniqAccumulators.addAll(smallAccumulatorMap.keySet()); - uniqAccumulators.addAll(largeAccumulatorMap.keySet()); - int num = uniqAccumulators.size(); + Set uniqueAccumulators = new HashSet(); + uniqueAccumulators.addAll(smallAccumulatorMap.keySet()); + uniqueAccumulators.addAll(largeAccumulatorMap.keySet()); + int num = uniqueAccumulators.size(); StringifiedAccumulatorResult[] resultStrings = new StringifiedAccumulatorResult[num]; int i = 0; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 4a3db94e8d352..36192a84f4a93 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -60,7 +60,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; -import scala.runtime.AbstractFunction0; import java.io.IOException; import java.net.InetSocketAddress; @@ -260,8 +259,7 @@ public Task(TaskDeploymentDescriptor tdd, this.libraryCache = checkNotNull(libraryCache); this.fileCache = checkNotNull(fileCache); this.network = checkNotNull(networkEnvironment); - - + this.accumulatorRegistry = new AccumulatorRegistry(tdd.getJobConfiguration(), jobId, executionId, getBlobCacheServerAddress()); @@ -324,18 +322,10 @@ private InetSocketAddress getBlobCacheServerAddress() { throw new RuntimeException("TaskManager not associated to JobManager."); } - final String jmHost; + String jmHost = "localhost"; ActorRef jobManagerActor = this.jobManager.actor(); - if (jobManagerActor == null) { - jmHost = "localhost"; - } else { - jmHost = jobManagerActor.path().address().host().getOrElse( - new AbstractFunction0() { - @Override - public String apply() { - return "localhost"; - } - }); + if (jobManagerActor != null && jobManagerActor.path().address().host().isDefined()) { + jmHost = jobManagerActor.path().address().host().get(); } int blobPort = this.libraryCache.getBlobServerPort(); return new InetSocketAddress(jmHost, blobPort); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java index 08af378af0f8f..41cd5f17627c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java @@ -48,7 +48,7 @@ public SerializedValue(T value) throws IOException { } public SerializedValue(byte[] data) { - this.serializedData = data == null ? null : data; + this.serializedData = data; } @SuppressWarnings("unchecked") @@ -74,7 +74,7 @@ public byte[] getSerializedData() { * */ public long getSizeInBytes() { if(serializedData == null) { - throw new RuntimeException("No data in this Blob."); + throw new RuntimeException("Data Payload is NULL."); } return serializedData.length; } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 54c74eda087cb..405c4140ffb6e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -424,13 +424,13 @@ object AkkaUtils { def getFramesize(config: Configuration): Long = config.getString( ConfigConstants.AKKA_FRAMESIZE, - ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d.]", "").toLong + ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d]", "").toLong - /** - * @return the threshold (in bytes) above which an object is considered too big - * to transfer using akka. For now this parameter is set to 80% of the - * akka.framesize. - * */ - def getLargeAccumulatorThreshold(config: Configuration): Long = - (0.8 * getFramesize(config)).toLong + def getLargeAccumulatorThreshold(config: Configuration): Long = { + val threshold = config.getDouble( + ConfigConstants.AKKA_FRAMESIZE_OVERSIZED_THRESHOLD, + ConfigConstants.DEFAULT_AKKA_FRAMESIZE_OVERSIZED_THRESHOLD) + + (threshold * getFramesize(config)).toLong + } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 3e3044befda0e..b231078fb611c 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -368,10 +368,6 @@ class JobManager( // and update the blobKeys to send to the client. largeAccumulatorResults = executionGraph. addLargeUserAccumulatorBlobKeys(largeAccumulatorResults, newBlobKeys) - - } else { - // do nothing - java.util.Collections.emptyMap() } val result = new SerializedJobExecutionResult(jobID, @@ -550,11 +546,10 @@ class JobManager( **/ private def getBlobCacheServerAddress: InetSocketAddress = { if (libraryCacheManager == null) { - throw new RuntimeException("LibraryCacheManage is not initialized yet.") + throw new RuntimeException("LibraryCacheManager is not initialized yet.") } - val jmHost: String = "localhost" val blobPort: Int = this.libraryCacheManager.getBlobServerPort - return new InetSocketAddress(jmHost, blobPort) + return new InetSocketAddress("localhost", blobPort) } /** @@ -953,8 +948,6 @@ class JobManager( try { eg.prepareForArchiving() - // todo KOSTAS: handle also the large accumulators. - archive ! decorateMessage(ArchiveExecutionGraph(jobID, eg)) } catch { case t: Throwable => log.error(s"Could not prepare the execution graph $eg for " + diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 364e7a8dea8a7..ec0bb105e3edf 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -24,25 +24,17 @@ import java.lang.reflect.Method import java.net.{InetAddress, InetSocketAddress} import java.util.UUID import java.util.concurrent.TimeUnit -import java.lang.reflect.Method -import java.lang.management.{OperatingSystemMXBean, ManagementFactory} import _root_.akka.actor._ import _root_.akka.pattern.ask import _root_.akka.util.Timeout - -import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.codahale.metrics.json.MetricsModule import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger - -import org.apache.flink.configuration.{Configuration, ConfigConstants, GlobalConfiguration, IllegalConfigurationException} - +import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration, IllegalConfigurationException} import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot -import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage} -import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.blob.{BlobCache, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager @@ -50,8 +42,7 @@ import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, Ta import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} import org.apache.flink.runtime.executiongraph.ExecutionAttemptID import org.apache.flink.runtime.filecache.FileCache -import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, -InstanceConnectionInfo, InstanceID} +import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID} import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync} import org.apache.flink.runtime.io.network.NetworkEnvironment @@ -69,7 +60,7 @@ import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner import org.apache.flink.runtime.util.{EnvironmentInformation, MathUtils, ZooKeeperUtil} -import org.apache.flink.runtime.StreamingMode +import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -78,10 +69,6 @@ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps import scala.util.{Failure, Success} -import scala.collection.JavaConverters._ -import scala.collection.JavaConversions._ - -import scala.language.postfixOps /** * The TaskManager is responsible for executing the individual tasks of a Flink job. It is From 2fc08b8e6ce44e2374346c2e237579666da4e9cb Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 29 Jul 2015 02:19:11 +0300 Subject: [PATCH 020/175] Minute change in ExecutionGraph. --- .../executiongraph/ExecutionGraph.java | 67 ++++++++++++------- 1 file changed, 42 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 808b11f7549a5..7372ad1ecde22 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -658,50 +658,67 @@ private void mergeLargeUserAccumulatorBlobKeys(Map> target } /** - * Gets a serialized map of the contents of the accumulators. + * Merges the partial accumulators referring to the same global accumulator received from the tasks, + * and serializes the final result. Each of the partial accumulators contains the partial result + * produced by each task, for that specific accumulator. * @return The accumulator map with serialized accumulator values. * @throws IOException */ public Map> getSmallAccumulatorsContentSerialized() throws IOException { - return serializeAccumulators(true); + Map> accumulatorMap = aggregateSmallUserAccumulators(true); + + Map> result = new HashMap>(); + for (Map.Entry> entry : accumulatorMap.entrySet()) { + result.put(entry.getKey(), new SerializedValue(entry.getValue().getLocalValue())); + } + return result; } /** - * Gets a serialized map of the objects of the accumulators. This means that the actual + * Serializes the objects of the accumulators (not only the content as the + * {@link #getSmallAccumulatorsContentSerialized()}. This means that the actual * objects are serialized, thus merging can still be applied after deserialization. + * This method assumes that partial accumulators received from the tasks that refer to + * the same global accumulator have already been merged. * @return The accumulator map with serialized accumulator objects. * @throws IOException */ public Map> getSmallAccumulatorsSerialized() throws IOException { - return serializeAccumulators(false); - } - - /** - * Merges and serializes all accumulator results from the tasks previously executed in - * the Executions. If onlyContent is set to true, then the Accumulators are - * merged and the content of the resulting Accumulator is serialized and returned. In other - * case, the result is assumed to be merged, so no additional merging is performed (as this - * could lead to duplicate entries), and the whole accumulator object is serialized and - * returned. - * @param onlyContent true if we want to aggregate accumulators and serialize just - * the content of the result, false if (partial) accumulators - * are already merged (so no additional merging is required), and we want the - * whole object serialized. - * @return The accumulator map - */ - private Map> serializeAccumulators(boolean onlyContent) throws IOException { - - Map> accumulatorMap = aggregateSmallUserAccumulators(onlyContent); + Map> accumulatorMap = aggregateSmallUserAccumulators(false); Map> result = new HashMap>(); for (Map.Entry> entry : accumulatorMap.entrySet()) { - Object toSerialize = onlyContent ? entry.getValue().getLocalValue() : entry.getValue(); - result.put(entry.getKey(), new SerializedValue(toSerialize)); + result.put(entry.getKey(), new SerializedValue(entry.getValue())); } - return result; } +// /** +// * Merges and serializes all accumulator results from the tasks previously executed in +// * the Executions. If onlyContent is set to true, then the Accumulators are +// * merged and the content of the resulting Accumulator is serialized and returned. In other +// * case, the result is assumed to be merged, so no additional merging is performed (as this +// * could lead to duplicate entries), and the whole accumulator object is serialized and +// * returned. +// * @param onlyContent true if we want to aggregate accumulators and serialize just +// * the content of the result, false if (partial) accumulators +// * are already merged (so no additional merging is required), and we want the +// * whole object serialized. +// * @return The accumulator map +// */ +// private Map> serializeAccumulators(boolean onlyContent) throws IOException { +// +// Map> accumulatorMap = aggregateSmallUserAccumulators(onlyContent); +// +// Map> result = new HashMap>(); +// for (Map.Entry> entry : accumulatorMap.entrySet()) { +// Object toSerialize = onlyContent ? entry.getValue().getLocalValue() : entry.getValue(); +// result.put(entry.getKey(), new SerializedValue(toSerialize)); +// } +// +// return result; +// } + /** * Returns the a stringified version of the user-defined accumulators. * @return an Array containing the StringifiedAccumulatorResult objects From ada9037bef760d46a4c3be2177e04bd72e620dad Mon Sep 17 00:00:00 2001 From: root Date: Mon, 27 Jul 2015 16:54:05 +0800 Subject: [PATCH 021/175] Fix [FLINK-2391] Storm-compatibility:method FlinkTopologyBuilder.createTopology() throws java.lang.NullPointerException bug This closes #940 --- .../flink/stormcompatibility/api/FlinkTopologyBuilder.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java index 6c3956195987e..4ecf4a6acdc67 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java @@ -64,6 +64,8 @@ public class FlinkTopologyBuilder { private final HashMap bolts = new HashMap(); /** All declared output schemas by operator ID */ private final HashMap outputSchemas = new HashMap(); + /** All spouts&bolts declarers by their ID */ + private final HashMap declarers = new HashMap(); /** * Creates a Flink program that used the specified spouts and bolts. @@ -84,6 +86,7 @@ public FlinkTopology createTopology() { final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); userSpout.declareOutputFields(declarer); this.outputSchemas.put(spoutId, declarer.outputSchema); + declarers.put(spoutId, declarer); /* TODO in order to support multiple output streams, use an additional wrapper (or modify StormSpoutWrapper * and StormCollector) @@ -124,6 +127,7 @@ public FlinkTopology createTopology() { final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); userBolt.declareOutputFields(declarer); this.outputSchemas.put(boltId, declarer.outputSchema); + declarers.put(boltId, declarer); final ComponentCommon common = stormTopolgoy.get_bolts().get(boltId).get_common(); @@ -153,7 +157,8 @@ public FlinkTopology createTopology() { // global grouping is emulated in Storm via an empty fields grouping list final List fields = grouping.get_fields(); if (fields.size() > 0) { - inputDataStream = inputDataStream.groupBy(declarer.getGroupingFieldIndexes(grouping + FlinkOutputFieldsDeclarer procDeclarer = this.declarers.get(producerId); + inputDataStream = inputDataStream.groupBy(procDeclarer.getGroupingFieldIndexes(grouping .get_fields())); } else { inputDataStream = inputDataStream.global(); From 3e49daf26b1b7d78e2ae846f40051bb7cfa7ab84 Mon Sep 17 00:00:00 2001 From: Peter Schrott Date: Tue, 14 Jul 2015 10:39:58 +0200 Subject: [PATCH 022/175] [docs] Updated method documentation. Changed .isEqualTo(..) to .equalsTo(..) This close #909 --- .../scala/org/apache/flink/api/scala/joinDataSet.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala index ce9d6e634c4a4..0ed74f4270476 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala @@ -40,7 +40,7 @@ import scala.reflect.ClassTag * {{{ * val left = ... * val right = ... - * val joinResult = left.join(right).where(0, 2).isEqualTo(0, 1) { + * val joinResult = left.join(right).where(0, 2).equalTo(0, 1) { * (left, right) => new MyJoinResult(left, right) * } * }}} @@ -49,7 +49,7 @@ import scala.reflect.ClassTag * {{{ * val left = ... * val right = ... - * val joinResult = left.join(right).where({_._1}).isEqualTo({_._1) { + * val joinResult = left.join(right).where({_._1}).equalTo({_._1) { * (left, right) => new MyJoinResult(left, right) * } * }}} @@ -210,12 +210,12 @@ class JoinDataSet[L, R]( /** * An unfinished join operation that results from [[DataSet.join()]] The keys for the left and right - * side must be specified using first `where` and then `isEqualTo`. For example: + * side must be specified using first `where` and then `equalTo`. For example: * * {{{ * val left = ... * val right = ... - * val joinResult = left.join(right).where(...).isEqualTo(...) + * val joinResult = left.join(right).where(...).equalTo(...) * }}} * @tparam L The type of the left input of the join. * @tparam R The type of the right input of the join. From a34c9416790e0cddedb3f2518fd0bea2331cbcc0 Mon Sep 17 00:00:00 2001 From: Alexander Alexandrov Date: Fri, 24 Jul 2015 20:57:04 +0200 Subject: [PATCH 023/175] [FLINK-2231] Create Serializer and Comparator for Scala Enumerations. This closes #935 --- .../api/scala/codegen/TypeAnalyzer.scala | 38 ++++++- .../api/scala/codegen/TypeDescriptors.scala | 6 +- .../scala/codegen/TypeInformationGen.scala | 18 ++- .../scala/typeutils/EnumValueComparator.scala | 105 ++++++++++++++++++ .../scala/typeutils/EnumValueSerializer.scala | 65 +++++++++++ .../scala/typeutils/EnumValueTypeInfo.scala | 52 +++++++++ .../runtime/EnumValueComparatorTest.scala | 53 +++++++++ .../ScalaSpecialTypesSerializerTest.scala | 27 +++-- 8 files changed, 348 insertions(+), 16 deletions(-) create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueComparator.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/EnumValueComparatorTest.scala diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala index 6ad73a5f5458c..0421557b442f3 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala @@ -28,7 +28,6 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] with TypeDescriptors[C] => import c.universe._ - import compat._ // This value is controlled by the udtRecycling compiler option var enableMutableUDTs = false @@ -61,6 +60,8 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] case EitherType(leftTpe, rightTpe) => analyzeEither(id, tpe, leftTpe, rightTpe) + case EnumValueType(enum) => EnumValueDescriptor(id, tpe, enum) + case TryType(elemTpe) => analyzeTry(id, tpe, elemTpe) case OptionType(elemTpe) => analyzeOption(id, tpe, elemTpe) @@ -302,6 +303,8 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] traversable match { case TypeRef(_, _, elemTpe :: Nil) => + import compat._ // this is needed in order to compile in Scala 2.11 + // determine whether we can find an implicit for the CanBuildFrom because // TypeInformationGen requires this. This catches the case where a user // has a custom class that implements Iterable[], for example. @@ -350,6 +353,37 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] } } + private object EnumValueType { + def unapply(tpe: Type): Option[ModuleSymbol] = { + // somewhat hacky solution based on the 'org.example.MyEnum.Value' FQN + // convention, compatible with Scala 2.10 + try { + val m = c.universe.rootMirror + // get fully-qualified type name, e.g. org.example.MyEnum.Value + val fqn = tpe.normalize.toString.split('.') + // get FQN parent + val owner = m.staticModule(fqn.slice(0, fqn.size - 1).mkString(".")) + + val enumerationSymbol = typeOf[scala.Enumeration].typeSymbol + if (owner.typeSignature.baseClasses.contains(enumerationSymbol)) { + Some(owner) + } else { + None + } + } catch { + case e: Throwable => None + } + // TODO: use this once 2.10 is no longer supported + // tpe is the Enumeration.Value alias, get the owner + // val owner = tpe.typeSymbol.owner + // if (owner.isModule && + // owner.typeSignature.baseClasses.contains(typeOf[scala.Enumeration].typeSymbol)) + // Some(owner.asModule) + // else + // None + } + } + private object TryType { def unapply(tpe: Type): Option[Type] = { if (tpe <:< typeOf[scala.util.Try[_]]) { @@ -441,7 +475,7 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] def getBoxInfo(prim: Symbol, primName: String, boxName: String) = { val (default, wrapper) = primitives(prim) - val box = { t: Tree => + val box = { t: Tree => Apply( Select( Select(Ident(newTermName("scala")), newTermName("Predef")), diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala index c6006a2a6d8db..b65defd0b1294 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala @@ -18,11 +18,7 @@ package org.apache.flink.api.scala.codegen import scala.language.postfixOps - import scala.reflect.macros.Context -import scala.reflect.classTag -import scala.reflect.ClassTag -import scala.Option.option2Iterable // These are only used internally while analyzing Scala types in TypeAnalyzer and TypeInformationGen @@ -48,6 +44,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C] case class EitherDescriptor(id: Int, tpe: Type, left: UDTDescriptor, right: UDTDescriptor) extends UDTDescriptor + case class EnumValueDescriptor(id: Int, tpe: Type, enum: ModuleSymbol) extends UDTDescriptor + case class TryDescriptor(id: Int, tpe: Type, elem: UDTDescriptor) extends UDTDescriptor case class OptionDescriptor(id: Int, tpe: Type, elem: UDTDescriptor) extends UDTDescriptor diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala index a6fbb71e4feaa..07f7205256491 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala @@ -21,9 +21,7 @@ import java.lang.reflect.{Field, Modifier} import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo._ - import org.apache.flink.api.common.typeutils._ -import org.apache.flink.api.java.tuple.Tuple import org.apache.flink.api.java.typeutils._ import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo} import org.apache.flink.types.Value @@ -32,7 +30,6 @@ import org.apache.hadoop.io.Writable import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.postfixOps - import scala.reflect.macros.Context private[flink] trait TypeInformationGen[C <: Context] { @@ -66,6 +63,8 @@ private[flink] trait TypeInformationGen[C <: Context] { case e: EitherDescriptor => mkEitherTypeInfo(e) + case e: EnumValueDescriptor => mkEnumValueTypeInfo(e) + case tr: TryDescriptor => mkTryTypeInfo(tr) case o: OptionDescriptor => mkOptionTypeInfo(o) @@ -150,6 +149,19 @@ private[flink] trait TypeInformationGen[C <: Context] { c.Expr[TypeInformation[T]](result) } + def mkEnumValueTypeInfo[T: c.WeakTypeTag](d: EnumValueDescriptor): c.Expr[TypeInformation[T]] = { + + val enumValueClass = c.Expr[Class[T]](Literal(Constant(weakTypeOf[T]))) + + val result = q""" + import org.apache.flink.api.scala.typeutils.EnumValueTypeInfo + + new EnumValueTypeInfo[${d.enum.typeSignature}](${d.enum}, $enumValueClass) + """ + + c.Expr[TypeInformation[T]](result) + } + def mkTryTypeInfo[T: c.WeakTypeTag](desc: TryDescriptor): c.Expr[TypeInformation[T]] = { val elemTypeInfo = mkTypeInfo(desc.elem)(c.WeakTypeTag(desc.elem.tpe)) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueComparator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueComparator.scala new file mode 100644 index 0000000000000..4aa0a73bcd836 --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueComparator.scala @@ -0,0 +1,105 @@ +/* + * 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.api.scala.typeutils + +import org.apache.flink.api.common.typeutils.TypeComparator +import org.apache.flink.api.common.typeutils.base.IntComparator +import org.apache.flink.core.memory.{DataOutputView, DataInputView, MemorySegment} + +/** + * Comparator for [[Enumeration]] values. + */ +@SerialVersionUID(1000L) +class EnumValueComparator[E <: Enumeration](ascComp: Boolean) extends TypeComparator[E#Value] { + + type T = E#Value + + final val intComparator = new IntComparator(ascComp) + + // We cannot use the Clone Constructor from Scala so we have to do it manually + def duplicate: TypeComparator[T] = { + new EnumValueComparator[E](ascComp) + } + + // -------------------------------------------------------------------------------------------- + // Comparator Methods + // -------------------------------------------------------------------------------------------- + + override def compareSerialized(firstSource: DataInputView, secondSource: DataInputView): Int = { + intComparator.compareSerialized(firstSource, secondSource) + } + + def supportsNormalizedKey: Boolean = { + intComparator.supportsNormalizedKey + } + + def getNormalizeKeyLen: Int = { + intComparator.getNormalizeKeyLen + } + + def isNormalizedKeyPrefixOnly(keyBytes: Int): Boolean = { + intComparator.isNormalizedKeyPrefixOnly(keyBytes) + } + + override def putNormalizedKey(v: T, target: MemorySegment, offset: Int, numBytes: Int): Unit = { + intComparator.putNormalizedKey(v.id, target, offset, numBytes) + } + + override def hash(record: T): Int = intComparator.hash(record.id) + + override def setReference(toCompare: T): Unit = { + intComparator.setReference(toCompare.id) + } + + override def equalToReference(candidate: T): Boolean = { + intComparator.equalToReference(candidate.id) + } + + override def compareToReference(referencedComparator: TypeComparator[T]): Int = { + intComparator.compareToReference(referencedComparator.asInstanceOf[this.type].intComparator) + } + + override def compare(first: E#Value, second: E#Value): Int = { + intComparator.compare(first.id, second.id) + } + + override def invertNormalizedKey(): Boolean = { + intComparator.invertNormalizedKey() + } + + override def writeWithKeyNormalization(record: T, target: DataOutputView): Unit = { + intComparator.writeWithKeyNormalization(record.id, target) + } + + override def supportsSerializationWithKeyNormalization(): Boolean = { + intComparator.supportsSerializationWithKeyNormalization() + } + + override def extractKeys(record: AnyRef, target: Array[AnyRef], index: Int): Int = { + target(index) = record + 1 + } + + override lazy val getFlatComparators: Array[TypeComparator[_]] = { + Array(this) + } + + override def readWithKeyDenormalization(reuse: T, source: DataInputView): T = { + throw new UnsupportedOperationException + } +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala new file mode 100644 index 0000000000000..8d03676e6162d --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala @@ -0,0 +1,65 @@ +/* + * 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.api.scala.typeutils + +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.IntSerializer +import org.apache.flink.core.memory.{DataInputView, DataOutputView} + +/** + * Serializer for [[Enumeration]] values. + */ +class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[E#Value] { + + type T = E#Value + + val intSerializer = new IntSerializer() + + override def duplicate: EnumValueSerializer[E] = this + + override def createInstance: T = enum(0) + + override def isImmutableType: Boolean = true + + override def getLength: Int = intSerializer.getLength + + override def copy(from: T): T = enum.apply(from.id) + + override def copy(from: T, reuse: T): T = copy(from) + + override def copy(src: DataInputView, tgt: DataOutputView): Unit = intSerializer.copy(src, tgt) + + override def serialize(v: T, tgt: DataOutputView): Unit = intSerializer.serialize(v.id, tgt) + + override def deserialize(source: DataInputView): T = enum(intSerializer.deserialize(source)) + + override def deserialize(reuse: T, source: DataInputView): T = deserialize(source) + + override def equals(obj: Any): Boolean = { + if (obj != null && obj.isInstanceOf[EnumValueSerializer[_]]) { + val other = obj.asInstanceOf[EnumValueSerializer[_]] + this.enum == other.enum + } else { + false + } + } + + override def hashCode(): Int = { + enum.hashCode() + } +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala new file mode 100644 index 0000000000000..c66e4bc9dac1d --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala @@ -0,0 +1,52 @@ +/* + * 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.api.scala.typeutils + +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} +import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} + +import scala.collection.JavaConverters._ + +/** + * TypeInformation for [[Enumeration]] values. + */ +class EnumValueTypeInfo[E <: Enumeration](enum: E, clazz: Class[E#Value]) + extends TypeInformation[E#Value] with AtomicType[E#Value] { + + type T = E#Value + + override def isBasicType: Boolean = false + override def isTupleType: Boolean = false + override def isKeyType: Boolean = true + override def getTotalFields: Int = 1 + override def getArity: Int = 1 + override def getTypeClass = clazz + override def getGenericParameters = List.empty[TypeInformation[_]].asJava + + + def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { + new EnumValueSerializer[E](enum) + } + + override def createComparator(ascOrder: Boolean, config: ExecutionConfig): TypeComparator[T] = { + new EnumValueComparator[E](ascOrder) + } + + override def toString = clazz.getCanonicalName +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/EnumValueComparatorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/EnumValueComparatorTest.scala new file mode 100644 index 0000000000000..a0cc03f8d4294 --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/EnumValueComparatorTest.scala @@ -0,0 +1,53 @@ +/* + * 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.api.scala.runtime + +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer} +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.typeutils.EnumValueTypeInfo + + +class EnumValueComparatorTest extends ComparatorTestBase[Suit.Value] { + + protected def createComparator(ascending: Boolean): TypeComparator[Suit.Value] = { + val ti = createTypeInformation[Suit.Value] + ti.asInstanceOf[EnumValueTypeInfo[Suit.type]].createComparator(ascending, new ExecutionConfig) + } + + protected def createSerializer: TypeSerializer[Suit.Value] = { + val ti = createTypeInformation[Suit.Value] + ti.createSerializer(new ExecutionConfig) + } + + protected def getSortedTestData: Array[Suit.Value] = { + dataISD + } + + private val dataISD = Array( + Suit.Clubs, + Suit.Diamonds, + Suit.Hearts, + Suit.Spades + ) +} + +object Suit extends Enumeration { + type Suit = Value + val Clubs, Diamonds, Hearts, Spades = Value +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala index e4454b7604b43..155160c08d988 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala @@ -18,15 +18,14 @@ package org.apache.flink.api.scala.runtime import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.common.typeutils.{SerializerTestInstance, TypeSerializer} import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.typeutils.EnumValueTypeInfo import org.junit.Assert._ - -import org.apache.flink.api.common.typeutils.{TypeSerializer, SerializerTestInstance} -import org.apache.flink.api.common.typeinfo.TypeInformation import org.junit.{Assert, Test} -import org.apache.flink.api.scala._ - import scala.util.{Failure, Success} class ScalaSpecialTypesSerializerTest { @@ -67,6 +66,12 @@ class ScalaSpecialTypesSerializerTest { runTests(testData) } + @Test + def testEnumValue(): Unit = { + val testData = Array(WeekDay.Mon, WeekDay.Fri, WeekDay.Tue, WeekDay.Sun, WeekDay.Wed) + runTests(testData) + } + @Test def testTry(): Unit = { val testData = Array(Success("Hell"), Failure(new RuntimeException("test"))) @@ -93,8 +98,11 @@ class ScalaSpecialTypesSerializerTest { val typeInfo = implicitly[TypeInformation[T]] val serializer = typeInfo.createSerializer(new ExecutionConfig) val typeClass = typeInfo.getTypeClass - val test = - new ScalaSpecialTypesSerializerTestInstance[T](serializer, typeClass, -1, instances) + val test = new ScalaSpecialTypesSerializerTestInstance[T]( + serializer, + typeClass, + serializer.getLength, + instances) test.testAll() } catch { case e: Exception => { @@ -159,3 +167,8 @@ class ScalaSpecialTypesSerializerTestInstance[T]( } } +object WeekDay extends Enumeration { + type WeekDay = Value + val Mon, Tue, Wed, Thu, Fri, Sat, Sun = Value +} + From 7bd57d7893ce53c389725a4536afc778dbcc5513 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 29 Jul 2015 01:29:13 +0200 Subject: [PATCH 024/175] [FLINK-2424] [core] Close output stream in serialization utility --- .../java/org/apache/flink/util/InstantiationUtil.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index 241e56af69d12..6857c310151e3 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -311,7 +311,13 @@ public static Object deserializeObject(byte[] bytes, ClassLoader cl) throws IOEx public static byte[] serializeObject(Object o) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); ObjectOutputStream oos = new ObjectOutputStream(baos); - oos.writeObject(o); + + try { + oos.writeObject(o); + } + finally { + oos.close(); + } return baos.toByteArray(); } From 833862a999326a8c1b236af0418c7bd3423c7097 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 29 Jul 2015 14:49:23 +0200 Subject: [PATCH 025/175] [hotfix] Code cleanups in the StreamConfig --- .../streaming/api/graph/StreamConfig.java | 55 ++++++++++++------- .../streaming/api/graph/StreamGraph.java | 8 ++- .../api/graph/StreamingJobGraphGenerator.java | 2 +- 3 files changed, 44 insertions(+), 21 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index d0e806425992b..1562f38b1a593 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -28,6 +28,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.tasks.StreamTaskException; @@ -37,6 +38,10 @@ public class StreamConfig implements Serializable { private static final long serialVersionUID = 1L; + // ------------------------------------------------------------------------ + // Config Keys + // ------------------------------------------------------------------------ + private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs"; private static final String NUMBER_OF_INPUTS = "numberOfInputs"; private static final String CHAINED_OUTPUTS = "chainedOutputs"; @@ -59,16 +64,22 @@ public class StreamConfig implements Serializable { private static final String EDGES_IN_ORDER = "edgesInOrder"; private static final String OUT_STREAM_EDGES = "outStreamEdges"; private static final String IN_STREAM_EDGES = "inStreamEdges"; + + private static final String CHECKPOINTING_ENABLED = "checkpointing"; private static final String STATEHANDLE_PROVIDER = "stateHandleProvider"; private static final String STATE_PARTITIONER = "statePartitioner"; - // DEFAULT VALUES + // ------------------------------------------------------------------------ + // Default Values + // ------------------------------------------------------------------------ + private static final long DEFAULT_TIMEOUT = 100; - public static final String STATE_MONITORING = "STATE_MONITORING"; - // CONFIG METHODS + // ------------------------------------------------------------------------ + // Config + // ------------------------------------------------------------------------ - private Configuration config; + private final Configuration config; public StreamConfig(Configuration config) { this.config = config; @@ -78,6 +89,11 @@ public Configuration getConfiguration() { return config; } + // ------------------------------------------------------------------------ + // Configured Properties + // ------------------------------------------------------------------------ + + public void setVertexID(Integer vertexID) { config.setInteger(VERTEX_NAME, vertexID); } @@ -335,12 +351,12 @@ public List getInPhysicalEdges(ClassLoader cl) { } } - public void setStateMonitoring(boolean stateMonitoring) { - config.setBoolean(STATE_MONITORING, stateMonitoring); + public void setCheckpointingEnabled(boolean enabled) { + config.setBoolean(CHECKPOINTING_ENABLED, enabled); } - public boolean getStateMonitoring() { - return config.getBoolean(STATE_MONITORING, false); + public boolean isCheckpointingEnabled() { + return config.getBoolean(CHECKPOINTING_ENABLED, false); } public void setOutEdgesInOrder(List outEdgeList) { @@ -435,28 +451,29 @@ public String toString() { builder.append("\n======================="); builder.append("Stream Config"); builder.append("======================="); - builder.append("\nTask name: " + getVertexID()); - builder.append("\nNumber of non-chained inputs: " + getNumberOfInputs()); - builder.append("\nNumber of non-chained outputs: " + getNumberOfOutputs()); - builder.append("\nOutput names: " + getNonChainedOutputs(cl)); + builder.append("\nTask name: ").append(getVertexID()); + builder.append("\nNumber of non-chained inputs: ").append(getNumberOfInputs()); + builder.append("\nNumber of non-chained outputs: ").append(getNumberOfOutputs()); + builder.append("\nOutput names: ").append(getNonChainedOutputs(cl)); builder.append("\nPartitioning:"); for (StreamEdge output : getNonChainedOutputs(cl)) { int outputname = output.getTargetId(); - builder.append("\n\t" + outputname + ": " + output.getPartitioner()); + builder.append("\n\t").append(outputname).append(": ").append(output.getPartitioner()); } - builder.append("\nChained subtasks: " + getChainedOutputs(cl)); + builder.append("\nChained subtasks: ").append(getChainedOutputs(cl)); try { - builder.append("\nOperator: " + getStreamOperator(cl).getClass().getSimpleName()); - } catch (Exception e) { + builder.append("\nOperator: ").append(getStreamOperator(cl).getClass().getSimpleName()); + } + catch (Exception e) { builder.append("\nOperator: Missing"); } - builder.append("\nBuffer timeout: " + getBufferTimeout()); - builder.append("\nState Monitoring: " + getStateMonitoring()); + builder.append("\nBuffer timeout: ").append(getBufferTimeout()); + builder.append("\nState Monitoring: ").append(isCheckpointingEnabled()); if (isChainStart() && getChainedOutputs(cl).size() > 0) { builder.append("\n\n\n---------------------\nChained task configs\n---------------------\n"); - builder.append(getTransitiveChainedTaskConfigs(cl)).toString(); + builder.append(getTransitiveChainedTaskConfigs(cl)); } return builder.toString(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index f1428b4ed18b2..4de522406196b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -42,6 +42,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -66,14 +67,19 @@ */ public class StreamGraph extends StreamingPlan { + /** The default interval for checkpoints, in milliseconds */ + public static final int DEFAULT_CHECKPOINTING_INTERVAL_MS = 5000; + private static final Logger LOG = LoggerFactory.getLogger(StreamGraph.class); + private String jobName = StreamExecutionEnvironment.DEFAULT_JOB_NAME; private final StreamExecutionEnvironment environemnt; private final ExecutionConfig executionConfig; + private CheckpointingMode checkpointingMode; private boolean checkpointingEnabled = false; - private long checkpointingInterval = 5000; + private long checkpointingInterval = DEFAULT_CHECKPOINTING_INTERVAL_MS; private boolean chaining = true; private Map streamNodes; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index c988150181941..5551bf32c6bb8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -269,7 +269,7 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setNumberOfOutputs(nonChainableOutputs.size()); config.setNonChainedOutputs(nonChainableOutputs); config.setChainedOutputs(chainableOutputs); - config.setStateMonitoring(streamGraph.isCheckpointingEnabled()); + config.setCheckpointingEnabled(streamGraph.isCheckpointingEnabled()); config.setStateHandleProvider(streamGraph.getStateHandleProvider()); config.setStatePartitioner((KeySelector) vertex.getStatePartitioner()); From b211a62111aa3c558586874d0ec5b168e6bb31f1 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 29 Jul 2015 14:12:42 +0200 Subject: [PATCH 026/175] [FLINK-2407] [streaming] Add an API switch to choose between "exactly once" and "at least once". --- .../streaming/api/CheckpointingMode.java | 75 +++++++++++ .../StreamExecutionEnvironment.java | 125 ++++++++++++------ .../streaming/api/graph/StreamConfig.java | 24 +++- .../streaming/api/graph/StreamGraph.java | 11 ++ .../api/graph/StreamingJobGraphGenerator.java | 12 +- .../runtime/io/StreamInputProcessor.java | 13 +- .../runtime/io/StreamTwoInputProcessor.java | 14 +- .../runtime/tasks/OneInputStreamTask.java | 1 + .../runtime/tasks/TwoInputStreamTask.java | 1 + .../flink/streaming/api/IterateTest.java | 6 +- .../streaming/graph/TranslationTest.java | 74 +++++++++++ .../scala/StreamExecutionEnvironment.scala | 90 +++++++++---- 12 files changed, 375 insertions(+), 71 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java new file mode 100644 index 0000000000000..db46d007c85a0 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java @@ -0,0 +1,75 @@ +/* + * 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.streaming.api; + +/** + * The checkpointing mode defines what consistency guarantees the system gives in the presence of + * failures. + * + *

When checkpointing is activated, the data streams are replayed such that lost parts of the + * processing are repeated. For stateful operations and functions, the checkpointing mode defines + * whether the system draws checkpoints such that a recovery behaves as if the operators/functions + * see each record "exactly once" ({@link #EXACTLY_ONCE}), or whether the checkpoints are drawn + * in a simpler fashion that typically encounteres some duplicates upon recovery + * ({@link #AT_LEAST_ONCE})

+ */ +public enum CheckpointingMode { + + /** + * Sets the checkpointing mode to "exactly once". This mode means that the system will + * checkpoint the operator and user function state in such a way that, upon recovery, + * every record will be reflected exactly once in the operator state. + * + *

For example, if a user function counts the number of elements in a stream, + * this number will consistently be equal to the number of actual elements in the stream, + * regardless of failures and recovery.

+ * + *

Note that this does not mean that each record flows through the streaming data flow + * only once. It means that upon recovery, the state of operators/functions is restored such + * that the resumed data streams pick up exactly at after the last modification to the state.

+ * + *

Note that this mode does not guarantee exactly-once behavior in the interaction with + * external systems (only state in Flink's operators and user functions). The reason for that + * is that a certain level of "collaboration" is required between two systems to achieve + * exactly-once guarantees. However, for certain systems, connectors can be written that facilitate + * this collaboration.

+ * + *

This mode sustains high throughput. Depending on the data flow graph and operations, + * this mode may increase the record latency, because operators need to align their input + * streams, in order to create a consistent snapshot point. The latency increase for simple + * dataflows (no repartitioning) is negligible. For simple dataflows with repartitioning, the average + * latency remains small, but the slowest records typically have an increased latency.

+ */ + EXACTLY_ONCE, + + /** + * Sets the checkpointing mode to "at least once". This mode means that the system will + * checkpoint the operator and user function state in a simpler way. Upon failure and recovery, + * some records may be reflected multiple times in the operator state. + * + *

For example, if a user function counts the number of elements in a stream, + * this number will equal to, or larger, than the actual number of elements in the stream, + * in the presence of failure and recovery.

+ * + *

This mode has minimal impact on latency and may be preferable in very-low latency + * scenarios, where a sustained very-low latency (such as few milliseconds) is needed, + * and where occasional duplicate messages (on recovery) do not matter.

+ */ + AT_LEAST_ONCE +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 58348e38d8e95..fba4e28d85096 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -45,6 +45,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.FileStateHandle; import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction; @@ -224,51 +225,94 @@ public StreamExecutionEnvironment disableOperatorChaining() { return this; } + // ------------------------------------------------------------------------ + // Checkpointing Settings + // ------------------------------------------------------------------------ + /** - * Method for enabling fault-tolerance. Activates monitoring and backup of - * streaming operator states. - *

- *

- * Setting this option assumes that the job is used in production and thus - * if not stated explicitly otherwise with calling with the - * {@link #setNumberOfExecutionRetries(int numberOfExecutionRetries)} method - * in case of failure the job will be resubmitted to the cluster - * indefinitely. + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. This method selects + * {@link CheckpointingMode#EXACTLY_ONCE} guarantees. + * + *

The job draws checkpoints periodically, in the given interval. The state will be + * stored in the configured state backend.

+ * + *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. For that reason, iterative jobs will not be started if used + * with enabled checkpointing. To override this mechanism, use the + * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.

* - * @param interval - * Time interval between state checkpoints in millis + * @param interval Time interval between state checkpoints in milliseconds. */ public StreamExecutionEnvironment enableCheckpointing(long interval) { + return enableCheckpointing(interval, CheckpointingMode.EXACTLY_ONCE); + } + + /** + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. + * + *

The job draws checkpoints periodically, in the given interval. The system uses the + * given {@link CheckpointingMode} for the checkpointing ("exactly once" vs "at least once"). + * The state will be stored in the configured state backend.

+ * + *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. For that reason, iterative jobs will not be started if used + * with enabled checkpointing. To override this mechanism, use the + * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.

+ * + * @param interval + * Time interval between state checkpoints in milliseconds. + * @param mode + * The checkpointing mode, selecting between "exactly once" and "at least once" guaranteed. + */ + public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode) { + if (mode == null) { + throw new NullPointerException("checkpoint mode must not be null"); + } + if (interval <= 0) { + throw new IllegalArgumentException("the checkpoint interval must be positive"); + } + streamGraph.setCheckpointingEnabled(true); streamGraph.setCheckpointingInterval(interval); + streamGraph.setCheckpointingMode(mode); return this; } /** - * Method for force-enabling fault-tolerance. Activates monitoring and - * backup of streaming operator states even for jobs containing iterations. - * - * Please note that the checkpoint/restore guarantees for iterative jobs are - * only best-effort at the moment. Records inside the loops may be lost - * during failure. - *

- *

- * Setting this option assumes that the job is used in production and thus - * if not stated explicitly otherwise with calling with the - * {@link #setNumberOfExecutionRetries(int numberOfExecutionRetries)} method - * in case of failure the job will be resubmitted to the cluster - * indefinitely. + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. + * + *

The job draws checkpoints periodically, in the given interval. The state will be + * stored in the configured state backend.

+ * + *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. If the "force" parameter is set to true, the system will execute the + * job nonetheless.

* * @param interval - * Time interval between state checkpoints in millis + * Time interval between state checkpoints in millis. + * @param mode + * The checkpointing mode, selecting between "exactly once" and "at least once" guaranteed. * @param force - * If true checkpointing will be enabled for iterative jobs as - * well + * If true checkpointing will be enabled for iterative jobs as well. */ @Deprecated - public StreamExecutionEnvironment enableCheckpointing(long interval, boolean force) { + public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode, boolean force) { + if (mode == null) { + throw new NullPointerException("checkpoint mode must not be null"); + } + if (interval <= 0) { + throw new IllegalArgumentException("the checkpoint interval must be positive"); + } + streamGraph.setCheckpointingEnabled(true); streamGraph.setCheckpointingInterval(interval); + streamGraph.setCheckpointingMode(mode); if (force) { streamGraph.forceCheckpoint(); } @@ -276,18 +320,22 @@ public StreamExecutionEnvironment enableCheckpointing(long interval, boolean for } /** - * Method for enabling fault-tolerance. Activates monitoring and backup of - * streaming operator states. - *

- *

- * Setting this option assumes that the job is used in production and thus - * if not stated explicitly otherwise with calling with the - * {@link #setNumberOfExecutionRetries(int numberOfExecutionRetries)} method - * in case of failure the job will be resubmitted to the cluster - * indefinitely. + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. This method selects + * {@link CheckpointingMode#EXACTLY_ONCE} guarantees. + * + *

The job draws checkpoints periodically, in the default interval. The state will be + * stored in the configured state backend.

+ * + *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. For that reason, iterative jobs will not be started if used + * with enabled checkpointing. To override this mechanism, use the + * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.

*/ public StreamExecutionEnvironment enableCheckpointing() { streamGraph.setCheckpointingEnabled(true); + streamGraph.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); return this; } @@ -323,8 +371,7 @@ public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { * A value of {@code -1} indicates that the system default value (as defined * in the configuration) should be used. * - * @return The number of times the system will try to re-execute failed - * tasks. + * @return The number of times the system will try to re-execute failed tasks. */ public int getNumberOfExecutionRetries() { return config.getNumberOfExecutionRetries(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 1562f38b1a593..62735afcc7256 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -68,13 +68,17 @@ public class StreamConfig implements Serializable { private static final String CHECKPOINTING_ENABLED = "checkpointing"; private static final String STATEHANDLE_PROVIDER = "stateHandleProvider"; private static final String STATE_PARTITIONER = "statePartitioner"; - + private static final String CHECKPOINT_MODE = "checkpointMode"; + + // ------------------------------------------------------------------------ // Default Values // ------------------------------------------------------------------------ private static final long DEFAULT_TIMEOUT = 100; - + private static final CheckpointingMode DEFAULT_CHECKPOINTING_MODE = CheckpointingMode.EXACTLY_ONCE; + + // ------------------------------------------------------------------------ // Config // ------------------------------------------------------------------------ @@ -351,6 +355,8 @@ public List getInPhysicalEdges(ClassLoader cl) { } } + // --------------------- checkpointing ----------------------- + public void setCheckpointingEnabled(boolean enabled) { config.setBoolean(CHECKPOINTING_ENABLED, enabled); } @@ -358,6 +364,20 @@ public void setCheckpointingEnabled(boolean enabled) { public boolean isCheckpointingEnabled() { return config.getBoolean(CHECKPOINTING_ENABLED, false); } + + public void setCheckpointMode(CheckpointingMode mode) { + config.setInteger(CHECKPOINT_MODE, mode.ordinal()); + } + + public CheckpointingMode getCheckpointMode() { + int ordinal = config.getInteger(CHECKPOINT_MODE, -1); + if (ordinal >= 0) { + return CheckpointingMode.values()[ordinal]; + } else { + return DEFAULT_CHECKPOINTING_MODE; + } + } + public void setOutEdgesInOrder(List outEdgeList) { try { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index 4de522406196b..65736f534b3b9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -147,6 +147,8 @@ public long getCheckpointingInterval() { return checkpointingInterval; } + // Checkpointing + public boolean isChainingEnabled() { return chaining; } @@ -155,6 +157,15 @@ public boolean isCheckpointingEnabled() { return checkpointingEnabled; } + public CheckpointingMode getCheckpointingMode() { + return checkpointingMode; + } + + public void setCheckpointingMode(CheckpointingMode checkpointingMode) { + this.checkpointingMode = checkpointingMode; + } + + public boolean isIterative() { return !streamLoops.isEmpty(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 5551bf32c6bb8..5280fb27c638d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; @@ -269,10 +270,19 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setNumberOfOutputs(nonChainableOutputs.size()); config.setNonChainedOutputs(nonChainableOutputs); config.setChainedOutputs(chainableOutputs); + config.setCheckpointingEnabled(streamGraph.isCheckpointingEnabled()); - config.setStateHandleProvider(streamGraph.getStateHandleProvider()); + if (streamGraph.isCheckpointingEnabled()) { + config.setCheckpointMode(streamGraph.getCheckpointingMode()); + config.setStateHandleProvider(streamGraph.getStateHandleProvider()); + } else { + // the at least once input handler is slightly cheaper (in the absence of checkpoints), + // so we use that one if checkpointing is not enabled + config.setCheckpointMode(CheckpointingMode.AT_LEAST_ONCE); + } config.setStatePartitioner((KeySelector) vertex.getStatePartitioner()); + Class vertexClass = vertex.getJobVertexClass(); if (vertexClass.equals(StreamIterationHead.class) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 4d60375173ac3..f7d7fb0bff0ed 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; @@ -79,12 +80,22 @@ public class StreamInputProcessor extends AbstractReader implements ReaderBa @SuppressWarnings("unchecked") public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSerializer, EventListener checkpointListener, + CheckpointingMode checkpointMode, IOManager ioManager, boolean enableWatermarkMultiplexing) throws IOException { super(InputGateUtil.createInputGate(inputGates)); - this.barrierHandler = new BarrierBuffer(inputGate, ioManager); + if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) { + this.barrierHandler = new BarrierBuffer(inputGate, ioManager); + } + else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) { + this.barrierHandler = new BarrierTracker(inputGate); + } + else { + throw new IllegalArgumentException("Unrecognized CheckpointingMode: " + checkpointMode); + } + if (checkpointListener != null) { this.barrierHandler.registerCheckpointEventHandler(checkpointListener); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 9668c7f310dca..ae97974c9558b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; @@ -90,16 +91,25 @@ public StreamTwoInputProcessor( TypeSerializer inputSerializer1, TypeSerializer inputSerializer2, EventListener checkpointListener, + CheckpointingMode checkpointMode, IOManager ioManager, boolean enableWatermarkMultiplexing) throws IOException { super(InputGateUtil.createInputGate(inputGates1, inputGates2)); - this.barrierHandler = new BarrierBuffer(inputGate, ioManager); + if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) { + this.barrierHandler = new BarrierBuffer(inputGate, ioManager); + } + else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) { + this.barrierHandler = new BarrierTracker(inputGate); + } + else { + throw new IllegalArgumentException("Unrecognized CheckpointingMode: " + checkpointMode); + } + if (checkpointListener != null) { this.barrierHandler.registerCheckpointEventHandler(checkpointListener); } - if (enableWatermarkMultiplexing) { MultiplexingStreamRecordSerializer ser = new MultiplexingStreamRecordSerializer(inputSerializer1); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index d0783206feb03..605b8f5cfeb1a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -44,6 +44,7 @@ public void registerInputOutput() { InputGate[] inputGates = getEnvironment().getAllInputGates(); inputProcessor = new StreamInputProcessor(inputGates, inSerializer, getCheckpointBarrierListener(), + configuration.getCheckpointMode(), getEnvironment().getIOManager(), getExecutionConfig().areTimestampsEnabled()); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index b4667b2b90307..99c053b124271 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -69,6 +69,7 @@ public void registerInputOutput() { this.inputProcessor = new StreamTwoInputProcessor(inputList1, inputList2, inputDeserializer1, inputDeserializer2, getCheckpointBarrierListener(), + configuration.getCheckpointMode(), getEnvironment().getIOManager(), getExecutionConfig().areTimestampsEnabled()); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java index 2a88a3225a456..0fad3dd14895b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java @@ -27,7 +27,6 @@ import java.util.Collections; import java.util.List; -import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.java.functions.KeySelector; @@ -351,6 +350,7 @@ public void testCoIteration() throws Exception { coIt.groupBy(1, 2); fail(); } catch (UnsupportedOperationException e) { + // this is expected } DataStream head = coIt @@ -479,7 +479,7 @@ public void testWithCheckPointing() throws Exception { // Test force checkpointing try { - env.enableCheckpointing(1, false); + env.enableCheckpointing(1, CheckpointingMode.EXACTLY_ONCE, false); env.execute(); // this statement should never be reached @@ -488,7 +488,7 @@ public void testWithCheckPointing() throws Exception { // expected behaviour } - env.enableCheckpointing(1, true); + env.enableCheckpointing(1, CheckpointingMode.EXACTLY_ONCE, true); env.getStreamGraph().getJobGraph(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java new file mode 100644 index 0000000000000..89679ea12bd31 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java @@ -0,0 +1,74 @@ +/* + * 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.streaming.graph; + +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.graph.StreamConfig; + +import org.junit.Test; + +import static org.junit.Assert.*; + +@SuppressWarnings("serial") +public class TranslationTest { + + @Test + public void testCheckpointModeTranslation() { + try { + // with deactivated fault tolerance, the checkpoint mode should be at-least-once + StreamExecutionEnvironment deactivated = getSimpleJob(); + + for (JobVertex vertex : deactivated.getStreamGraph().getJobGraph().getVertices()) { + assertEquals(CheckpointingMode.AT_LEAST_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode()); + } + + // with activated fault tolerance, the checkpoint mode should be by default exactly once + StreamExecutionEnvironment activated = getSimpleJob(); + activated.enableCheckpointing(1000L); + for (JobVertex vertex : activated.getStreamGraph().getJobGraph().getVertices()) { + assertEquals(CheckpointingMode.EXACTLY_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode()); + } + + // explicitly setting the mode + StreamExecutionEnvironment explicit = getSimpleJob(); + explicit.enableCheckpointing(1000L, CheckpointingMode.AT_LEAST_ONCE); + for (JobVertex vertex : explicit.getStreamGraph().getJobGraph().getVertices()) { + assertEquals(CheckpointingMode.AT_LEAST_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode()); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + private static StreamExecutionEnvironment getSimpleJob() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.generateSequence(1, 10000000) + .addSink(new SinkFunction() { + @Override + public void invoke(Long value) {} + }); + + return env; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index 70e652f3edec9..e983451dd4762 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer import org.apache.flink.api.scala.ClosureCleaner import org.apache.flink.runtime.state.StateHandleProvider +import org.apache.flink.streaming.api.CheckpointingMode import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaEnv} import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext @@ -112,37 +113,80 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { this } + // ------------------------------------------------------------------------ + // Checkpointing Settings + // ------------------------------------------------------------------------ /** - * Method for enabling fault-tolerance. Activates monitoring and backup of streaming - * operator states. Time interval between state checkpoints is specified in in millis. - * - * If the force flag is set to true, checkpointing will be enabled for iterative jobs as - * well.Please note that the checkpoint/restore guarantees for iterative jobs are - * only best-effort at the moment. Records inside the loops may be lost during failure. + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. * - * Setting this option assumes that the job is used in production and thus if not stated - * explicitly otherwise with calling with the - * [[setNumberOfExecutionRetries(int)]] method in case of - * failure the job will be resubmitted to the cluster indefinitely. + * The job draws checkpoints periodically, in the given interval. The state will be + * stored in the configured state backend. + * + * NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. If the "force" parameter is set to true, the system will execute the + * job nonetheless. + * + * @param interval + * Time interval between state checkpoints in millis. + * @param mode + * The checkpointing mode, selecting between "exactly once" and "at least once" guarantees. + * @param force + * If true checkpointing will be enabled for iterative jobs as well. */ @deprecated - def enableCheckpointing(interval : Long, force: Boolean) : StreamExecutionEnvironment = { - javaEnv.enableCheckpointing(interval, force) + def enableCheckpointing(interval : Long, + mode: CheckpointingMode, + force: Boolean) : StreamExecutionEnvironment = { + javaEnv.enableCheckpointing(interval, mode, force) this } - - /** - * Method for enabling fault-tolerance. Activates monitoring and backup of streaming - * operator states. Time interval between state checkpoints is specified in in millis. - * - * Setting this option assumes that the job is used in production and thus if not stated - * explicitly otherwise with calling with the - * [[setNumberOfExecutionRetries(int)]] method in case of - * failure the job will be resubmitted to the cluster indefinitely. + + /** + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. + * + * The job draws checkpoints periodically, in the given interval. The system uses the + * given [[CheckpointingMode]] for the checkpointing ("exactly once" vs "at least once"). + * The state will be stored in the configured state backend. + * + * NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. For that reason, iterative jobs will not be started if used + * with enabled checkpointing. To override this mechanism, use the + * [[enableCheckpointing(long, CheckpointingMode, boolean)]] method. + * + * @param interval + * Time interval between state checkpoints in milliseconds. + * @param mode + * The checkpointing mode, selecting between "exactly once" and "at least once" guarantees. + */ + def enableCheckpointing(interval : Long, + mode: CheckpointingMode) : StreamExecutionEnvironment = { + javaEnv.enableCheckpointing(interval, mode) + this + } + + /** + * Enables checkpointing for the streaming job. The distributed state of the streaming + * dataflow will be periodically snapshotted. In case of a failure, the streaming + * dataflow will be restarted from the latest completed checkpoint. + * + * The job draws checkpoints periodically, in the given interval. The program will use + * [[CheckpointingMode.EXACTLY_ONCE]] mode. The state will be stored in the + * configured state backend. + * + * NOTE: Checkpointing iterative streaming dataflows in not properly supported at + * the moment. For that reason, iterative jobs will not be started if used + * with enabled checkpointing. To override this mechanism, use the + * [[enableCheckpointing(long, CheckpointingMode, boolean)]] method. + * + * @param interval + * Time interval between state checkpoints in milliseconds. */ def enableCheckpointing(interval : Long) : StreamExecutionEnvironment = { - javaEnv.enableCheckpointing(interval) - this + enableCheckpointing(interval, CheckpointingMode.EXACTLY_ONCE) } /** From 73af891141f453d607104e28d3a0e0a3438a409b Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Tue, 28 Jul 2015 22:01:32 +0200 Subject: [PATCH 027/175] [FLINK-2419] Add test for sinks after keyBy and groupBy Closes #947 --- .../flink/streaming/api/DataStreamTest.java | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index 764c6f2d38f19..324143f04eb3b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -460,6 +460,48 @@ public String map2(Integer value) { fail(e.getMessage()); } } + + @Test + public void sinkKeyTest() { + StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); + StreamGraph streamGraph = env.getStreamGraph(); + + DataStream sink = env.generateSequence(1, 100).print(); + assertTrue(streamGraph.getStreamNode(sink.getId()).getStatePartitioner() == null); + assertTrue(streamGraph.getStreamNode(sink.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner); + + KeySelector key1 = new KeySelector() { + + private static final long serialVersionUID = 1L; + + @Override + public Long getKey(Long value) throws Exception { + return (long) 0; + } + }; + + DataStream sink2 = env.generateSequence(1, 100).keyBy(key1).print(); + + assertTrue(streamGraph.getStreamNode(sink2.getId()).getStatePartitioner() != null); + assertEquals(key1, streamGraph.getStreamNode(sink2.getId()).getStatePartitioner()); + assertTrue(streamGraph.getStreamNode(sink2.getId()).getInEdges().get(0).getPartitioner() instanceof FieldsPartitioner); + + KeySelector key2 = new KeySelector() { + + private static final long serialVersionUID = 1L; + + @Override + public Long getKey(Long value) throws Exception { + return (long) 0; + } + }; + + DataStream sink3 = env.generateSequence(1, 100).keyBy(key2).print(); + + assertTrue(streamGraph.getStreamNode(sink3.getId()).getStatePartitioner() != null); + assertEquals(key2, streamGraph.getStreamNode(sink3.getId()).getStatePartitioner()); + assertTrue(streamGraph.getStreamNode(sink3.getId()).getInEdges().get(0).getPartitioner() instanceof FieldsPartitioner); + } @Test public void testChannelSelectors() { From e5aacbe67ca8498007f23196cdebe652f1a0327a Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 29 Jul 2015 22:34:20 +0300 Subject: [PATCH 028/175] Rearranging the client code. --- .../accumulators/LargeAccumulatorHelper.java | 124 ++++++++++++++++-- .../LargeAccumulatorSnapshot.java | 2 +- .../flink/runtime/client/JobClient.java | 82 ++---------- .../client/SerializedJobExecutionResult.java | 12 +- 4 files changed, 128 insertions(+), 92 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java index 7d9ca8af0c8ef..ccd99b32b8fa9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java @@ -19,11 +19,15 @@ package org.apache.flink.runtime.accumulators; import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.runtime.blob.BlobClient; import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.blob.BlobServerProtocol; import org.apache.flink.runtime.util.SerializedValue; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collections; @@ -34,16 +38,20 @@ public class LargeAccumulatorHelper { /** - * Puts the blobs of the large accumulators on the BlobCache. + * Serializes and Puts the blobs of the large accumulators on the BlobCache. + * * @param blobServerAddress the address of the server to the blobCache. - * @param accumulators the accumulators to be stored in the cache. + * @param accumulators the accumulators to be stored in the cache. * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. - * */ - public static Map> storeAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, - Map> accumulators) throws IOException { + */ + public static Map> storeAccumulatorsToBlobCache( + InetSocketAddress blobServerAddress, + Map> accumulators) throws IOException { + if (blobServerAddress == null) { throw new RuntimeException("Undefined Blob Server Address."); } + if (accumulators.isEmpty()) { return Collections.emptyMap(); } @@ -54,12 +62,13 @@ public static Map> storeAccumulatorsToBlobCache(InetSocket /** * Puts the blobs of the large accumulators on the BlobCache. + * * @param blobServerAddress the address of the server to the blobCache. - * @param accumulators the accumulators to be stored in the cache. + * @param accumulators the accumulators to be stored in the cache. * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. - * */ + */ public static Map> storeSerializedAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, - Map> accumulators) throws IOException { + Map> accumulators) throws IOException { if (blobServerAddress == null) { throw new RuntimeException("Undefined Blob Server Address."); } @@ -77,7 +86,7 @@ public static Map> storeSerializedAccumulatorsToBlobCache( String accumulatorName = entry.getKey(); byte[] accumulatorPayload = entry.getValue().getSerializedData(); - if(accumulatorPayload != null) { + if (accumulatorPayload != null) { BlobKey blobKey = bc.put(accumulatorPayload); List accKeys = keys.get(accumulatorName); if (accKeys == null) { @@ -91,7 +100,7 @@ public static Map> storeSerializedAccumulatorsToBlobCache( throw new IOException("Failed to send oversized accumulators to the BlobCache: ", e); } finally { try { - if(bc != null) { + if (bc != null) { bc.close(); } } catch (IOException e) { @@ -102,6 +111,101 @@ public static Map> storeSerializedAccumulatorsToBlobCache( return keys; } + /** + * When the result of the job contains oversized (i.e. bigger that the akka.framesize) accumulators + * then these are put in the BlobCache for the client to fetch and merge. This method gets, deserializes, + * and merges these user-defined accumulators values. + * + * @param blobServerAddress the address that the BlobCache is listening to. + * @param keys the blob keys to fetch. + * @param loader the classloader used to deserialize the accumulators fetched. + * @return the accumulators, grouped by name. + * */ + public static Map> getDeserializeAndMergeAccumulatorsFromBlobCache( + InetSocketAddress blobServerAddress, Map> keys, ClassLoader loader) + throws IOException, ClassNotFoundException { + + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + + if (keys.isEmpty()) { + return Collections.emptyMap(); + } + + Map> accumulators = + new HashMap>(); + + Map>> accumulatorBlobs = + getSerializedAccumulatorsFromBlobCache(blobServerAddress, keys); + + for (String accumulatorName : accumulatorBlobs.keySet()) { + Accumulator existing = accumulators.get(accumulatorName); + + for (SerializedValue acc : accumulatorBlobs.get(accumulatorName)) { + Accumulator accumulator = (Accumulator) acc.deserializeValue(loader); + if(existing == null) { + existing = accumulator; + accumulators.put(accumulatorName, existing); + } else { + AccumulatorHelper.mergeAccumulators(accumulatorName, existing, accumulator); + } + } + } + return accumulators; + } + + /** + * Gets the user-defined accumulators values. + * + * @return the serialized map + */ + public static Map>> getSerializedAccumulatorsFromBlobCache( + InetSocketAddress blobServerAddress, Map> keys) throws IOException { + + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + + if (keys.isEmpty()) { + return Collections.emptyMap(); + } + + Map>> accumulatorBlobs = + new HashMap>>(); + + BlobClient bc = new BlobClient(blobServerAddress); + + final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE]; + for (String accName : keys.keySet()) { + List accBlobKeys = keys.get(accName); + List> accBlobs = new ArrayList>(); + + for (BlobKey bk : accBlobKeys) { + InputStream is = bc.get(bk); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + while (true) { + final int read = is.read(buf); + if (read < 0) { + break; + } + os.write(buf, 0, read); + } + os.flush(); + byte[] blob = os.toByteArray(); + accBlobs.add(new SerializedValue(blob)); + is.close(); + os.close(); + + // after getting them, clean up and delete the blobs from the BlobCache. + bc.delete(bk); + } + accumulatorBlobs.put(accName, accBlobs); + } + bc.close(); + return accumulatorBlobs; + } + private static Map> serializeAccumulators(Map> accumulators) throws IOException { if (accumulators.isEmpty()) { return Collections.emptyMap(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java index 823a03c2e3a08..fa6d6f87ab4b8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java @@ -30,7 +30,7 @@ public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { /** - * In case some accumulators do not fit in akka, we store them in the blobCache and put + * In case some accumulators do not fit in an Akka message payload, we store them in the blobCache and put * in the snapshot only the mapping between the name of the accumulator, and its blobKey * in the cache. This list holds exactly this mapping. * */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index 3b64b8293b96a..c7020d96cfcb6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -29,18 +29,17 @@ import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.accumulators.LargeAccumulatorHelper; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.blob.BlobClient; import org.apache.flink.runtime.blob.BlobKey; -import org.apache.flink.runtime.blob.BlobServerProtocol; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.JobClientMessages; import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,15 +50,10 @@ import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; -import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.InputStream; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeoutException; @@ -208,15 +202,18 @@ public static JobExecutionResult returnFinalJobExecutionResult( throw new NullPointerException(); } - Map> blobsToFetch = partialResult.getBlobKeysToLargeAccumulators(); - - Map>> accumulatorBlobs; + Map> largeAccumulators; try { - accumulatorBlobs = getLargeAccumulatorBlobs(jobManagerGateway, blobsToFetch, timeout); + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + Map> blobsKeysToFetch = partialResult.getBlobKeysToLargeAccumulators(); + + largeAccumulators = LargeAccumulatorHelper.getDeserializeAndMergeAccumulatorsFromBlobCache( + serverAddress, blobsKeysToFetch, userCodeClassLoader); + } catch (IOException e) { throw new IOException("Failed to fetch the oversized accumulators from the BlobCache", e); } - return partialResult.mergeToJobExecutionResult(userCodeClassLoader, accumulatorBlobs); + return partialResult.mergeToJobExecutionResult(userCodeClassLoader, largeAccumulators); } /** @@ -287,63 +284,7 @@ public static void uploadJarFiles( } } - /** - * If the result of the job contained oversized (i.e. bigger that the akka.framesize) accumulators - * then these are put in the BlobCache for the client to fetch and merge. This method gets - * them from the BlobCache (if there was any). If the list of blobs to fetch is empty, then - * an empty result is returned, as all (partial) accumulators were small enough to be sent - * directly to the JobManager and be merged there. - * - * @param jobManagerGateway the reference to the jobManager actor. - * @param keys the accumulators to fetch (based on their name) along with their associated BlobKeys. - * @param timeout the timeout to wait for the connection to the blob server. - * @return the serialized accumulators, grouped by name. - * */ - private static Map>> getLargeAccumulatorBlobs( - ActorGateway jobManagerGateway, Map> keys, FiniteDuration timeout) throws IOException { - - if (keys.isEmpty()) { - return Collections.emptyMap(); - } - - Map>> accumulatorBlobs = - new HashMap>>(); - - InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); - BlobClient bc = new BlobClient(serverAddress); - - final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE]; - for(String accName: keys.keySet()) { - List accBlobKeys = keys.get(accName); - List> accBlobs = new ArrayList>(); - - for(BlobKey bk: accBlobKeys) { - InputStream is = bc.get(bk); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - while (true) { - final int read = is.read(buf); - if (read < 0) { - break; - } - os.write(buf, 0, read); - } - os.flush(); - byte[] blob = os.toByteArray(); - accBlobs.add(new SerializedValue(blob)); - is.close(); - os.close(); - - // after getting them, clean up and delete the blobs from the BlobCache. - bc.delete(bk); - } - accumulatorBlobs.put(accName, accBlobs); - } - bc.close(); - return accumulatorBlobs; - } - private static InetSocketAddress getBlobServerAddress(ActorGateway jobManagerGateway, FiniteDuration timeout) throws IOException { - Timeout tOut = new Timeout(timeout); Future futureBlobPort = jobManagerGateway.ask( JobManagerMessages.getRequestBlobManagerPort(), timeout); @@ -362,7 +303,6 @@ private static InetSocketAddress getBlobServerAddress(ActorGateway jobManagerGat Option jmHost = jobManagerGateway.actor().path().address().host(); String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; - InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, port); - return serverAddress; + return new InetSocketAddress(jmHostname, port); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java index 0b1115dbe326a..d8b88e83a20cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.util.SerializedValue; @@ -131,7 +130,7 @@ public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOExce * the Accumulators received from the BlobCache as blobs, that are to be merged with local data. * @return the final result after the merging of the different partial accumulators. * */ - public JobExecutionResult mergeToJobExecutionResult(ClassLoader loader, Map>> accumulatorsToMerge) throws IOException, ClassNotFoundException { + public JobExecutionResult mergeToJobExecutionResult(ClassLoader loader, Map> accumulatorsToMerge) throws IOException, ClassNotFoundException { if(accumulatorsToMerge == null || accumulatorsToMerge.isEmpty()) { return toJobExecutionResult(loader); } @@ -139,14 +138,7 @@ public JobExecutionResult mergeToJobExecutionResult(ClassLoader loader, Map accumulators = new HashMap(); for(String name: accumulatorsToMerge.keySet()) { - List> blobs = accumulatorsToMerge.get(name); - - // merge the serialized accumulators - Accumulator acc = (Accumulator) blobs.get(0).deserializeValue(loader); - for(int i = 1; i < blobs.size(); i++) { - AccumulatorHelper.mergeAccumulators(name, acc, - (Accumulator) blobs.get(i).deserializeValue(loader)); - } + Accumulator acc = accumulatorsToMerge.get(name); // add also the data from the non-oversized (i.e. the ones that were sent through akka) // accumulators, if any From 11de39d10118a9cdc1e3a3b287c170100ed7b0c9 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 29 Jul 2015 22:59:54 +0300 Subject: [PATCH 029/175] Changed the javadoc in some classes. --- .../accumulators/BaseAccumulatorSnapshot.java | 10 +++++----- .../accumulators/LargeAccumulatorSnapshot.java | 16 ++++++++-------- .../accumulators/SmallAccumulatorSnapshot.java | 9 +++++---- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java index 8c4faf03bc358..4962452bce8ad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java @@ -27,6 +27,11 @@ import java.io.Serializable; import java.util.Map; +/** + * This class and its subclasses ({@link SmallAccumulatorSnapshot} and {@link LargeAccumulatorSnapshot}) + * encapsulate a map of accumulators (user- and system- defined)for a single task. It is used for the + * transfer from TaskManagers to the JobManager and from the JobManager to the Client. + */ public class BaseAccumulatorSnapshot implements Serializable { private static final long serialVersionUID = 42L; @@ -37,11 +42,6 @@ public class BaseAccumulatorSnapshot implements Serializable { /** Flink internal accumulators which can be deserialized using the system class loader. */ private final SerializedValue>> flinkAccumulators; - /** - * This class and its subclasses encapsulate a map of accumulators for a single - * task. It is used for the transfer from TaskManagers to the JobManager and from - * the JobManager to the Client. - */ public BaseAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, Map> flinkAccumulators) throws IOException { this.jobID = jobID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java index fa6d6f87ab4b8..1710e6d6b621e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java @@ -27,6 +27,14 @@ import java.util.List; import java.util.Map; +/** + * In case some user-defined accumulators do not fit in an Akka message payload, we store them in the + * blobCache, and put in the snapshot only the mapping between the name of the accumulator, + * and its blobKey in the cache. This clase is a subclass of the BaseAccumulatorSnapshot + * and holds the (potential) references to blobs stored in the BlobCache and containing + * these oversized accumulators. It is used for the transfer from TaskManagers to the + * JobManager and from the JobManager to the Client. + */ public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { /** @@ -36,14 +44,6 @@ public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { * */ private final Map> largeUserAccumulatorBlobs; - /** - * In case some user-defined accumulators do not fit in an Akka message payload, we store them in the - * blobCache, and put in the snapshot only the mapping between the name of the accumulator, - * and its blobKey in the cache. This clase is a subclass of the BaseAccumulatorSnapshot - * and holds the (potential) references to blobs stored in the BlobCache and containing - * these oversized accumulators. It is used for the transfer from TaskManagers to the - * JobManager and from the JobManager to the Client. - */ public LargeAccumulatorSnapshot( JobID jobID, ExecutionAttemptID executionAttemptID, Map> flinkAccumulators, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java index f2a381ce72565..d8dd2cb94f49a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java @@ -26,6 +26,11 @@ import java.io.IOException; import java.util.Map; +/** + * This is a subclass of the BaseAccumulatorSnapshot that serves at storing the task user-defined + * accumulators that are small enough to be sent to the JobManager using akka. It is used for the + * transfer from TaskManagers to the JobManager and from the JobManager to the Client. + * */ public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { /** @@ -33,10 +38,6 @@ public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { */ private final SerializedValue>> userAccumulators; - /** - * This is a subclass of the BaseAccumulatorSnapshot that serves at storing the task user-defined - * accumulators that are small enough to be sent to the JobManager using akka. - * */ public SmallAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, Map> flinkAccumulators, SerializedValue>> userAccumulators) throws IOException { From 0b4a5394569237eead9f29d8abe93690c18a1eb0 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 29 Jul 2015 23:02:35 +0300 Subject: [PATCH 030/175] . --- .../flink/runtime/accumulators/BaseAccumulatorSnapshot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java index 4962452bce8ad..758525ddaff50 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java @@ -29,7 +29,7 @@ /** * This class and its subclasses ({@link SmallAccumulatorSnapshot} and {@link LargeAccumulatorSnapshot}) - * encapsulate a map of accumulators (user- and system- defined)for a single task. It is used for the + * encapsulate a map of accumulators (user- and system- defined) for a single task. It is used for the * transfer from TaskManagers to the JobManager and from the JobManager to the Client. */ public class BaseAccumulatorSnapshot implements Serializable { From d4fe1019645026a221d8581746a1a342fd37c9f7 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 30 Jul 2015 01:11:36 +0300 Subject: [PATCH 031/175] . --- .../scala/org/apache/flink/runtime/jobmanager/JobManager.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index b231078fb611c..b1129edf54d40 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -363,6 +363,7 @@ class JobManager( // store the accumulators in the blobCache and get the keys. val newBlobKeys = LargeAccumulatorHelper.storeSerializedAccumulatorsToBlobCache( getBlobCacheServerAddress, serializedSmallAccumulators) + smallAccumulatorResults.clear() // and update the blobKeys to send to the client. From 22983c613d92636c82ee75637e8c256d53b6b6ab Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 30 Jul 2015 01:14:39 +0300 Subject: [PATCH 032/175] . --- .../org/apache/flink/runtime/jobmanager/JobManager.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index b1129edf54d40..f92efc9958f61 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -295,8 +295,8 @@ class JobManager( case ex: Exception => log.error(s"Could not serialize the next input split of " + s"class ${nextInputSplit.getClass}.", ex) - vertex.fail(new RuntimeException("Could not serialize the next input split " + - "of class " + nextInputSplit.getClass + ".", ex)) + vertex.fail(new RuntimeException("Could not serialize the next input " + + "split of class " + nextInputSplit.getClass + ".", ex)) null } @@ -323,8 +323,8 @@ class JobManager( currentJobs.get(jobID) match { case Some((executionGraph, jobInfo)) => executionGraph.getJobName - log.info(s"Status of job $jobID (${executionGraph.getJobName}) changed to $newJobStatus.", - error) + log.info(s"Status of job $jobID (${executionGraph.getJobName}) " + + "changed to $newJobStatus.", error) if (newJobStatus.isTerminalState) { jobInfo.end = timeStamp From a2a75ac795b6896a7f5968fdadc7e19ac27312db Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 30 Jul 2015 07:45:34 +0300 Subject: [PATCH 033/175] . --- .../scala/org/apache/flink/runtime/jobmanager/JobManager.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index f92efc9958f61..55ac7fa16dd4e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -960,7 +960,6 @@ class JobManager( try { libraryCacheManager.unregisterJob(jobID) - } catch { case t: Throwable => log.error(s"Could not properly unregister job $jobID form the library cache.", t) From 466f9bf9c98b05bf35ca29af1b04343b7c9ea5f6 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Fri, 17 Jul 2015 00:31:09 +0200 Subject: [PATCH 034/175] [cascading] add getJobConf() to HadoopInputSplit --- .../api/java/hadoop/mapred/wrapper/HadoopInputSplit.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/wrapper/HadoopInputSplit.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/wrapper/HadoopInputSplit.java index d949dfdbebeba..15f94b2d28a40 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/wrapper/HadoopInputSplit.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/wrapper/HadoopInputSplit.java @@ -80,6 +80,10 @@ public org.apache.hadoop.mapred.InputSplit getHadoopInputSplit() { return hadoopInputSplit; } + public JobConf getJobConf() { + return this.jobConf; + } + // ------------------------------------------------------------------------ // Serialization // ------------------------------------------------------------------------ From ca6dd42759f180779f9dfef63535f297fcb2eaf0 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 29 Jul 2015 14:51:14 +0200 Subject: [PATCH 035/175] [cascading] load user classloader when configuring InputFormat This closes #950. --- .../apache/flink/runtime/jobgraph/InputFormatVertex.java | 8 +++++++- .../org/apache/flink/runtime/operators/DataSinkTask.java | 8 +++++++- .../apache/flink/runtime/operators/DataSourceTask.java | 8 +++++++- 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java index 011850c09c908..781108c733b8b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java @@ -71,14 +71,20 @@ public void initializeOnMaster(ClassLoader loader) throws Exception { catch (Throwable t) { throw new Exception("Instantiating the InputFormat (" + formatDescription + ") failed: " + t.getMessage(), t); } - + + Thread thread = Thread.currentThread(); + ClassLoader original = thread.getContextClassLoader(); // configure try { + thread.setContextClassLoader(loader); inputFormat.configure(cfg.getStubParameters()); } catch (Throwable t) { throw new Exception("Configuring the InputFormat (" + formatDescription + ") failed: " + t.getMessage(), t); } + finally { + thread.setContextClassLoader(original); + } setInputSplitSource(inputFormat); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index b3130a1060f52..d291b5a3547d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -323,15 +323,21 @@ private void initOutputFormat() { catch (ClassCastException ccex) { throw new RuntimeException("The stub class is not a proper subclass of " + OutputFormat.class.getName(), ccex); } - + + Thread thread = Thread.currentThread(); + ClassLoader original = thread.getContextClassLoader(); // configure the stub. catch exceptions here extra, to report them as originating from the user code try { + thread.setContextClassLoader(userCodeClassLoader); this.format.configure(this.config.getStubParameters()); } catch (Throwable t) { throw new RuntimeException("The user defined 'configure()' method in the Output Format caused an error: " + t.getMessage(), t); } + finally { + thread.setContextClassLoader(original); + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 3f1c6425a7beb..df814089ea3cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -266,13 +266,19 @@ private void initInputFormat() { ccex); } - // configure the stub. catch exceptions here extra, to report them as originating from the user code + Thread thread = Thread.currentThread(); + ClassLoader original = thread.getContextClassLoader(); + // configure the stub. catch exceptions here extra, to report them as originating from the user code try { + thread.setContextClassLoader(userCodeClassLoader); this.format.configure(this.config.getStubParameters()); } catch (Throwable t) { throw new RuntimeException("The user defined 'configure()' method caused an error: " + t.getMessage(), t); } + finally { + thread.setContextClassLoader(original); + } // get the factory for the type serializer this.serializerFactory = this.config.getOutputSerializer(userCodeClassLoader); From 1b3bdce5ccb24a485ebc6452d16f78c770486481 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 30 Jul 2015 15:03:23 +0200 Subject: [PATCH 036/175] [client] send cancel message via the job manager's ActorGateway --- .../org/apache/flink/client/program/Client.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 50590dfb61a5d..1de0703c266f9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -27,8 +27,6 @@ import java.net.UnknownHostException; import java.util.List; -import akka.pattern.Patterns; -import akka.util.Timeout; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.configuration.IllegalConfigurationException; @@ -459,11 +457,17 @@ public void cancel(JobID jobId) throws Exception { try { jobManager = JobManager.getJobManagerRemoteReference(jobManagerAddress, actorSystem, timeout); } catch (Exception e) { - LOG.error("Error in getting the remote reference for the job manager", e); - throw new ProgramInvocationException("Failed to resolve JobManager", e); + throw new ProgramInvocationException("Error getting the remote actor reference for the job manager.", e); + } + + Future response; + try { + ActorGateway jobManagerGateway = JobManager.getJobManagerGateway(jobManager, timeout); + response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout); + } catch (Exception e) { + throw new ProgramInvocationException("Failed to query the job manager gateway.", e); } - Future response = Patterns.ask(jobManager, new JobManagerMessages.CancelJob(jobId), new Timeout(timeout)); Object result = Await.result(response, timeout); if (result instanceof JobManagerMessages.CancellationSuccess) { From 0558644ae1a1c8e0f21867ce1963aaa625170690 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Mon, 27 Jul 2015 08:56:13 +0200 Subject: [PATCH 037/175] [FLINK-2324] [streaming] Partitioned state checkpointing rework + test update --- .../operators/AbstractUdfStreamOperator.java | 35 ++--- .../api/operators/StatefulStreamOperator.java | 6 +- .../streaming/api/state/EagerStateStore.java | 34 ++--- .../streaming/api/state/LazyStateStore.java | 122 ------------------ .../api/state/OperatorStateHandle.java | 50 +++++++ .../api}/state/PartitionedStateStore.java | 7 +- .../state/PartitionedStreamOperatorState.java | 7 +- .../api/state/StreamOperatorState.java | 24 ++-- .../api/state/WrapperStateHandle.java | 13 +- .../streaming/runtime/tasks/StreamTask.java | 16 +-- .../api/state/StatefulOperatorTest.java | 3 +- .../StreamCheckpointingITCase.java | 99 +++++++------- 12 files changed, 175 insertions(+), 241 deletions(-) delete mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/LazyStateStore.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java rename {flink-runtime/src/main/java/org/apache/flink/runtime => flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api}/state/PartitionedStateStore.java (84%) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java index 23c4ab87505c9..f21aacc197f28 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java @@ -27,17 +27,16 @@ import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.PartitionedStateHandle; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.api.state.OperatorStateHandle; +import org.apache.flink.streaming.api.state.PartitionedStreamOperatorState; import org.apache.flink.streaming.api.state.StreamOperatorState; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; -import com.google.common.collect.ImmutableMap; - /** * This is used as the base class for operators that have a user-defined * function. @@ -78,7 +77,7 @@ public void close() throws Exception { @Override @SuppressWarnings({ "unchecked", "rawtypes" }) - public void restoreInitialState(Tuple2, Map> snapshots) throws Exception { + public void restoreInitialState(Tuple2, Map> snapshots) throws Exception { // Restore state using the Checkpointed interface if (userFunction instanceof Checkpointed) { ((Checkpointed) userFunction).restoreState(snapshots.f0.getState()); @@ -86,49 +85,51 @@ public void restoreInitialState(Tuple2, Map snapshot : snapshots.f1.entrySet()) { - Map> handles = snapshot.getValue().getState(); - StreamOperatorState restoredState = runtimeContext.getState(snapshot.getKey(), - !(handles instanceof ImmutableMap)); - restoredState.restoreState(snapshot.getValue().getState()); + for (Entry snapshot : snapshots.f1.entrySet()) { + StreamOperatorState restoredOpState = runtimeContext.getState(snapshot.getKey(), snapshot.getValue().isPartitioned()); + StateHandle checkpointHandle = snapshot.getValue(); + restoredOpState.restoreState(checkpointHandle); } } } @SuppressWarnings({ "rawtypes", "unchecked" }) - public Tuple2, Map> getStateSnapshotFromFunction(long checkpointId, long timestamp) + public Tuple2, Map> getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception { // Get all the states for the operator Map operatorStates = runtimeContext.getOperatorStates(); - Map operatorStateSnapshots; + Map operatorStateSnapshots; if (operatorStates.isEmpty()) { // We return null to signal that there is nothing to checkpoint operatorStateSnapshots = null; } else { // Checkpoint the states and store the handles in a map - Map snapshots = new HashMap(); + Map snapshots = new HashMap(); for (Entry state : operatorStates.entrySet()) { + boolean isPartitioned = state.getValue() instanceof PartitionedStreamOperatorState; snapshots.put(state.getKey(), - new PartitionedStateHandle(state.getValue().snapshotState(checkpointId, timestamp))); + new OperatorStateHandle(state.getValue().snapshotState(checkpointId, timestamp), + isPartitioned)); } operatorStateSnapshots = snapshots; } StateHandle checkpointedSnapshot = null; - + // if the UDF implements the Checkpointed interface we draw a snapshot if (userFunction instanceof Checkpointed) { StateHandleProvider provider = runtimeContext.getStateHandleProvider(); checkpointedSnapshot = provider.createStateHandle(((Checkpointed) userFunction) .snapshotState(checkpointId, timestamp)); } + // if we have either operator or checkpointed state we store it in a + // tuple2 otherwise return null if (operatorStateSnapshots != null || checkpointedSnapshot != null) { - return new Tuple2, Map>( - checkpointedSnapshot, operatorStateSnapshots); + return Tuple2.of(checkpointedSnapshot, operatorStateSnapshots); } else { return null; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java index afc36e01ae895..d400fc43b7223 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java @@ -21,8 +21,8 @@ import java.util.Map; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.state.PartitionedStateHandle; import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.streaming.api.state.OperatorStateHandle; /** * Interface for Stream operators that can have state. This interface is used for checkpointing @@ -32,9 +32,9 @@ */ public interface StatefulStreamOperator extends StreamOperator { - void restoreInitialState(Tuple2, Map> state) throws Exception; + void restoreInitialState(Tuple2, Map> state) throws Exception; - Tuple2, Map> getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception; + Tuple2, Map> getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception; void notifyCheckpointComplete(long checkpointId) throws Exception; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java index 6d3bad61a1d4d..213303a414129 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java @@ -24,20 +24,20 @@ import java.util.Map.Entry; import org.apache.flink.api.common.state.StateCheckpointer; -import org.apache.flink.runtime.state.PartitionedStateStore; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateHandleProvider; public class EagerStateStore implements PartitionedStateStore { - private StateCheckpointer checkpointer; - private final StateHandleProvider provider; + private StateCheckpointer checkpointer; + private final StateHandleProvider provider; private Map fetchedState; + @SuppressWarnings("unchecked") public EagerStateStore(StateCheckpointer checkpointer, StateHandleProvider provider) { this.checkpointer = checkpointer; - this.provider = provider; + this.provider = (StateHandleProvider) provider; fetchedState = new HashMap(); } @@ -58,23 +58,25 @@ public Map getPartitionedState() throws Exception { } @Override - public Map> snapshotStates(long checkpointId, - long checkpointTimestamp) { - - Map> handles = new HashMap>(); - + public StateHandle snapshotStates(long checkpointId, long checkpointTimestamp) { + // we map the values in the state-map using the state-checkpointer and store it as a checkpoint + Map checkpoints = new HashMap(); for (Entry stateEntry : fetchedState.entrySet()) { - handles.put(stateEntry.getKey(), provider.createStateHandle(checkpointer.snapshotState( - stateEntry.getValue(), checkpointId, checkpointTimestamp))); + checkpoints.put(stateEntry.getKey(), + checkpointer.snapshotState(stateEntry.getValue(), checkpointId, checkpointTimestamp)); } - return handles; + return provider.createStateHandle((Serializable) checkpoints); } @Override - public void restoreStates(Map> snapshots) throws Exception { - for (Entry> snapshotEntry : snapshots.entrySet()) { - fetchedState.put(snapshotEntry.getKey(), - checkpointer.restoreState(snapshotEntry.getValue().getState())); + public void restoreStates(StateHandle snapshot) throws Exception { + + @SuppressWarnings("unchecked") + Map checkpoints = (Map) snapshot.getState(); + + // we map the values back to the state from the checkpoints + for (Entry snapshotEntry : checkpoints.entrySet()) { + fetchedState.put(snapshotEntry.getKey(), (S) checkpointer.restoreState(snapshotEntry.getValue())); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/LazyStateStore.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/LazyStateStore.java deleted file mode 100644 index 14484eafe98a1..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/LazyStateStore.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.streaming.api.state; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.flink.api.common.state.StateCheckpointer; -import org.apache.flink.runtime.state.PartitionedStateStore; -import org.apache.flink.runtime.state.StateHandle; -import org.apache.flink.runtime.state.StateHandleProvider; - -/** - * Implementation of the {@link PartitionedStateStore} interface for lazy - * retrieval and snapshotting of the partitioned operator states. Lazy state - * access considerably speeds up recovery and makes resource access smoother by - * avoiding request congestion in the persistent storage layer. - * - *

- * The logic implemented here can also be used later to push unused state to the - * persistent layer and also avoids re-snapshotting the unmodified states. - *

- * - * @param - * Type of the operator states. - * @param - * Type of the state checkpoints. - */ -public class LazyStateStore implements PartitionedStateStore { - - protected StateCheckpointer checkpointer; - protected final StateHandleProvider provider; - - private final Map> unfetchedState; - private final Map fetchedState; - - public LazyStateStore(StateCheckpointer checkpointer, StateHandleProvider provider) { - this.checkpointer = checkpointer; - this.provider = provider; - - unfetchedState = new HashMap>(); - fetchedState = new HashMap(); - } - - @Override - public S getStateForKey(Serializable key) throws Exception { - S state = fetchedState.get(key); - if (state != null) { - return state; - } else { - StateHandle handle = unfetchedState.get(key); - if (handle != null) { - state = checkpointer.restoreState(handle.getState()); - fetchedState.put(key, state); - unfetchedState.remove(key); - return state; - } else { - return null; - } - } - } - - @Override - public void setStateForKey(Serializable key, S state) { - fetchedState.put(key, state); - unfetchedState.remove(key); - } - - @Override - public Map getPartitionedState() throws Exception { - for (Entry> handleEntry : unfetchedState.entrySet()) { - fetchedState.put(handleEntry.getKey(), - checkpointer.restoreState(handleEntry.getValue().getState())); - } - unfetchedState.clear(); - return fetchedState; - } - - @Override - public Map> snapshotStates(long checkpointId, - long checkpointTimestamp) { - for (Entry stateEntry : fetchedState.entrySet()) { - unfetchedState.put(stateEntry.getKey(), provider.createStateHandle(checkpointer - .snapshotState(stateEntry.getValue(), checkpointId, checkpointTimestamp))); - } - return unfetchedState; - } - - @Override - public void restoreStates(Map> snapshots) { - unfetchedState.putAll(snapshots); - } - - @Override - public boolean containsKey(Serializable key) { - return fetchedState.containsKey(key) || unfetchedState.containsKey(key); - } - - @Override - public void setCheckPointer(StateCheckpointer checkpointer) { - this.checkpointer = checkpointer; - } - -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java new file mode 100644 index 0000000000000..87536ed7d8ec2 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java @@ -0,0 +1,50 @@ +/* + * 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.streaming.api.state; + +import java.io.Serializable; + +import org.apache.flink.runtime.state.StateHandle; + +public class OperatorStateHandle implements StateHandle { + + private static final long serialVersionUID = 1L; + + private final StateHandle handle; + private final boolean isPartitioned; + + public OperatorStateHandle(StateHandle handle, boolean isPartitioned){ + this.handle = handle; + this.isPartitioned = isPartitioned; + } + + public boolean isPartitioned(){ + return isPartitioned; + } + + @Override + public Serializable getState() throws Exception { + return handle.getState(); + } + + @Override + public void discardState() throws Exception { + handle.discardState(); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateStore.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java similarity index 84% rename from flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateStore.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java index 6353eda1bfebb..5201058bb1c26 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateStore.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java @@ -16,12 +16,13 @@ * limitations under the License. */ -package org.apache.flink.runtime.state; +package org.apache.flink.streaming.api.state; import java.io.Serializable; import java.util.Map; import org.apache.flink.api.common.state.StateCheckpointer; +import org.apache.flink.runtime.state.StateHandle; /** * Interface for storing and accessing partitioned state. The interface is @@ -40,9 +41,9 @@ public interface PartitionedStateStore { Map getPartitionedState() throws Exception; - Map> snapshotStates(long checkpointId, long checkpointTimestamp) throws Exception; + StateHandle snapshotStates(long checkpointId, long checkpointTimestamp) throws Exception; - void restoreStates(Map> snapshots) throws Exception; + void restoreStates(StateHandle snapshot) throws Exception; boolean containsKey(Serializable key); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java index b22aed4155169..b165a94ee378b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.state.OperatorState; import org.apache.flink.api.common.state.StateCheckpointer; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.runtime.state.PartitionedStateStore; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -83,7 +82,7 @@ public S value() throws IOException { if (stateStore.containsKey(key)) { return stateStore.getStateForKey(key); } else { - return checkpointer.restoreState((C) InstantiationUtil.deserializeObject( + return (S) checkpointer.restoreState((C) InstantiationUtil.deserializeObject( defaultState, cl)); } } catch (Exception e) { @@ -123,13 +122,13 @@ public void setCurrentInput(IN input) { } @Override - public Map> snapshotState(long checkpointId, + public StateHandle snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { return stateStore.snapshotStates(checkpointId, checkpointTimestamp); } @Override - public void restoreState(Map> snapshots) throws Exception { + public void restoreState(StateHandle snapshots) throws Exception { stateStore.restoreStates(snapshots); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java index 2724efbfc8ac1..6e0a3ea385a2d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java @@ -42,15 +42,14 @@ */ public class StreamOperatorState implements OperatorState { - public static final Serializable DEFAULTKEY = -1; - private S state; protected StateCheckpointer checkpointer; - private final StateHandleProvider provider; + private final StateHandleProvider provider; + @SuppressWarnings("unchecked") public StreamOperatorState(StateCheckpointer checkpointer, StateHandleProvider provider) { this.checkpointer = checkpointer; - this.provider = provider; + this.provider = (StateHandleProvider) provider; } @SuppressWarnings("unchecked") @@ -85,23 +84,24 @@ public void setCheckpointer(StateCheckpointer checkpointer) { this.checkpointer = checkpointer; } - protected StateHandleProvider getStateHandleProvider() { + protected StateHandleProvider getStateHandleProvider() { return provider; } - public Map> snapshotState(long checkpointId, - long checkpointTimestamp) throws Exception { - return ImmutableMap.of(DEFAULTKEY, provider.createStateHandle(checkpointer.snapshotState( - value(), checkpointId, checkpointTimestamp))); + public StateHandle snapshotState(long checkpointId, long checkpointTimestamp) + throws Exception { + return provider.createStateHandle(checkpointer.snapshotState(value(), checkpointId, + checkpointTimestamp)); } - public void restoreState(Map> snapshots) throws Exception { - update(checkpointer.restoreState(snapshots.get(DEFAULTKEY).getState())); + @SuppressWarnings("unchecked") + public void restoreState(StateHandle snapshot) throws Exception { + update((S) checkpointer.restoreState((C) snapshot.getState())); } public Map getPartitionedState() throws Exception { - return ImmutableMap.of(DEFAULTKEY, state); + return ImmutableMap.of((Serializable) 0, state); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java index 1adef48ea14cb..27c697a7251e2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java @@ -24,7 +24,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.LocalStateHandle; -import org.apache.flink.runtime.state.PartitionedStateHandle; import org.apache.flink.runtime.state.StateHandle; /** @@ -36,24 +35,22 @@ public class WrapperStateHandle extends LocalStateHandle { private static final long serialVersionUID = 1L; - public WrapperStateHandle(List, Map>> state) { + public WrapperStateHandle(List, Map>> state) { super((Serializable) state); } @Override public void discardState() throws Exception { @SuppressWarnings("unchecked") - List, Map>> chainedStates = (List, Map>>) getState(); - for (Tuple2, Map> state : chainedStates) { + List, Map>> chainedStates = (List, Map>>) getState(); + for (Tuple2, Map> state : chainedStates) { if (state != null) { if (state.f0 != null) { state.f0.discardState(); } if (state.f1 != null) { - for (PartitionedStateHandle statePartitions : state.f1.values()) { - for (StateHandle handle : statePartitions.getState().values()) { - handle.discardState(); - } + for (StateHandle opState : state.f1.values()) { + opState.discardState(); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index d8298337d49bc..2098da83e1ce3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -38,13 +38,13 @@ import org.apache.flink.runtime.jobgraph.tasks.OperatorStateCarrier; import org.apache.flink.runtime.state.FileStateHandle; import org.apache.flink.runtime.state.LocalStateHandle; -import org.apache.flink.runtime.state.PartitionedStateHandle; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StatefulStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.state.OperatorStateHandle; import org.apache.flink.streaming.api.state.WrapperStateHandle; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -210,12 +210,12 @@ public EventListener getCheckpointBarrierListener() { @Override public void setInitialState(StateHandle stateHandle) throws Exception { - // We retrieve end restore the states for the chained operators. - List, Map>> chainedStates = (List, Map>>) stateHandle.getState(); + // We retrieve end restore the states for the chained oeprators. + List, Map>> chainedStates = (List, Map>>) stateHandle.getState(); - // We restore all stateful chained operators + // We restore all stateful operators for (int i = 0; i < chainedStates.size(); i++) { - Tuple2, Map> state = chainedStates.get(i); + Tuple2, Map> state = chainedStates.get(i); // If state is not null we need to restore it if (state != null) { StreamOperator chainedOperator = outputHandler.getChainedOperators().get(i); @@ -234,15 +234,14 @@ public void triggerCheckpoint(long checkpointId, long timestamp) throws Exceptio LOG.debug("Starting checkpoint {} on task {}", checkpointId, getName()); // We wrap the states of the chained operators in a list, marking non-stateful oeprators with null - List, Map>> chainedStates = new ArrayList, Map>>(); + List, Map>> chainedStates = new ArrayList, Map>>(); // A wrapper handle is created for the List of statehandles WrapperStateHandle stateHandle; try { // We construct a list of states for chained tasks - for (StreamOperator chainedOperator : outputHandler - .getChainedOperators()) { + for (StreamOperator chainedOperator : outputHandler.getChainedOperators()) { if (chainedOperator instanceof StatefulStreamOperator) { chainedStates.add(((StatefulStreamOperator) chainedOperator) .getStateSnapshotFromFunction(checkpointId, timestamp)); @@ -281,7 +280,6 @@ public void triggerCheckpoint(long checkpointId, long timestamp) throws Exceptio @SuppressWarnings("rawtypes") @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { - // we do nothing here so far. this should call commit on the source function, for example synchronized (checkpointLock) { for (StreamOperator chainedOperator : outputHandler.getChainedOperators()) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java index a7a8a09f187fe..6ca38b7ffe442 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java @@ -41,7 +41,6 @@ import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider; -import org.apache.flink.runtime.state.PartitionedStateHandle; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.KeyedDataStream; @@ -166,7 +165,7 @@ public void close() { }, context); if (serializedState != null) { - op.restoreInitialState((Tuple2, Map>) InstantiationUtil + op.restoreInitialState((Tuple2, Map>) InstantiationUtil .deserializeObject(serializedState, Thread.currentThread() .getContextClassLoader())); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java index 438e980c2bb04..3f99fa011e01b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java @@ -18,32 +18,33 @@ package org.apache.flink.test.checkpointing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; + import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.state.OperatorState; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.test.util.ForkableFlinkMiniCluster; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Random; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * A simple test that runs a streaming topology with checkpointing enabled. @@ -94,7 +95,7 @@ public static void shutdownCluster() { * Runs the following program: * *
-	 *     [ (source)->(filter)->(map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
+	 *     [ (source)->(filter) ]-s->[ (map) ] -> [ (map) ] -> [ (groupBy/count)->(sink) ]
 	 * 
*/ @Test @@ -114,37 +115,22 @@ public void runCheckpointedProgram() { stream // -------------- first vertex, chained to the source ---------------- - .filter(new StringRichFilterFunction()) + .filter(new StringRichFilterFunction()).shuffle() // -------------- seconds vertex - the stateful one that also fails ---------------- .map(new StringPrefixCountRichMapFunction()) .startNewChain() .map(new StatefulCounterFunction()) - // -------------- third vertex - reducer and the sink ---------------- + // -------------- third vertex - counter and the sink ---------------- .groupBy("prefix") - .reduce(new OnceFailingReducer(NUM_STRINGS)) - .addSink(new RichSinkFunction() { - - private Map counts = new HashMap(); + .map(new OnceFailingPrefixCounter(NUM_STRINGS)) + .addSink(new SinkFunction() { @Override - public void invoke(PrefixCount value) { - Character first = value.prefix.charAt(0); - Long previous = counts.get(first); - if (previous == null) { - counts.put(first, value.count); - } else { - counts.put(first, Math.max(previous, value.count)); - } + public void invoke(PrefixCount value) throws Exception { + // Do nothing here } - -// @Override -// public void close() { -// for (Long count : counts.values()) { -// assertEquals(NUM_STRINGS / 40, count.longValue()); -// } -// } }); env.execute(); @@ -163,14 +149,20 @@ public void invoke(PrefixCount value) { for (long l : StatefulCounterFunction.counts) { countSum += l; } - - // verify that we counted exactly right - // this line should be uncommented once the "exactly one off by one" is fixed - // if this fails we see at which point the count is off + long reduceInputCount = 0; + for(long l: OnceFailingPrefixCounter.counts){ + reduceInputCount += l; + } + assertEquals(NUM_STRINGS, filterSum); assertEquals(NUM_STRINGS, mapSum); assertEquals(NUM_STRINGS, countSum); + assertEquals(NUM_STRINGS, reduceInputCount); + // verify that we counted exactly right + for (Long count : OnceFailingPrefixCounter.prefixCounts.values()) { + assertEquals(new Long(NUM_STRINGS / 40), count); + } } catch (Exception e) { e.printStackTrace(); @@ -277,7 +269,10 @@ public void close() throws IOException { } - private static class OnceFailingReducer extends RichReduceFunction { + private static class OnceFailingPrefixCounter extends RichMapFunction { + + private static Map prefixCounts = new ConcurrentHashMap(); + static final long[] counts = new long[PARALLELISM]; private static volatile boolean hasFailed = false; @@ -285,30 +280,44 @@ private static class OnceFailingReducer extends RichReduceFunction private long failurePos; private long count; + + private OperatorState pCount; + private OperatorState inputCount; - OnceFailingReducer(long numElements) { + OnceFailingPrefixCounter(long numElements) { this.numElements = numElements; } @Override - public void open(Configuration parameters) { + public void open(Configuration parameters) throws IOException { long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; count = 0; + pCount = getRuntimeContext().getOperatorState("prefix-count", 0L, true); + inputCount = getRuntimeContext().getOperatorState("input-count", 0L, false); } @Override - public PrefixCount reduce(PrefixCount value1, PrefixCount value2) throws Exception { + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = inputCount.value(); + } + + @Override + public PrefixCount map(PrefixCount value) throws Exception { count++; if (!hasFailed && count >= failurePos) { hasFailed = true; throw new Exception("Test Failure"); } - - value1.count += value2.count; - return value1; + inputCount.update(inputCount.value() + 1); + + long currentPrefixCount = pCount.value() + value.count; + pCount.update(currentPrefixCount); + prefixCounts.put(value.prefix, currentPrefixCount); + value.count = currentPrefixCount; + return value; } } @@ -316,7 +325,7 @@ public PrefixCount reduce(PrefixCount value1, PrefixCount value2) throws Excepti // Custom Type Classes // -------------------------------------------------------------------------------------------- - public static class PrefixCount { + public static class PrefixCount implements Serializable { public String prefix; public String value; From 58cd4ea615bbbbec682ec82f6380bc30c12c5899 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Tue, 28 Jul 2015 15:46:13 +0200 Subject: [PATCH 038/175] [FLINK-2324] [streaming] ITCase added for partitioned states --- .../PartitionedStateCheckpointingITCase.java | 257 ++++++++++++++++++ 1 file changed, 257 insertions(+) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java new file mode 100644 index 0000000000000..88361e28cceea --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java @@ -0,0 +1,257 @@ +/* + * 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.test.checkpointing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.state.OperatorState; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * A simple test that runs a streaming topology with checkpointing enabled. + * + * The test triggers a failure after a while and verifies that, after + * completion, the state reflects the "exactly once" semantics. + * + * It is designed to check partitioned states. + */ +@SuppressWarnings("serial") +public class PartitionedStateCheckpointingITCase { + + private static final int NUM_TASK_MANAGERS = 2; + private static final int NUM_TASK_SLOTS = 3; + private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + + private static ForkableFlinkMiniCluster cluster; + + @BeforeClass + public static void startCluster() { + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); + config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + + cluster = new ForkableFlinkMiniCluster(config, false); + } catch (Exception e) { + e.printStackTrace(); + fail("Failed to start test cluster: " + e.getMessage()); + } + } + + @AfterClass + public static void shutdownCluster() { + try { + cluster.shutdown(); + cluster = null; + } catch (Exception e) { + e.printStackTrace(); + fail("Failed to stop test cluster: " + e.getMessage()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void runCheckpointedProgram() { + + final long NUM_STRINGS = 10000000L; + assertTrue("Broken test setup", (NUM_STRINGS/2) % 40 == 0); + + try { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", + cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(500); + env.getConfig().disableSysoutLogging(); + + DataStream stream1 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2)); + DataStream stream2 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2)); + + stream1.union(stream2) + .groupBy(new IdentityKeySelector()) + .map(new OnceFailingPartitionedSum(NUM_STRINGS)) + .keyBy(0) + .addSink(new CounterSink()); + + env.execute(); + + // verify that we counted exactly right + for (Entry sum : OnceFailingPartitionedSum.allSums.entrySet()) { + assertEquals(new Long(sum.getKey() * NUM_STRINGS / 40), sum.getValue()); + } + System.out.println("new"); + for (Long count : CounterSink.allCounts.values()) { + assertEquals(new Long(NUM_STRINGS / 40), count); + } + + assertEquals(40, CounterSink.allCounts.size()); + assertEquals(40, OnceFailingPartitionedSum.allSums.size()); + + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + // Custom Functions + // -------------------------------------------------------------------------------------------- + + private static class IntGeneratingSourceFunction extends RichParallelSourceFunction { + + private final long numElements; + + private OperatorState index; + private int step; + + private volatile boolean isRunning; + + static final long[] counts = new long[PARALLELISM]; + + @Override + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = index.value(); + } + + IntGeneratingSourceFunction(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) throws IOException { + step = getRuntimeContext().getNumberOfParallelSubtasks(); + + index = getRuntimeContext().getOperatorState("index", + getRuntimeContext().getIndexOfThisSubtask(), false); + + isRunning = true; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final Object lockingObject = ctx.getCheckpointLock(); + + while (isRunning && index.value() < numElements) { + + synchronized (lockingObject) { + index.update(index.value() + step); + ctx.collect(index.value() % 40); + } + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class OnceFailingPartitionedSum extends RichMapFunction> { + + private static Map allSums = new ConcurrentHashMap(); + private static volatile boolean hasFailed = false; + + private final long numElements; + + private long failurePos; + private long count; + + private OperatorState sum; + + OnceFailingPartitionedSum(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) throws IOException { + long failurePosMin = (long) (0.4 * numElements / getRuntimeContext() + .getNumberOfParallelSubtasks()); + long failurePosMax = (long) (0.7 * numElements / getRuntimeContext() + .getNumberOfParallelSubtasks()); + + failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; + count = 0; + sum = getRuntimeContext().getOperatorState("sum", 0L, true); + } + + @Override + public Tuple2 map(Integer value) throws Exception { + count++; + if (!hasFailed && count >= failurePos) { + hasFailed = true; + throw new Exception("Test Failure"); + } + + long currentSum = sum.value() + value; + sum.update(currentSum); + allSums.put(value, currentSum); + return new Tuple2(value, currentSum); + } + } + + private static class CounterSink extends RichSinkFunction> { + + private static Map allCounts = new ConcurrentHashMap(); + + private OperatorState counts; + + @Override + public void open(Configuration parameters) throws IOException { + counts = getRuntimeContext().getOperatorState("count", 0L, true); + } + + @Override + public void invoke(Tuple2 value) throws Exception { + long currentCount = counts.value() + 1; + counts.update(currentCount); + allCounts.put(value.f0, currentCount); + + } + } + + private static class IdentityKeySelector implements KeySelector { + + @Override + public T getKey(T value) throws Exception { + return value; + } + + } +} From 83e14cb15a41c417fd7d024c383a4f4d50ec5a19 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Thu, 30 Jul 2015 07:26:40 +0200 Subject: [PATCH 039/175] [FLINK-2324] [streaming] Added test for different StateHandle wrappers Closes #937 --- .../operators/AbstractUdfStreamOperator.java | 8 +- .../api/state/OperatorStateHandle.java | 4 + .../streaming/api/state/StateHandleTest.java | 134 ++++++++++++++++++ .../StreamCheckpointingITCase.java | 14 +- 4 files changed, 156 insertions(+), 4 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java index f21aacc197f28..585b4cece839d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java @@ -79,7 +79,7 @@ public void close() throws Exception { @SuppressWarnings({ "unchecked", "rawtypes" }) public void restoreInitialState(Tuple2, Map> snapshots) throws Exception { // Restore state using the Checkpointed interface - if (userFunction instanceof Checkpointed) { + if (userFunction instanceof Checkpointed && snapshots.f0 != null) { ((Checkpointed) userFunction).restoreState(snapshots.f0.getState()); } @@ -122,8 +122,10 @@ public Tuple2, Map> getSt // if the UDF implements the Checkpointed interface we draw a snapshot if (userFunction instanceof Checkpointed) { StateHandleProvider provider = runtimeContext.getStateHandleProvider(); - checkpointedSnapshot = provider.createStateHandle(((Checkpointed) userFunction) - .snapshotState(checkpointId, timestamp)); + Serializable state = ((Checkpointed) userFunction).snapshotState(checkpointId, timestamp); + if (state != null) { + checkpointedSnapshot = provider.createStateHandle(state); + } } // if we have either operator or checkpointed state we store it in a diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java index 87536ed7d8ec2..f308ba881a4a8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java @@ -46,5 +46,9 @@ public Serializable getState() throws Exception { public void discardState() throws Exception { handle.discardState(); } + + public StateHandle getHandle() { + return handle; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java new file mode 100644 index 0000000000000..38117e8761e33 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java @@ -0,0 +1,134 @@ +/* + * 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.streaming.api.state; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.util.InstantiationUtil; +import org.junit.Test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +public class StateHandleTest { + + @Test + public void operatorStateHandleTest() throws Exception { + + MockHandle h1 = new MockHandle(1); + + OperatorStateHandle opHandle = new OperatorStateHandle(h1, true); + assertEquals(1, opHandle.getState()); + + OperatorStateHandle dsHandle = serializeDeserialize(opHandle); + MockHandle h2 = (MockHandle) dsHandle.getHandle(); + assertFalse(h2.discarded); + assertNotNull(h1.state); + assertNull(h2.state); + + dsHandle.discardState(); + + assertTrue(h2.discarded); + } + + @Test + public void wrapperStateHandleTest() throws Exception { + + MockHandle h1 = new MockHandle(1); + MockHandle h2 = new MockHandle(2); + StateHandle h3 = new MockHandle(3); + + OperatorStateHandle opH1 = new OperatorStateHandle(h1, true); + OperatorStateHandle opH2 = new OperatorStateHandle(h2, false); + + Map opHandles = ImmutableMap.of("h1", opH1, "h2", opH2); + + Tuple2, Map> fullState = Tuple2.of(h3, + opHandles); + + List, Map>> chainedStates = ImmutableList + .of(fullState); + + WrapperStateHandle wrapperHandle = new WrapperStateHandle(chainedStates); + + WrapperStateHandle dsWrapper = serializeDeserialize(wrapperHandle); + + @SuppressWarnings("unchecked") + Tuple2, Map> dsFullState = ((List, Map>>) dsWrapper + .getState()).get(0); + + Map dsOpHandles = dsFullState.f1; + + assertNull(dsFullState.f0.getState()); + assertFalse(((MockHandle) dsFullState.f0).discarded); + assertFalse(((MockHandle) dsOpHandles.get("h1").getHandle()).discarded); + assertNull(dsOpHandles.get("h1").getState()); + assertFalse(((MockHandle) dsOpHandles.get("h2").getHandle()).discarded); + assertNull(dsOpHandles.get("h2").getState()); + + dsWrapper.discardState(); + + assertTrue(((MockHandle) dsFullState.f0).discarded); + assertTrue(((MockHandle) dsOpHandles.get("h1").getHandle()).discarded); + assertTrue(((MockHandle) dsOpHandles.get("h2").getHandle()).discarded); + + } + + @SuppressWarnings("unchecked") + private > X serializeDeserialize(X handle) throws IOException, + ClassNotFoundException { + byte[] serialized = InstantiationUtil.serializeObject(handle); + return (X) InstantiationUtil.deserializeObject(serialized, Thread.currentThread() + .getContextClassLoader()); + } + + @SuppressWarnings("serial") + private static class MockHandle implements StateHandle { + + boolean discarded = false; + transient T state; + + public MockHandle(T state) { + this.state = state; + } + + @Override + public void discardState() { + state = null; + discarded = true; + } + + @Override + public T getState() { + return state; + } + } + +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java index 3f99fa011e01b..93dda5f684d42 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java @@ -372,7 +372,8 @@ public void restoreState(Long state) { } } - private static class StringPrefixCountRichMapFunction extends RichMapFunction { + private static class StringPrefixCountRichMapFunction extends RichMapFunction + implements Checkpointed { OperatorState count; static final long[] counts = new long[PARALLELISM]; @@ -392,5 +393,16 @@ public void open(Configuration conf) throws IOException { public void close() throws IOException { counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value(); } + + @Override + public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return null; + } + + @Override + public void restoreState(Integer state) { + // verify that we never store/restore null state + fail(); + } } } From 1919ae735ca557dad5f05bf787599b76fe80928f Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 30 Jul 2015 16:49:01 +0200 Subject: [PATCH 040/175] [cleanup] move updateAccumulators method to a more sensible location --- .../executiongraph/ExecutionGraph.java | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 833518ce84513..be92bd4b79b95 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -137,30 +137,6 @@ public class ExecutionGraph implements Serializable { /** The currently executed tasks, for callbacks */ private final ConcurrentHashMap currentExecutions; - /** - * Updates the accumulators during the runtime of a job. Final accumulator results are transferred - * through the UpdateTaskExecutionState message. - * @param accumulatorSnapshot The serialized flink and user-defined accumulators - */ - public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { - Map> flinkAccumulators; - Map> userAccumulators; - try { - flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - userAccumulators = accumulatorSnapshot.deserializeUserAccumulators(userClassLoader); - - ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); - Execution execution = currentExecutions.get(execID); - if (execution != null) { - execution.setAccumulators(flinkAccumulators, userAccumulators); - } else { - LOG.warn("Received accumulator result for unknown execution {}.", execID); - } - } catch (Exception e) { - LOG.error("Cannot update accumulators for job " + jobID, e); - } - } - /** A list of all libraries required during the job execution. Libraries have to be stored * inside the BlobService and are referenced via the BLOB keys. */ private final List requiredJarFiles; @@ -1007,6 +983,30 @@ void deregisterExecution(Execution exec) { } } + /** + * Updates the accumulators during the runtime of a job. Final accumulator results are transferred + * through the UpdateTaskExecutionState message. + * @param accumulatorSnapshot The serialized flink and user-defined accumulators + */ + public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { + Map> flinkAccumulators; + Map> userAccumulators; + try { + flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); + userAccumulators = accumulatorSnapshot.deserializeUserAccumulators(userClassLoader); + + ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); + Execution execution = currentExecutions.get(execID); + if (execution != null) { + execution.setAccumulators(flinkAccumulators, userAccumulators); + } else { + LOG.warn("Received accumulator result for unknown execution {}.", execID); + } + } catch (Exception e) { + LOG.error("Cannot update accumulators for job " + jobID, e); + } + } + // -------------------------------------------------------------------------------------------- // Listeners & Observers // -------------------------------------------------------------------------------------------- From 68b1559319a870c3120263c5d10967ff6b690682 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 21 Jul 2015 21:22:19 +0200 Subject: [PATCH 041/175] [FLINK-1927][FLINK-2173][py] Operator distribution rework, fix file paths Python operators are no longer serialized and shipped across the cluster. Instead the plan file is executed on each node, followed by usage of the respective operator object. - removed dill library - filepaths are always explicitly passed to python - fix error reporting This closes #931. --- .../api/java/common/streaming/Receiver.java | 8 +- .../api/java/common/streaming/Sender.java | 6 +- .../api/java/python/PythonPlanBinder.java | 82 +- .../java/python/functions/PythonCoGroup.java | 4 +- .../functions/PythonCombineIdentity.java | 4 +- .../python/functions/PythonMapPartition.java | 4 +- .../java/python/streaming/PythonStreamer.java | 66 +- .../languagebinding/api/python/dill/__diff.py | 229 ---- .../api/python/dill/__init__.py | 73 -- .../api/python/dill/_objects.py | 530 --------- .../languagebinding/api/python/dill/detect.py | 222 ---- .../languagebinding/api/python/dill/dill.py | 1034 ----------------- .../api/python/dill/objtypes.py | 27 - .../api/python/dill/pointers.py | 122 -- .../languagebinding/api/python/dill/source.py | 1010 ---------------- .../languagebinding/api/python/dill/temp.py | 236 ---- .../languagebinding/api/python/executor.py | 54 - .../api/python/flink/connection/Collector.py | 4 - .../api/python/flink/connection/Connection.py | 14 +- .../api/python/flink/functions/Function.py | 13 +- .../api/python/flink/plan/Environment.py | 70 +- pom.xml | 2 - 22 files changed, 110 insertions(+), 3704 deletions(-) delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__diff.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__init__.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/_objects.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/detect.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/dill.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/objtypes.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/pointers.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/source.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/temp.py delete mode 100644 flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/executor.py diff --git a/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java b/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java index 2741714647ecd..23720d71a89a1 100644 --- a/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java +++ b/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java @@ -50,11 +50,7 @@ public void open(String path) throws IOException { setupMappedFile(path); } - private void setupMappedFile(String path) throws FileNotFoundException, IOException { - String inputFilePath = function == null - ? FLINK_TMP_DATA_DIR + "/" + "output" - : path; - + private void setupMappedFile(String inputFilePath) throws FileNotFoundException, IOException { File x = new File(FLINK_TMP_DATA_DIR); x.mkdirs(); @@ -100,7 +96,7 @@ private void loadBuffer() throws IOException { count++; } if (fileBuffer.get(0) == 0) { - throw new RuntimeException("External process not respoonding."); + throw new RuntimeException("External process not responding."); } fileBuffer.position(1); } diff --git a/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Sender.java b/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Sender.java index d45b0197978f1..3e0c317ad4342 100644 --- a/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Sender.java +++ b/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Sender.java @@ -65,11 +65,7 @@ public void open(String path) throws IOException { setupMappedFile(path); } - private void setupMappedFile(String path) throws FileNotFoundException, IOException { - String outputFilePath = function == null - ? FLINK_TMP_DATA_DIR + "/" + "input" - : path; - + private void setupMappedFile(String outputFilePath) throws FileNotFoundException, IOException { File x = new File(FLINK_TMP_DATA_DIR); x.mkdirs(); diff --git a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/PythonPlanBinder.java b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/PythonPlanBinder.java index c278f5c6599da..7c30418455ff8 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/PythonPlanBinder.java +++ b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/PythonPlanBinder.java @@ -52,7 +52,6 @@ public class PythonPlanBinder extends PlanBinder { public static final String FLINK_PYTHON_DC_ID = "flink"; public static final String FLINK_PYTHON_PLAN_NAME = "/plan.py"; - public static final String FLINK_PYTHON_EXECUTOR_NAME = "/executor.py"; public static final String FLINK_PYTHON2_BINARY_KEY = "python.binary.python2"; public static final String FLINK_PYTHON3_BINARY_KEY = "python.binary.python3"; @@ -64,6 +63,8 @@ public class PythonPlanBinder extends PlanBinder { protected static final String FLINK_DIR = System.getenv("FLINK_ROOT_DIR"); protected static String FULL_PATH; + public static StringBuilder arguments = new StringBuilder(); + private Process process; public static boolean usePython3 = false; @@ -172,12 +173,11 @@ private static void distributeFiles(ExecutionEnvironment env) throws IOException } private void startPython(String[] args) throws IOException { - StringBuilder argsBuilder = new StringBuilder(); for (String arg : args) { - argsBuilder.append(" ").append(arg); + arguments.append(" ").append(arg); } receiver = new Receiver(null); - receiver.open(null); + receiver.open(FLINK_TMP_DATA_DIR + "/output"); String pythonBinaryPath = usePython3 ? FLINK_PYTHON3_BINARY_PATH : FLINK_PYTHON2_BINARY_PATH; @@ -186,7 +186,7 @@ private void startPython(String[] args) throws IOException { } catch (IOException ex) { throw new RuntimeException(pythonBinaryPath + " does not point to a valid python binary."); } - process = Runtime.getRuntime().exec(pythonBinaryPath + " -B " + FLINK_PYTHON_FILE_PATH + FLINK_PYTHON_PLAN_NAME + argsBuilder.toString()); + process = Runtime.getRuntime().exec(pythonBinaryPath + " -B " + FLINK_PYTHON_FILE_PATH + FLINK_PYTHON_PLAN_NAME + arguments.toString()); new StreamPrinter(process.getInputStream()).start(); new StreamPrinter(process.getErrorStream()).start(); @@ -201,8 +201,15 @@ private void startPython(String[] args) throws IOException { if (value != 0) { throw new RuntimeException("Plan file caused an error. Check log-files for details."); } + if (value == 0) { + throw new RuntimeException("Plan file exited prematurely without an error."); + } } catch (IllegalThreadStateException ise) {//Process still running } + + process.getOutputStream().write("plan\n".getBytes()); + process.getOutputStream().write((FLINK_TMP_DATA_DIR + "/output\n").getBytes()); + process.getOutputStream().flush(); } private void close() { @@ -231,10 +238,7 @@ private void close() { //=====Plan Binding================================================================================================= protected class PythonOperationInfo extends OperationInfo { - protected byte[] operator; - protected String meta; protected boolean combine; - protected byte[] combineOperator; protected String name; @Override @@ -244,11 +248,8 @@ public String toString() { sb.append("ParentID: ").append(parentID).append("\n"); sb.append("OtherID: ").append(otherID).append("\n"); sb.append("Name: ").append(name).append("\n"); - sb.append("Operator: ").append(operator == null ? null : "").append("\n"); - sb.append("Meta: ").append(meta).append("\n"); sb.append("Types: ").append(types).append("\n"); sb.append("Combine: ").append(combine).append("\n"); - sb.append("CombineOP: ").append(combineOperator == null ? null : "").append("\n"); sb.append("Keys1: ").append(Arrays.toString(keys1)).append("\n"); sb.append("Keys2: ").append(Arrays.toString(keys2)).append("\n"); sb.append("Projections: ").append(Arrays.toString(projections)).append("\n"); @@ -264,8 +265,6 @@ protected PythonOperationInfo(AbstractOperation identifier) throws IOException { otherID = (Integer) receiver.getRecord(true); keys1 = tupleToIntArray((Tuple) receiver.getRecord(true)); keys2 = tupleToIntArray((Tuple) receiver.getRecord(true)); - operator = (byte[]) receiver.getRecord(); - meta = (String) receiver.getRecord(); tmpType = receiver.getRecord(); types = tmpType == null ? null : getForObject(tmpType); name = (String) receiver.getRecord(); @@ -274,8 +273,6 @@ protected PythonOperationInfo(AbstractOperation identifier) throws IOException { case CROSS_H: case CROSS_T: otherID = (Integer) receiver.getRecord(true); - operator = (byte[]) receiver.getRecord(); - meta = (String) receiver.getRecord(); tmpType = receiver.getRecord(); types = tmpType == null ? null : getForObject(tmpType); int cProjectCount = (Integer) receiver.getRecord(true); @@ -289,9 +286,6 @@ protected PythonOperationInfo(AbstractOperation identifier) throws IOException { break; case REDUCE: case GROUPREDUCE: - operator = (byte[]) receiver.getRecord(); - combineOperator = (byte[]) receiver.getRecord(); - meta = (String) receiver.getRecord(); tmpType = receiver.getRecord(); types = tmpType == null ? null : getForObject(tmpType); combine = (Boolean) receiver.getRecord(); @@ -303,8 +297,6 @@ protected PythonOperationInfo(AbstractOperation identifier) throws IOException { keys1 = tupleToIntArray((Tuple) receiver.getRecord(true)); keys2 = tupleToIntArray((Tuple) receiver.getRecord(true)); otherID = (Integer) receiver.getRecord(true); - operator = (byte[]) receiver.getRecord(); - meta = (String) receiver.getRecord(); tmpType = receiver.getRecord(); types = tmpType == null ? null : getForObject(tmpType); int jProjectCount = (Integer) receiver.getRecord(true); @@ -320,8 +312,6 @@ protected PythonOperationInfo(AbstractOperation identifier) throws IOException { case FLATMAP: case MAP: case FILTER: - operator = (byte[]) receiver.getRecord(); - meta = (String) receiver.getRecord(); tmpType = receiver.getRecord(); types = tmpType == null ? null : getForObject(tmpType); name = (String) receiver.getRecord(); @@ -344,7 +334,7 @@ protected DataSet applyCoGroupOperation(DataSet op1, DataSet op2, int[] firstKey op2, new Keys.ExpressionKeys(firstKeys, op1.getType()), new Keys.ExpressionKeys(secondKeys, op2.getType()), - new PythonCoGroup(info.setID, info.operator, info.types, info.meta), + new PythonCoGroup(info.setID, info.types), info.types, info.name); } @@ -353,13 +343,13 @@ protected DataSet applyCrossOperation(DataSet op1, DataSet op2, DatasizeHint mod switch (mode) { case NONE: return op1.cross(op2).name("PythonCrossPreStep") - .mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + .mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); case HUGE: return op1.crossWithHuge(op2).name("PythonCrossPreStep") - .mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + .mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); case TINY: return op1.crossWithTiny(op2).name("PythonCrossPreStep") - .mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + .mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); default: throw new IllegalArgumentException("Invalid Cross mode specified: " + mode); } @@ -367,25 +357,25 @@ protected DataSet applyCrossOperation(DataSet op1, DataSet op2, DatasizeHint mod @Override protected DataSet applyFilterOperation(DataSet op1, PythonOperationInfo info) { - return op1.mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + return op1.mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); } @Override protected DataSet applyFlatMapOperation(DataSet op1, PythonOperationInfo info) { - return op1.mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + return op1.mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); } @Override protected DataSet applyGroupReduceOperation(DataSet op1, PythonOperationInfo info) { if (info.combine) { - return op1.reduceGroup(new PythonCombineIdentity(info.setID, info.combineOperator, info.meta)) + return op1.reduceGroup(new PythonCombineIdentity(info.setID * -1)) .setCombinable(true).name("PythonCombine") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } else { return op1.reduceGroup(new PythonCombineIdentity()) .setCombinable(false).name("PythonGroupReducePreStep") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } } @@ -393,14 +383,14 @@ protected DataSet applyGroupReduceOperation(DataSet op1, PythonOperationInfo inf @Override protected DataSet applyGroupReduceOperation(UnsortedGrouping op1, PythonOperationInfo info) { if (info.combine) { - return op1.reduceGroup(new PythonCombineIdentity(info.setID, info.combineOperator, info.meta)) + return op1.reduceGroup(new PythonCombineIdentity(info.setID * -1)) .setCombinable(true).name("PythonCombine") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } else { return op1.reduceGroup(new PythonCombineIdentity()) .setCombinable(false).name("PythonGroupReducePreStep") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } } @@ -408,14 +398,14 @@ protected DataSet applyGroupReduceOperation(UnsortedGrouping op1, PythonOperatio @Override protected DataSet applyGroupReduceOperation(SortedGrouping op1, PythonOperationInfo info) { if (info.combine) { - return op1.reduceGroup(new PythonCombineIdentity(info.setID, info.combineOperator, info.meta)) + return op1.reduceGroup(new PythonCombineIdentity(info.setID * -1)) .setCombinable(true).name("PythonCombine") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } else { return op1.reduceGroup(new PythonCombineIdentity()) .setCombinable(false).name("PythonGroupReducePreStep") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } } @@ -425,13 +415,13 @@ protected DataSet applyJoinOperation(DataSet op1, DataSet op2, int[] firstKeys, switch (mode) { case NONE: return op1.join(op2).where(firstKeys).equalTo(secondKeys).name("PythonJoinPreStep") - .mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + .mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); case HUGE: return op1.joinWithHuge(op2).where(firstKeys).equalTo(secondKeys).name("PythonJoinPreStep") - .mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + .mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); case TINY: return op1.joinWithTiny(op2).where(firstKeys).equalTo(secondKeys).name("PythonJoinPreStep") - .mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + .mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); default: throw new IllegalArgumentException("Invalid join mode specified."); } @@ -439,33 +429,33 @@ protected DataSet applyJoinOperation(DataSet op1, DataSet op2, int[] firstKeys, @Override protected DataSet applyMapOperation(DataSet op1, PythonOperationInfo info) { - return op1.mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + return op1.mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); } @Override protected DataSet applyMapPartitionOperation(DataSet op1, PythonOperationInfo info) { - return op1.mapPartition(new PythonMapPartition(info.setID, info.operator, info.types, info.meta)).name(info.name); + return op1.mapPartition(new PythonMapPartition(info.setID, info.types)).name(info.name); } @Override protected DataSet applyReduceOperation(DataSet op1, PythonOperationInfo info) { return op1.reduceGroup(new PythonCombineIdentity()) .setCombinable(false).name("PythonReducePreStep") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } @Override protected DataSet applyReduceOperation(UnsortedGrouping op1, PythonOperationInfo info) { if (info.combine) { - return op1.reduceGroup(new PythonCombineIdentity(info.setID, info.combineOperator, info.meta)) + return op1.reduceGroup(new PythonCombineIdentity(info.setID * -1)) .setCombinable(true).name("PythonCombine") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } else { return op1.reduceGroup(new PythonCombineIdentity()) .setCombinable(false).name("PythonReducePreStep") - .mapPartition(new PythonMapPartition(info.setID * -1, info.operator, info.types, info.meta)) + .mapPartition(new PythonMapPartition(info.setID, info.types)) .name(info.name); } } diff --git a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCoGroup.java b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCoGroup.java index 01f18eb186893..26d554d110ac3 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCoGroup.java +++ b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCoGroup.java @@ -31,9 +31,9 @@ public class PythonCoGroup extends RichCoGroupFunction typeInformation; - public PythonCoGroup(int id, byte[] operator, TypeInformation typeInformation, String metaInformation) { + public PythonCoGroup(int id, TypeInformation typeInformation) { this.typeInformation = typeInformation; - streamer = new PythonStreamer(this, id, operator, metaInformation); + streamer = new PythonStreamer(this, id); } /** diff --git a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCombineIdentity.java b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCombineIdentity.java index 3395f073f556c..a8ff96cbfd337 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCombineIdentity.java +++ b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonCombineIdentity.java @@ -31,8 +31,8 @@ public PythonCombineIdentity() { streamer = null; } - public PythonCombineIdentity(int id, byte[] operator, String metaInformation) { - streamer = new PythonStreamer(this, id, operator, metaInformation); + public PythonCombineIdentity(int id) { + streamer = new PythonStreamer(this, id); } @Override diff --git a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonMapPartition.java b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonMapPartition.java index f582e3da2cefc..1f13e5cf0d7cb 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonMapPartition.java +++ b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/functions/PythonMapPartition.java @@ -31,9 +31,9 @@ public class PythonMapPartition extends RichMapPartitionFunction typeInformation; - public PythonMapPartition(int id, byte[] operator, TypeInformation typeInformation, String metaInformation) { + public PythonMapPartition(int id, TypeInformation typeInformation) { this.typeInformation = typeInformation; - streamer = new PythonStreamer(this, id, operator, metaInformation); + streamer = new PythonStreamer(this, id); } /** diff --git a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/streaming/PythonStreamer.java b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/streaming/PythonStreamer.java index 7dc2240c956cc..6d21c4c829607 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/streaming/PythonStreamer.java +++ b/flink-staging/flink-language-binding/flink-python/src/main/java/org/apache/flink/languagebinding/api/java/python/streaming/PythonStreamer.java @@ -13,10 +13,10 @@ package org.apache.flink.languagebinding.api.java.python.streaming; import java.io.IOException; +import java.io.OutputStream; import java.lang.reflect.Field; import org.apache.flink.api.common.functions.AbstractRichFunction; import static org.apache.flink.languagebinding.api.java.common.PlanBinder.DEBUG; -import static org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.FLINK_PYTHON_EXECUTOR_NAME; import static org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.FLINK_PYTHON_DC_ID; import static org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.FLINK_PYTHON_PLAN_NAME; import static org.apache.flink.languagebinding.api.java.common.PlanBinder.FLINK_TMP_DATA_DIR; @@ -30,24 +30,22 @@ * This streamer is used by functions to send/receive data to/from an external python process. */ public class PythonStreamer extends Streamer { - private final byte[] operator; private Process process; - private final String metaInformation; private final int id; private final boolean usePython3; private final boolean debug; private Thread shutdownThread; + private final String planArguments; private String inputFilePath; private String outputFilePath; - public PythonStreamer(AbstractRichFunction function, int id, byte[] operator, String metaInformation) { + public PythonStreamer(AbstractRichFunction function, int id) { super(function); - this.operator = operator; - this.metaInformation = metaInformation; this.id = id; this.usePython3 = PythonPlanBinder.usePython3; this.debug = DEBUG; + planArguments = PythonPlanBinder.arguments.toString(); } /** @@ -67,21 +65,8 @@ private void startPython() throws IOException { sender.open(inputFilePath); receiver.open(outputFilePath); - ProcessBuilder pb = new ProcessBuilder(); - String path = function.getRuntimeContext().getDistributedCache().getFile(FLINK_PYTHON_DC_ID).getAbsolutePath(); - String executorPath = path + FLINK_PYTHON_EXECUTOR_NAME; - String[] frag = metaInformation.split("\\|"); - StringBuilder importString = new StringBuilder(); - if (frag[0].contains("__main__")) { - importString.append("from "); - importString.append(FLINK_PYTHON_PLAN_NAME.substring(1, FLINK_PYTHON_PLAN_NAME.length() - 3)); - importString.append(" import "); - importString.append(frag[1]); - } else { - importString.append("import "); - importString.append(FLINK_PYTHON_PLAN_NAME.substring(1, FLINK_PYTHON_PLAN_NAME.length() - 3)); - } + String planPath = path + FLINK_PYTHON_PLAN_NAME; String pythonBinaryPath = usePython3 ? FLINK_PYTHON3_BINARY_PATH : FLINK_PYTHON2_BINARY_PATH; @@ -90,14 +75,13 @@ private void startPython() throws IOException { } catch (IOException ex) { throw new RuntimeException(pythonBinaryPath + " does not point to a valid python binary."); } - pb.command(pythonBinaryPath, "-O", "-B", executorPath, "" + server.getLocalPort()); if (debug) { socket.setSoTimeout(0); LOG.info("Waiting for Python Process : " + function.getRuntimeContext().getTaskName() - + " Run python /tmp/flink" + FLINK_PYTHON_EXECUTOR_NAME + " " + server.getLocalPort()); + + " Run python " + planPath + planArguments); } else { - process = pb.start(); + process = Runtime.getRuntime().exec(pythonBinaryPath + " -O -B " + planPath + planArguments); new StreamPrinter(process.getInputStream()).start(); new StreamPrinter(process.getErrorStream(), true, msg).start(); } @@ -114,31 +98,13 @@ public void run() { Runtime.getRuntime().addShutdownHook(shutdownThread); - socket = server.accept(); - in = socket.getInputStream(); - out = socket.getOutputStream(); - - byte[] opSize = new byte[4]; - putInt(opSize, 0, operator.length); - out.write(opSize, 0, 4); - out.write(operator, 0, operator.length); - - byte[] meta = importString.toString().getBytes("utf-8"); - putInt(opSize, 0, meta.length); - out.write(opSize, 0, 4); - out.write(meta, 0, meta.length); - - byte[] input = inputFilePath.getBytes("utf-8"); - putInt(opSize, 0, input.length); - out.write(opSize, 0, 4); - out.write(input, 0, input.length); - - byte[] output = outputFilePath.getBytes("utf-8"); - putInt(opSize, 0, output.length); - out.write(opSize, 0, 4); - out.write(output, 0, output.length); - - out.flush(); + OutputStream processOutput = process.getOutputStream(); + processOutput.write("operator\n".getBytes()); + processOutput.write(("" + server.getLocalPort() + "\n").getBytes()); + processOutput.write((id + "\n").getBytes()); + processOutput.write((inputFilePath + "\n").getBytes()); + processOutput.write((outputFilePath + "\n").getBytes()); + processOutput.flush(); try { // wait a bit to catch syntax errors Thread.sleep(2000); @@ -151,6 +117,10 @@ public void run() { } catch (IllegalThreadStateException ise) { //process still active -> start receiving data } } + + socket = server.accept(); + in = socket.getInputStream(); + out = socket.getOutputStream(); } /** diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__diff.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__diff.py deleted file mode 100644 index 79301a6c39651..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__diff.py +++ /dev/null @@ -1,229 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE - -""" -Module to show if an object has changed since it was memorised -""" - -import os -import sys -import types -try: - import numpy - HAS_NUMPY = True -except: - HAS_NUMPY = False -try: - import builtins -except ImportError: - import __builtin__ as builtins - -# memo of objects indexed by id to a tuple (attributes, sequence items) -# attributes is a dict indexed by attribute name to attribute id -# sequence items is either a list of ids, of a dictionary of keys to ids -memo = {} -id_to_obj = {} -# types that cannot have changing attributes -builtins_types = set((str, list, dict, set, frozenset, int)) -dont_memo = set(id(i) for i in (memo, sys.modules, sys.path_importer_cache, - os.environ, id_to_obj)) - - -def get_attrs(obj): - """ - Gets all the attributes of an object though its __dict__ or return None - """ - if type(obj) in builtins_types \ - or type(obj) is type and obj in builtins_types: - return - try: - return obj.__dict__ - except: - return - - -def get_seq(obj, cache={str: False, frozenset: False, list: True, set: True, - dict: True, tuple: True, type: False, - types.ModuleType: False, types.FunctionType: False, - types.BuiltinFunctionType: False}): - """ - Gets all the items in a sequence or return None - """ - o_type = type(obj) - hsattr = hasattr - if o_type in cache: - if cache[o_type]: - if hsattr(obj, "copy"): - return obj.copy() - return obj - elif HAS_NUMPY and o_type in (numpy.ndarray, numpy.ma.core.MaskedConstant): - if obj.shape and obj.size: - return obj - else: - return [] - elif hsattr(obj, "__contains__") and hsattr(obj, "__iter__") \ - and hsattr(obj, "__len__") and hsattr(o_type, "__contains__") \ - and hsattr(o_type, "__iter__") and hsattr(o_type, "__len__"): - cache[o_type] = True - if hsattr(obj, "copy"): - return obj.copy() - return obj - else: - cache[o_type] = False - return None - - -def memorise(obj, force=False): - """ - Adds an object to the memo, and recursively adds all the objects - attributes, and if it is a container, its items. Use force=True to update - an object already in the memo. Updating is not recursively done. - """ - obj_id = id(obj) - if obj_id in memo and not force or obj_id in dont_memo: - return - id_ = id - g = get_attrs(obj) - if g is None: - attrs_id = None - else: - attrs_id = dict((key,id_(value)) for key, value in g.items()) - - s = get_seq(obj) - if s is None: - seq_id = None - elif hasattr(s, "items"): - seq_id = dict((id_(key),id_(value)) for key, value in s.items()) - else: - seq_id = [id_(i) for i in s] - - memo[obj_id] = attrs_id, seq_id - id_to_obj[obj_id] = obj - mem = memorise - if g is not None: - [mem(value) for key, value in g.items()] - - if s is not None: - if hasattr(s, "items"): - [(mem(key), mem(item)) - for key, item in s.items()] - else: - [mem(item) for item in s] - - -def release_gone(): - itop, mp, src = id_to_obj.pop, memo.pop, sys.getrefcount - [(itop(id_), mp(id_)) for id_, obj in list(id_to_obj.items()) - if src(obj) < 4] - - -def whats_changed(obj, seen=None, simple=False, first=True): - """ - Check an object against the memo. Returns a list in the form - (attribute changes, container changed). Attribute changes is a dict of - attribute name to attribute value. container changed is a boolean. - If simple is true, just returns a boolean. None for either item means - that it has not been checked yet - """ - # Special cases - if first: - # ignore the _ variable, which only appears in interactive sessions - if "_" in builtins.__dict__: - del builtins._ - if seen is None: - seen = {} - - obj_id = id(obj) - - if obj_id in seen: - if simple: - return any(seen[obj_id]) - return seen[obj_id] - - # Safety checks - if obj_id in dont_memo: - seen[obj_id] = [{}, False] - if simple: - return False - return seen[obj_id] - elif obj_id not in memo: - if simple: - return True - else: - raise RuntimeError("Object not memorised " + str(obj)) - - seen[obj_id] = ({}, False) - - chngd = whats_changed - id_ = id - - # compare attributes - attrs = get_attrs(obj) - if attrs is None: - changed = {} - else: - obj_attrs = memo[obj_id][0] - obj_get = obj_attrs.get - changed = dict((key,None) for key in obj_attrs if key not in attrs) - for key, o in attrs.items(): - if id_(o) != obj_get(key, None) or chngd(o, seen, True, False): - changed[key] = o - - # compare sequence - items = get_seq(obj) - seq_diff = False - if items is not None: - obj_seq = memo[obj_id][1] - if len(items) != len(obj_seq): - seq_diff = True - elif hasattr(obj, "items"): # dict type obj - obj_get = obj_seq.get - for key, item in items.items(): - if id_(item) != obj_get(id_(key)) \ - or chngd(key, seen, True, False) \ - or chngd(item, seen, True, False): - seq_diff = True - break - else: - for i, j in zip(items, obj_seq): # list type obj - if id_(i) != j or chngd(i, seen, True, False): - seq_diff = True - break - seen[obj_id] = changed, seq_diff - if simple: - return changed or seq_diff - return changed, seq_diff - - -def has_changed(*args, **kwds): - kwds['simple'] = True # ignore simple if passed in - return whats_changed(*args, **kwds) - -__import__ = __import__ - - -def _imp(*args, **kwds): - """ - Replaces the default __import__, to allow a module to be memorised - before the user can change it - """ - before = set(sys.modules.keys()) - mod = __import__(*args, **kwds) - after = set(sys.modules.keys()).difference(before) - for m in after: - memorise(sys.modules[m]) - return mod - -builtins.__import__ = _imp -if hasattr(builtins, "_"): - del builtins._ - -# memorise all already imported modules. This implies that this must be -# imported first for any changes to be recorded -for mod in sys.modules.values(): - memorise(mod) -release_gone() diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__init__.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__init__.py deleted file mode 100644 index b03eda9c777ac..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/__init__.py +++ /dev/null @@ -1,73 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE - -from __future__ import absolute_import - -from .dill import dump, dumps, load, loads, dump_session, load_session, \ - Pickler, Unpickler, register, copy, pickle, pickles, \ - HIGHEST_PROTOCOL, DEFAULT_PROTOCOL, PicklingError, UnpicklingError, \ - HANDLE_FMODE, CONTENTS_FMODE, FILE_FMODE -from . import source, temp, detect - -# make sure "trace" is turned off -detect.trace(False) - -try: - from imp import reload -except ImportError: - pass - -# put the objects in order, if possible -try: - from collections import OrderedDict as odict -except ImportError: - try: - from ordereddict import OrderedDict as odict - except ImportError: - odict = dict -objects = odict() -# local import of dill._objects -#from . import _objects -#objects.update(_objects.succeeds) -#del _objects - -# local import of dill.objtypes -from . import objtypes as types - -def load_types(pickleable=True, unpickleable=True): - """load pickleable and/or unpickleable types to dill.types""" - # local import of dill.objects - from . import _objects - if pickleable: - objects.update(_objects.succeeds) - else: - [objects.pop(obj,None) for obj in _objects.succeeds] - if unpickleable: - objects.update(_objects.failures) - else: - [objects.pop(obj,None) for obj in _objects.failures] - objects.update(_objects.registered) - del _objects - # reset contents of types to 'empty' - [types.__dict__.pop(obj) for obj in list(types.__dict__.keys()) \ - if obj.find('Type') != -1] - # add corresponding types from objects to types - reload(types) - -def extend(use_dill=True): - '''add (or remove) dill types to/from pickle''' - from .dill import _revert_extension, _extend - if use_dill: _extend() - else: _revert_extension() - return - -extend() - -del absolute_import -del odict - -# end of file diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/_objects.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/_objects.py deleted file mode 100644 index b89bc0e1df62b..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/_objects.py +++ /dev/null @@ -1,530 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE -""" -all Python Standard Library objects (currently: CH 1-15 @ 2.7) -and some other common objects (i.e. numpy.ndarray) -""" - -__all__ = ['registered','failures','succeeds'] - -# helper imports -import warnings; warnings.filterwarnings("ignore", category=DeprecationWarning) -import sys -PY3 = (hex(sys.hexversion) >= '0x30000f0') -if PY3: - import queue as Queue - import dbm as anydbm -else: - import Queue - import anydbm - import sets # deprecated/removed - import mutex # removed -try: - from cStringIO import StringIO # has StringI and StringO types -except ImportError: # only has StringIO type - if PY3: - from io import BytesIO as StringIO - else: - from StringIO import StringIO -import re -import array -import collections -import codecs -import struct -import datetime -import calendar -import weakref -import pprint -import decimal -import functools -import itertools -import operator -import tempfile -import shelve -import zlib -import gzip -import zipfile -import tarfile -import xdrlib -import csv -import hashlib -import hmac -import os -import logging -import optparse -import curses -#import __hello__ -import threading -import socket -import contextlib -try: - import bz2 - import sqlite3 - if PY3: import dbm.ndbm as dbm - else: import dbm - HAS_ALL = True -except ImportError: # Ubuntu - HAS_ALL = False -try: - import ctypes - HAS_CTYPES = True -except ImportError: # MacPorts - HAS_CTYPES = False -from curses import textpad, panel - -# helper objects -class _class: - def _method(self): - pass -# @classmethod -# def _clsmethod(cls): #XXX: test me -# pass -# @staticmethod -# def _static(self): #XXX: test me -# pass -class _class2: - def __call__(self): - pass -_instance2 = _class2() -class _newclass(object): - def _method(self): - pass -# @classmethod -# def _clsmethod(cls): #XXX: test me -# pass -# @staticmethod -# def _static(self): #XXX: test me -# pass -def _function(x): yield x -def _function2(): - try: raise - except: - from sys import exc_info - e, er, tb = exc_info() - return er, tb -if HAS_CTYPES: - class _Struct(ctypes.Structure): - pass - _Struct._fields_ = [("_field", ctypes.c_int),("next", ctypes.POINTER(_Struct))] -_filedescrip, _tempfile = tempfile.mkstemp('r') # deleted in cleanup -_tmpf = tempfile.TemporaryFile('w') - -# put the objects in order, if possible -try: - from collections import OrderedDict as odict -except ImportError: - try: - from ordereddict import OrderedDict as odict - except ImportError: - odict = dict -# objects used by dill for type declaration -registered = d = odict() -# objects dill fails to pickle -failures = x = odict() -# all other type objects -succeeds = a = odict() - -# types module (part of CH 8) -a['BooleanType'] = bool(1) -a['BuiltinFunctionType'] = len -a['BuiltinMethodType'] = a['BuiltinFunctionType'] -a['BytesType'] = _bytes = codecs.latin_1_encode('\x00')[0] # bytes(1) -a['ClassType'] = _class -a['ComplexType'] = complex(1) -a['DictType'] = _dict = {} -a['DictionaryType'] = a['DictType'] -a['FloatType'] = float(1) -a['FunctionType'] = _function -a['InstanceType'] = _instance = _class() -a['IntType'] = _int = int(1) -a['ListType'] = _list = [] -a['NoneType'] = None -a['ObjectType'] = object() -a['StringType'] = _str = str(1) -a['TupleType'] = _tuple = () -a['TypeType'] = type -if PY3: - a['LongType'] = _int - a['UnicodeType'] = _str -else: - a['LongType'] = long(1) - a['UnicodeType'] = unicode(1) -# built-in constants (CH 4) -a['CopyrightType'] = copyright -# built-in types (CH 5) -a['ClassObjectType'] = _newclass # -a['ClassInstanceType'] = _newclass() # -a['SetType'] = _set = set() -a['FrozenSetType'] = frozenset() -# built-in exceptions (CH 6) -a['ExceptionType'] = _exception = _function2()[0] -# string services (CH 7) -a['SREPatternType'] = _srepattern = re.compile('') -# data types (CH 8) -a['ArrayType'] = array.array("f") -a['DequeType'] = collections.deque([0]) -a['DefaultDictType'] = collections.defaultdict(_function, _dict) -a['TZInfoType'] = datetime.tzinfo() -a['DateTimeType'] = datetime.datetime.today() -a['CalendarType'] = calendar.Calendar() -if not PY3: - a['SetsType'] = sets.Set() - a['ImmutableSetType'] = sets.ImmutableSet() - a['MutexType'] = mutex.mutex() -# numeric and mathematical types (CH 9) -a['DecimalType'] = decimal.Decimal(1) -a['CountType'] = itertools.count(0) -# data compression and archiving (CH 12) -a['TarInfoType'] = tarfile.TarInfo() -# generic operating system services (CH 15) -a['LoggerType'] = logging.getLogger() -a['FormatterType'] = logging.Formatter() # pickle ok -a['FilterType'] = logging.Filter() # pickle ok -a['LogRecordType'] = logging.makeLogRecord(_dict) # pickle ok -a['OptionParserType'] = _oparser = optparse.OptionParser() # pickle ok -a['OptionGroupType'] = optparse.OptionGroup(_oparser,"foo") # pickle ok -a['OptionType'] = optparse.Option('--foo') # pickle ok -if HAS_CTYPES: - a['CCharType'] = _cchar = ctypes.c_char() - a['CWCharType'] = ctypes.c_wchar() # fail == 2.6 - a['CByteType'] = ctypes.c_byte() - a['CUByteType'] = ctypes.c_ubyte() - a['CShortType'] = ctypes.c_short() - a['CUShortType'] = ctypes.c_ushort() - a['CIntType'] = ctypes.c_int() - a['CUIntType'] = ctypes.c_uint() - a['CLongType'] = ctypes.c_long() - a['CULongType'] = ctypes.c_ulong() - a['CLongLongType'] = ctypes.c_longlong() - a['CULongLongType'] = ctypes.c_ulonglong() - a['CFloatType'] = ctypes.c_float() - a['CDoubleType'] = ctypes.c_double() - a['CSizeTType'] = ctypes.c_size_t() - a['CLibraryLoaderType'] = ctypes.cdll - a['StructureType'] = _Struct - a['BigEndianStructureType'] = ctypes.BigEndianStructure() -#NOTE: also LittleEndianStructureType and UnionType... abstract classes -#NOTE: remember for ctypesobj.contents creates a new python object -#NOTE: ctypes.c_int._objects is memberdescriptor for object's __dict__ -#NOTE: base class of all ctypes data types is non-public _CData - -try: # python 2.6 - import fractions - import number - import io - from io import StringIO as TextIO - # built-in functions (CH 2) - a['ByteArrayType'] = bytearray([1]) - # numeric and mathematical types (CH 9) - a['FractionType'] = fractions.Fraction() - a['NumberType'] = numbers.Number() - # generic operating system services (CH 15) - a['IOBaseType'] = io.IOBase() - a['RawIOBaseType'] = io.RawIOBase() - a['TextIOBaseType'] = io.TextIOBase() - a['BufferedIOBaseType'] = io.BufferedIOBase() - a['UnicodeIOType'] = TextIO() # the new StringIO - a['LoggingAdapterType'] = logging.LoggingAdapter(_logger,_dict) # pickle ok - if HAS_CTYPES: - a['CBoolType'] = ctypes.c_bool(1) - a['CLongDoubleType'] = ctypes.c_longdouble() -except ImportError: - pass -try: # python 2.7 - import argparse - # data types (CH 8) - a['OrderedDictType'] = collections.OrderedDict(_dict) - a['CounterType'] = collections.Counter(_dict) - if HAS_CTYPES: - a['CSSizeTType'] = ctypes.c_ssize_t() - # generic operating system services (CH 15) - a['NullHandlerType'] = logging.NullHandler() # pickle ok # new 2.7 - a['ArgParseFileType'] = argparse.FileType() # pickle ok -#except AttributeError: -except ImportError: - pass - -# -- pickle fails on all below here ----------------------------------------- -# types module (part of CH 8) -a['CodeType'] = compile('','','exec') -a['DictProxyType'] = type.__dict__ -a['DictProxyType2'] = _newclass.__dict__ -a['EllipsisType'] = Ellipsis -a['ClosedFileType'] = open(os.devnull, 'wb', buffering=0).close() -a['GetSetDescriptorType'] = array.array.typecode -a['LambdaType'] = _lambda = lambda x: lambda y: x #XXX: works when not imported! -a['MemberDescriptorType'] = type.__dict__['__weakrefoffset__'] -a['MemberDescriptorType2'] = datetime.timedelta.days -a['MethodType'] = _method = _class()._method #XXX: works when not imported! -a['ModuleType'] = datetime -a['NotImplementedType'] = NotImplemented -a['SliceType'] = slice(1) -a['UnboundMethodType'] = _class._method #XXX: works when not imported! -a['TextWrapperType'] = open(os.devnull, 'r') # same as mode='w','w+','r+' -a['BufferedRandomType'] = open(os.devnull, 'r+b') # same as mode='w+b' -a['BufferedReaderType'] = open(os.devnull, 'rb') # (default: buffering=-1) -a['BufferedWriterType'] = open(os.devnull, 'wb') -try: # oddities: deprecated - from _pyio import open as _open - a['PyTextWrapperType'] = _open(os.devnull, 'r', buffering=-1) - a['PyBufferedRandomType'] = _open(os.devnull, 'r+b', buffering=-1) - a['PyBufferedReaderType'] = _open(os.devnull, 'rb', buffering=-1) - a['PyBufferedWriterType'] = _open(os.devnull, 'wb', buffering=-1) -except ImportError: - pass -# other (concrete) object types -if PY3: - d['CellType'] = (_lambda)(0).__closure__[0] - a['XRangeType'] = _xrange = range(1) -else: - d['CellType'] = (_lambda)(0).func_closure[0] - a['XRangeType'] = _xrange = xrange(1) -d['MethodDescriptorType'] = type.__dict__['mro'] -d['WrapperDescriptorType'] = type.__repr__ -a['WrapperDescriptorType2'] = type.__dict__['__module__'] -# built-in functions (CH 2) -if PY3: _methodwrap = (1).__lt__ -else: _methodwrap = (1).__cmp__ -d['MethodWrapperType'] = _methodwrap -a['StaticMethodType'] = staticmethod(_method) -a['ClassMethodType'] = classmethod(_method) -a['PropertyType'] = property() -d['SuperType'] = super(Exception, _exception) -# string services (CH 7) -if PY3: _in = _bytes -else: _in = _str -a['InputType'] = _cstrI = StringIO(_in) -a['OutputType'] = _cstrO = StringIO() -# data types (CH 8) -a['WeakKeyDictionaryType'] = weakref.WeakKeyDictionary() -a['WeakValueDictionaryType'] = weakref.WeakValueDictionary() -a['ReferenceType'] = weakref.ref(_instance) -a['DeadReferenceType'] = weakref.ref(_class()) -a['ProxyType'] = weakref.proxy(_instance) -a['DeadProxyType'] = weakref.proxy(_class()) -a['CallableProxyType'] = weakref.proxy(_instance2) -a['DeadCallableProxyType'] = weakref.proxy(_class2()) -a['QueueType'] = Queue.Queue() -# numeric and mathematical types (CH 9) -d['PartialType'] = functools.partial(int,base=2) -if PY3: - a['IzipType'] = zip('0','1') -else: - a['IzipType'] = itertools.izip('0','1') -a['ChainType'] = itertools.chain('0','1') -d['ItemGetterType'] = operator.itemgetter(0) -d['AttrGetterType'] = operator.attrgetter('__repr__') -# file and directory access (CH 10) -if PY3: _fileW = _cstrO -else: _fileW = _tmpf -# data persistence (CH 11) -if HAS_ALL: - a['ConnectionType'] = _conn = sqlite3.connect(':memory:') - a['CursorType'] = _conn.cursor() -a['ShelveType'] = shelve.Shelf({}) -# data compression and archiving (CH 12) -if HAS_ALL: - a['BZ2FileType'] = bz2.BZ2File(os.devnull) #FIXME: fail >= 3.3 - a['BZ2CompressorType'] = bz2.BZ2Compressor() - a['BZ2DecompressorType'] = bz2.BZ2Decompressor() -#a['ZipFileType'] = _zip = zipfile.ZipFile(os.devnull,'w') #FIXME: fail >= 3.2 -#_zip.write(_tempfile,'x') [causes annoying warning/error printed on import] -#a['ZipInfoType'] = _zip.getinfo('x') -a['TarFileType'] = tarfile.open(fileobj=_fileW,mode='w') -# file formats (CH 13) -a['DialectType'] = csv.get_dialect('excel') -a['PackerType'] = xdrlib.Packer() -# optional operating system services (CH 16) -a['LockType'] = threading.Lock() -a['RLockType'] = threading.RLock() -# generic operating system services (CH 15) # also closed/open and r/w/etc... -a['NamedLoggerType'] = _logger = logging.getLogger(__name__) #FIXME: fail >= 3.2 and <= 2.6 -#a['FrozenModuleType'] = __hello__ #FIXME: prints "Hello world..." -# interprocess communication (CH 17) -if PY3: - a['SocketType'] = _socket = socket.socket() #FIXME: fail >= 3.3 - a['SocketPairType'] = socket.socketpair()[0] #FIXME: fail >= 3.3 -else: - a['SocketType'] = _socket = socket.socket() - a['SocketPairType'] = _socket._sock -# python runtime services (CH 27) -if PY3: - a['GeneratorContextManagerType'] = contextlib.contextmanager(max)([1]) -else: - a['GeneratorContextManagerType'] = contextlib.GeneratorContextManager(max) - -try: # ipython - __IPYTHON__ is True # is ipython -except NameError: - # built-in constants (CH 4) - a['QuitterType'] = quit - d['ExitType'] = a['QuitterType'] -try: # numpy - from numpy import ufunc as _numpy_ufunc - from numpy import array as _numpy_array - from numpy import int32 as _numpy_int32 - a['NumpyUfuncType'] = _numpy_ufunc - a['NumpyArrayType'] = _numpy_array - a['NumpyInt32Type'] = _numpy_int32 -except ImportError: - pass -try: # python 2.6 - # numeric and mathematical types (CH 9) - a['ProductType'] = itertools.product('0','1') - # generic operating system services (CH 15) - a['FileHandlerType'] = logging.FileHandler(os.devnull) #FIXME: fail >= 3.2 and <= 2.6 - a['RotatingFileHandlerType'] = logging.handlers.RotatingFileHandler(os.devnull) - a['SocketHandlerType'] = logging.handlers.SocketHandler('localhost',514) - a['MemoryHandlerType'] = logging.handlers.MemoryHandler(1) -except AttributeError: - pass -try: # python 2.7 - # data types (CH 8) - a['WeakSetType'] = weakref.WeakSet() # 2.7 -# # generic operating system services (CH 15) [errors when dill is imported] -# a['ArgumentParserType'] = _parser = argparse.ArgumentParser('PROG') -# a['NamespaceType'] = _parser.parse_args() # pickle ok -# a['SubParsersActionType'] = _parser.add_subparsers() -# a['MutuallyExclusiveGroupType'] = _parser.add_mutually_exclusive_group() -# a['ArgumentGroupType'] = _parser.add_argument_group() -except AttributeError: - pass - -# -- dill fails in some versions below here --------------------------------- -# types module (part of CH 8) -a['FileType'] = open(os.devnull, 'rb', buffering=0) # same 'wb','wb+','rb+' -# FIXME: FileType fails >= 3.1 -# built-in functions (CH 2) -a['ListIteratorType'] = iter(_list) # empty vs non-empty FIXME: fail < 3.2 -a['TupleIteratorType']= iter(_tuple) # empty vs non-empty FIXME: fail < 3.2 -a['XRangeIteratorType'] = iter(_xrange) # empty vs non-empty FIXME: fail < 3.2 -# data types (CH 8) -a['PrettyPrinterType'] = pprint.PrettyPrinter() #FIXME: fail >= 3.2 and == 2.5 -# numeric and mathematical types (CH 9) -a['CycleType'] = itertools.cycle('0') #FIXME: fail < 3.2 -# file and directory access (CH 10) -a['TemporaryFileType'] = _tmpf #FIXME: fail >= 3.2 and == 2.5 -# data compression and archiving (CH 12) -a['GzipFileType'] = gzip.GzipFile(fileobj=_fileW) #FIXME: fail > 3.2 and <= 2.6 -# generic operating system services (CH 15) -a['StreamHandlerType'] = logging.StreamHandler() #FIXME: fail >= 3.2 and == 2.5 -try: # python 2.6 - # numeric and mathematical types (CH 9) - a['PermutationsType'] = itertools.permutations('0') #FIXME: fail < 3.2 - a['CombinationsType'] = itertools.combinations('0',1) #FIXME: fail < 3.2 -except AttributeError: - pass -try: # python 2.7 - # numeric and mathematical types (CH 9) - a['RepeatType'] = itertools.repeat(0) #FIXME: fail < 3.2 - a['CompressType'] = itertools.compress('0',[1]) #FIXME: fail < 3.2 - #XXX: ...and etc -except AttributeError: - pass - -# -- dill fails on all below here ------------------------------------------- -# types module (part of CH 8) -x['GeneratorType'] = _generator = _function(1) #XXX: priority -x['FrameType'] = _generator.gi_frame #XXX: inspect.currentframe() -x['TracebackType'] = _function2()[1] #(see: inspect.getouterframes,getframeinfo) -# other (concrete) object types -# (also: Capsule / CObject ?) -# built-in functions (CH 2) -x['SetIteratorType'] = iter(_set) #XXX: empty vs non-empty -# built-in types (CH 5) -if PY3: - x['DictionaryItemIteratorType'] = iter(type.__dict__.items()) - x['DictionaryKeyIteratorType'] = iter(type.__dict__.keys()) - x['DictionaryValueIteratorType'] = iter(type.__dict__.values()) -else: - x['DictionaryItemIteratorType'] = type.__dict__.iteritems() - x['DictionaryKeyIteratorType'] = type.__dict__.iterkeys() - x['DictionaryValueIteratorType'] = type.__dict__.itervalues() -# string services (CH 7) -x['StructType'] = struct.Struct('c') -x['CallableIteratorType'] = _srepattern.finditer('') -x['SREMatchType'] = _srepattern.match('') -x['SREScannerType'] = _srepattern.scanner('') -x['StreamReader'] = codecs.StreamReader(_cstrI) #XXX: ... and etc -# python object persistence (CH 11) -# x['DbShelveType'] = shelve.open('foo','n')#,protocol=2) #XXX: delete foo -if HAS_ALL: - x['DbmType'] = dbm.open(_tempfile,'n') -# x['DbCursorType'] = _dbcursor = anydbm.open('foo','n') #XXX: delete foo -# x['DbType'] = _dbcursor.db -# data compression and archiving (CH 12) -x['ZlibCompressType'] = zlib.compressobj() -x['ZlibDecompressType'] = zlib.decompressobj() -# file formats (CH 13) -x['CSVReaderType'] = csv.reader(_cstrI) -x['CSVWriterType'] = csv.writer(_cstrO) -x['CSVDictReaderType'] = csv.DictReader(_cstrI) -x['CSVDictWriterType'] = csv.DictWriter(_cstrO,{}) -# cryptographic services (CH 14) -x['HashType'] = hashlib.md5() -x['HMACType'] = hmac.new(_in) -# generic operating system services (CH 15) -#x['CursesWindowType'] = _curwin = curses.initscr() #FIXME: messes up tty -#x['CursesTextPadType'] = textpad.Textbox(_curwin) -#x['CursesPanelType'] = panel.new_panel(_curwin) -if HAS_CTYPES: - x['CCharPType'] = ctypes.c_char_p() - x['CWCharPType'] = ctypes.c_wchar_p() - x['CVoidPType'] = ctypes.c_void_p() - x['CDLLType'] = _cdll = ctypes.CDLL(None) - x['PyDLLType'] = _pydll = ctypes.pythonapi - x['FuncPtrType'] = _cdll._FuncPtr() - x['CCharArrayType'] = ctypes.create_string_buffer(1) - x['CWCharArrayType'] = ctypes.create_unicode_buffer(1) - x['CParamType'] = ctypes.byref(_cchar) - x['LPCCharType'] = ctypes.pointer(_cchar) - x['LPCCharObjType'] = _lpchar = ctypes.POINTER(ctypes.c_char) - x['NullPtrType'] = _lpchar() - x['NullPyObjectType'] = ctypes.py_object() - x['PyObjectType'] = ctypes.py_object(1) - x['FieldType'] = _field = _Struct._field - x['CFUNCTYPEType'] = _cfunc = ctypes.CFUNCTYPE(ctypes.c_char) - x['CFunctionType'] = _cfunc(str) -try: # python 2.6 - # numeric and mathematical types (CH 9) - x['MethodCallerType'] = operator.methodcaller('mro') # 2.6 -except AttributeError: - pass -try: # python 2.7 - # built-in types (CH 5) - x['MemoryType'] = memoryview(_in) # 2.7 - x['MemoryType2'] = memoryview(bytearray(_in)) # 2.7 - if PY3: - x['DictItemsType'] = _dict.items() # 2.7 - x['DictKeysType'] = _dict.keys() # 2.7 - x['DictValuesType'] = _dict.values() # 2.7 - else: - x['DictItemsType'] = _dict.viewitems() # 2.7 - x['DictKeysType'] = _dict.viewkeys() # 2.7 - x['DictValuesType'] = _dict.viewvalues() # 2.7 - # generic operating system services (CH 15) - x['RawTextHelpFormatterType'] = argparse.RawTextHelpFormatter('PROG') - x['RawDescriptionHelpFormatterType'] = argparse.RawDescriptionHelpFormatter('PROG') - x['ArgDefaultsHelpFormatterType'] = argparse.ArgumentDefaultsHelpFormatter('PROG') -except NameError: - pass -try: # python 2.7 (and not 3.1) - x['CmpKeyType'] = _cmpkey = functools.cmp_to_key(_methodwrap) # 2.7, >=3.2 - x['CmpKeyObjType'] = _cmpkey('0') #2.7, >=3.2 -except AttributeError: - pass -if PY3: # oddities: removed, etc - x['BufferType'] = x['MemoryType'] -else: - x['BufferType'] = buffer('') - -# -- cleanup ---------------------------------------------------------------- -a.update(d) # registered also succeed -os.remove(_tempfile) - - -# EOF diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/detect.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/detect.py deleted file mode 100644 index 749a57395719b..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/detect.py +++ /dev/null @@ -1,222 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE -""" -Methods for detecting objects leading to pickling failures. -""" - -from __future__ import absolute_import -from inspect import ismethod, isfunction, istraceback, isframe, iscode -from .pointers import parent, reference, at, parents, children - -from .dill import _trace as trace -from .dill import PY3 - -def outermost(func): # is analogous to getsource(func,enclosing=True) - """get outermost enclosing object (i.e. the outer function in a closure) - - NOTE: this is the object-equivalent of getsource(func, enclosing=True) - """ - if PY3: - if ismethod(func): - _globals = func.__func__.__globals__ or {} - elif isfunction(func): - _globals = func.__globals__ or {} - else: - return #XXX: or raise? no matches - _globals = _globals.items() - else: - if ismethod(func): - _globals = func.im_func.func_globals or {} - elif isfunction(func): - _globals = func.func_globals or {} - else: - return #XXX: or raise? no matches - _globals = _globals.iteritems() - # get the enclosing source - from .source import getsourcelines - try: lines,lnum = getsourcelines(func, enclosing=True) - except: #TypeError, IOError - lines,lnum = [],None - code = ''.join(lines) - # get all possible names,objects that are named in the enclosing source - _locals = ((name,obj) for (name,obj) in _globals if name in code) - # now only save the objects that generate the enclosing block - for name,obj in _locals: #XXX: don't really need 'name' - try: - if getsourcelines(obj) == (lines,lnum): return obj - except: #TypeError, IOError - pass - return #XXX: or raise? no matches - -def nestedcode(func): #XXX: or return dict of {co_name: co} ? - """get the code objects for any nested functions (e.g. in a closure)""" - func = code(func) - if not iscode(func): return [] #XXX: or raise? no matches - nested = [] - for co in func.co_consts: - if co is None: continue - co = code(co) - if co: nested.append(co) - return nested - -def code(func): - '''get the code object for the given function or method - - NOTE: use dill.source.getsource(CODEOBJ) to get the source code - ''' - if PY3: - im_func = '__func__' - func_code = '__code__' - else: - im_func = 'im_func' - func_code = 'func_code' - if ismethod(func): func = getattr(func, im_func) - if isfunction(func): func = getattr(func, func_code) - if istraceback(func): func = func.tb_frame - if isframe(func): func = func.f_code - if iscode(func): return func - return - -def nested(func): #XXX: or return dict of {__name__: obj} ? - """get any functions inside of func (e.g. inner functions in a closure) - - NOTE: results may differ if the function has been executed or not. - If len(nestedcode(func)) > len(nested(func)), try calling func(). - If possible, python builds code objects, but delays building functions - until func() is called. - """ - if PY3: - att1 = '__code__' - att0 = '__func__' - else: - att1 = 'func_code' # functions - att0 = 'im_func' # methods - - import gc - funcs = [] - # get the code objects, and try to track down by referrence - for co in nestedcode(func): - # look for function objects that refer to the code object - for obj in gc.get_referrers(co): - # get methods - _ = getattr(obj, att0, None) # ismethod - if getattr(_, att1, None) is co: funcs.append(obj) - # get functions - elif getattr(obj, att1, None) is co: funcs.append(obj) - # get frame objects - elif getattr(obj, 'f_code', None) is co: funcs.append(obj) - # get code objects - elif hasattr(obj, 'co_code') and obj is co: funcs.append(obj) -# frameobjs => func.func_code.co_varnames not in func.func_code.co_cellvars -# funcobjs => func.func_code.co_cellvars not in func.func_code.co_varnames -# frameobjs are not found, however funcobjs are... -# (see: test_mixins.quad ... and test_mixins.wtf) -# after execution, code objects get compiled, and them may be found by gc - return funcs - - -def freevars(func): - """get objects defined in enclosing code that are referred to by func - - returns a dict of {name:object}""" - if PY3: - im_func = '__func__' - func_code = '__code__' - func_closure = '__closure__' - else: - im_func = 'im_func' - func_code = 'func_code' - func_closure = 'func_closure' - if ismethod(func): func = getattr(func, im_func) - if isfunction(func): - closures = getattr(func, func_closure) or () - func = getattr(func, func_code).co_freevars # get freevars - else: - return {} - return dict((name,c.cell_contents) for (name,c) in zip(func,closures)) - -def globalvars(func): - """get objects defined in global scope that are referred to by func - - return a dict of {name:object}""" - if PY3: - im_func = '__func__' - func_code = '__code__' - func_globals = '__globals__' - else: - im_func = 'im_func' - func_code = 'func_code' - func_globals = 'func_globals' - if ismethod(func): func = getattr(func, im_func) - if isfunction(func): - globs = getattr(func, func_globals) or {} - func = getattr(func, func_code).co_names # get names - else: - return {} - #NOTE: if name not in func_globals, then we skip it... - return dict((name,globs[name]) for name in func if name in globs) - -def varnames(func): - """get names of variables defined by func - - returns a tuple (local vars, local vars referrenced by nested functions)""" - func = code(func) - if not iscode(func): - return () #XXX: better ((),())? or None? - return func.co_varnames, func.co_cellvars - - -def baditems(obj, exact=False, safe=False): #XXX: obj=globals() ? - """get items in object that fail to pickle""" - if not hasattr(obj,'__iter__'): # is not iterable - return [j for j in (badobjects(obj,0,exact,safe),) if j is not None] - obj = obj.values() if getattr(obj,'values',None) else obj - _obj = [] # can't use a set, as items may be unhashable - [_obj.append(badobjects(i,0,exact,safe)) for i in obj if i not in _obj] - return [j for j in _obj if j is not None] - - -def badobjects(obj, depth=0, exact=False, safe=False): - """get objects that fail to pickle""" - from dill import pickles - if not depth: - if pickles(obj,exact,safe): return None - return obj - return dict(((attr, badobjects(getattr(obj,attr),depth-1,exact,safe)) \ - for attr in dir(obj) if not pickles(getattr(obj,attr),exact,safe))) - -def badtypes(obj, depth=0, exact=False, safe=False): - """get types for objects that fail to pickle""" - from dill import pickles - if not depth: - if pickles(obj,exact,safe): return None - return type(obj) - return dict(((attr, badtypes(getattr(obj,attr),depth-1,exact,safe)) \ - for attr in dir(obj) if not pickles(getattr(obj,attr),exact,safe))) - -def errors(obj, depth=0, exact=False, safe=False): - """get errors for objects that fail to pickle""" - from dill import pickles, copy - if not depth: - try: - pik = copy(obj) - if exact: - assert pik == obj, \ - "Unpickling produces %s instead of %s" % (pik,obj) - assert type(pik) == type(obj), \ - "Unpickling produces %s instead of %s" % (type(pik),type(obj)) - return None - except Exception: - import sys - return sys.exc_info()[1] - return dict(((attr, errors(getattr(obj,attr),depth-1,exact,safe)) \ - for attr in dir(obj) if not pickles(getattr(obj,attr),exact,safe))) - -del absolute_import - - -# EOF diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/dill.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/dill.py deleted file mode 100644 index cddb9ca122a67..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/dill.py +++ /dev/null @@ -1,1034 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE -""" -dill: a utility for serialization of python objects - -Based on code written by Oren Tirosh and Armin Ronacher. -Extended to a (near) full set of the builtin types (in types module), -and coded to the pickle interface, by . -Initial port to python3 by Jonathan Dobson, continued by mmckerns. -Test against "all" python types (Std. Lib. CH 1-15 @ 2.7) by mmckerns. -Test against CH16+ Std. Lib. ... TBD. -""" -__all__ = ['dump','dumps','load','loads','dump_session','load_session', - 'Pickler','Unpickler','register','copy','pickle','pickles', - 'HIGHEST_PROTOCOL','DEFAULT_PROTOCOL','PicklingError', - 'UnpicklingError','HANDLE_FMODE','CONTENTS_FMODE','FILE_FMODE'] - -import logging -log = logging.getLogger("dill") -log.addHandler(logging.StreamHandler()) -def _trace(boolean): - """print a trace through the stack when pickling; useful for debugging""" - if boolean: log.setLevel(logging.INFO) - else: log.setLevel(logging.WARN) - return - -import os -import sys -diff = None -_use_diff = False -PY3 = sys.version_info[0] == 3 -if PY3: #XXX: get types from dill.objtypes ? - import builtins as __builtin__ - from pickle import _Pickler as StockPickler, _Unpickler as StockUnpickler - from _thread import LockType - #from io import IOBase - from types import CodeType, FunctionType, MethodType, GeneratorType, \ - TracebackType, FrameType, ModuleType, BuiltinMethodType - BufferType = memoryview #XXX: unregistered - ClassType = type # no 'old-style' classes - EllipsisType = type(Ellipsis) - #FileType = IOBase - NotImplementedType = type(NotImplemented) - SliceType = slice - TypeType = type # 'new-style' classes #XXX: unregistered - XRangeType = range - DictProxyType = type(object.__dict__) -else: - import __builtin__ - from pickle import Pickler as StockPickler, Unpickler as StockUnpickler - from thread import LockType - from types import CodeType, FunctionType, ClassType, MethodType, \ - GeneratorType, DictProxyType, XRangeType, SliceType, TracebackType, \ - NotImplementedType, EllipsisType, FrameType, ModuleType, \ - BufferType, BuiltinMethodType, TypeType -from pickle import HIGHEST_PROTOCOL, PicklingError, UnpicklingError -try: - from pickle import DEFAULT_PROTOCOL -except ImportError: - DEFAULT_PROTOCOL = HIGHEST_PROTOCOL -import __main__ as _main_module -import marshal -import gc -# import zlib -from weakref import ReferenceType, ProxyType, CallableProxyType -from functools import partial -from operator import itemgetter, attrgetter -# new in python2.5 -if sys.hexversion >= 0x20500f0: - from types import MemberDescriptorType, GetSetDescriptorType -# new in python3.3 -if sys.hexversion < 0x03030000: - FileNotFoundError = IOError -try: - import ctypes - HAS_CTYPES = True -except ImportError: - HAS_CTYPES = False -try: - from numpy import ufunc as NumpyUfuncType - from numpy import ndarray as NumpyArrayType - def ndarrayinstance(obj): - try: - if not isinstance(obj, NumpyArrayType): return False - except ReferenceError: return False # handle 'R3' weakref in 3.x - # verify that __reduce__ has not been overridden - NumpyInstance = NumpyArrayType((0,),'int8') - if id(obj.__reduce_ex__) == id(NumpyInstance.__reduce_ex__) and \ - id(obj.__reduce__) == id(NumpyInstance.__reduce__): return True - return False -except ImportError: - NumpyUfuncType = None - NumpyArrayType = None - def ndarrayinstance(obj): return False - -# make sure to add these 'hand-built' types to _typemap -if PY3: - CellType = type((lambda x: lambda y: x)(0).__closure__[0]) -else: - CellType = type((lambda x: lambda y: x)(0).func_closure[0]) -WrapperDescriptorType = type(type.__repr__) -MethodDescriptorType = type(type.__dict__['mro']) -MethodWrapperType = type([].__repr__) -PartialType = type(partial(int,base=2)) -SuperType = type(super(Exception, TypeError())) -ItemGetterType = type(itemgetter(0)) -AttrGetterType = type(attrgetter('__repr__')) -FileType = type(open(os.devnull, 'rb', buffering=0)) -TextWrapperType = type(open(os.devnull, 'r', buffering=-1)) -BufferedRandomType = type(open(os.devnull, 'r+b', buffering=-1)) -BufferedReaderType = type(open(os.devnull, 'rb', buffering=-1)) -BufferedWriterType = type(open(os.devnull, 'wb', buffering=-1)) -try: - from _pyio import open as _open - PyTextWrapperType = type(_open(os.devnull, 'r', buffering=-1)) - PyBufferedRandomType = type(_open(os.devnull, 'r+b', buffering=-1)) - PyBufferedReaderType = type(_open(os.devnull, 'rb', buffering=-1)) - PyBufferedWriterType = type(_open(os.devnull, 'wb', buffering=-1)) -except ImportError: - PyTextWrapperType = PyBufferedRandomType = PyBufferedReaderType = PyBufferedWriterType = None -try: - from cStringIO import StringIO, InputType, OutputType -except ImportError: - if PY3: - from io import BytesIO as StringIO - else: - from StringIO import StringIO - InputType = OutputType = None -try: - __IPYTHON__ is True # is ipython - ExitType = None # IPython.core.autocall.ExitAutocall - singletontypes = ['exit', 'quit', 'get_ipython'] -except NameError: - try: ExitType = type(exit) # apparently 'exit' can be removed - except NameError: ExitType = None - singletontypes = [] - -### File modes -# Pickles the file handle, preserving mode. The position of the unpickled -# object is as for a new file handle. -HANDLE_FMODE = 0 -# Pickles the file contents, creating a new file if on load the file does -# not exist. The position = min(pickled position, EOF) and mode is chosen -# as such that "best" preserves behavior of the original file. -CONTENTS_FMODE = 1 -# Pickles the entire file (handle and contents), preserving mode and position. -FILE_FMODE = 2 - -### Shorthands (modified from python2.5/lib/pickle.py) -def copy(obj, *args, **kwds): - """use pickling to 'copy' an object""" - return loads(dumps(obj, *args, **kwds)) - -def dump(obj, file, protocol=None, byref=False, fmode=HANDLE_FMODE):#, strictio=False): - """pickle an object to a file""" - strictio = False #FIXME: strict=True needs cleanup - if protocol is None: protocol = DEFAULT_PROTOCOL - pik = Pickler(file, protocol) - pik._main_module = _main_module - # save settings - _byref = pik._byref - _strictio = pik._strictio - _fmode = pik._fmode - # apply kwd settings - pik._byref = bool(byref) - pik._strictio = bool(strictio) - pik._fmode = fmode - # hack to catch subclassed numpy array instances - if NumpyArrayType and ndarrayinstance(obj): - @register(type(obj)) - def save_numpy_array(pickler, obj): - log.info("Nu: (%s, %s)" % (obj.shape,obj.dtype)) - npdict = getattr(obj, '__dict__', None) - f, args, state = obj.__reduce__() - pik.save_reduce(_create_array, (f, args, state, npdict), obj=obj) - return - # end hack - pik.dump(obj) - # return to saved settings - pik._byref = _byref - pik._strictio = _strictio - pik._fmode = _fmode - return - -def dumps(obj, protocol=None, byref=False, fmode=HANDLE_FMODE):#, strictio=False): - """pickle an object to a string""" - file = StringIO() - dump(obj, file, protocol, byref, fmode)#, strictio) - return file.getvalue() - -def load(file): - """unpickle an object from a file""" - pik = Unpickler(file) - pik._main_module = _main_module - obj = pik.load() - if type(obj).__module__ == _main_module.__name__: # point obj class to main - try: obj.__class__ == getattr(pik._main_module, type(obj).__name__) - except AttributeError: pass # defined in a file - #_main_module.__dict__.update(obj.__dict__) #XXX: should update globals ? - return obj - -def loads(str): - """unpickle an object from a string""" - file = StringIO(str) - return load(file) - -# def dumpzs(obj, protocol=None): -# """pickle an object to a compressed string""" -# return zlib.compress(dumps(obj, protocol)) - -# def loadzs(str): -# """unpickle an object from a compressed string""" -# return loads(zlib.decompress(str)) - -### End: Shorthands ### - -### Pickle the Interpreter Session -def dump_session(filename='/tmp/session.pkl', main_module=_main_module): - """pickle the current state of __main__ to a file""" - f = open(filename, 'wb') - try: - pickler = Pickler(f, 2) - pickler._main_module = main_module - _byref = pickler._byref - pickler._byref = False # disable pickling by name reference - pickler._session = True # is best indicator of when pickling a session - pickler.dump(main_module) - pickler._session = False - pickler._byref = _byref - finally: - f.close() - return - -def load_session(filename='/tmp/session.pkl', main_module=_main_module): - """update the __main__ module with the state from the session file""" - f = open(filename, 'rb') - try: - unpickler = Unpickler(f) - unpickler._main_module = main_module - unpickler._session = True - module = unpickler.load() - unpickler._session = False - main_module.__dict__.update(module.__dict__) - finally: - f.close() - return - -### End: Pickle the Interpreter - -### Extend the Picklers -class Pickler(StockPickler): - """python's Pickler extended to interpreter sessions""" - dispatch = StockPickler.dispatch.copy() - _main_module = None - _session = False - _byref = False - _strictio = False - _fmode = HANDLE_FMODE - pass - - def __init__(self, *args, **kwargs): - StockPickler.__init__(self, *args, **kwargs) - self._main_module = _main_module - self._diff_cache = {} - -class Unpickler(StockUnpickler): - """python's Unpickler extended to interpreter sessions and more types""" - _main_module = None - _session = False - - def find_class(self, module, name): - if (module, name) == ('__builtin__', '__main__'): - return self._main_module.__dict__ #XXX: above set w/save_module_dict - return StockUnpickler.find_class(self, module, name) - pass - - def __init__(self, *args, **kwargs): - StockUnpickler.__init__(self, *args, **kwargs) - self._main_module = _main_module - -''' -def dispatch_table(): - """get the dispatch table of registered types""" - return Pickler.dispatch -''' - -pickle_dispatch_copy = StockPickler.dispatch.copy() - -def pickle(t, func): - """expose dispatch table for user-created extensions""" - Pickler.dispatch[t] = func - return - -def register(t): - def proxy(func): - Pickler.dispatch[t] = func - return func - return proxy - -def _revert_extension(): - for type, func in list(StockPickler.dispatch.items()): - if func.__module__ == __name__: - del StockPickler.dispatch[type] - if type in pickle_dispatch_copy: - StockPickler.dispatch[type] = pickle_dispatch_copy[type] - -def use_diff(on=True): - """ - reduces size of pickles by only including object which have changed. - Decreases pickle size but increases CPU time needed. - Also helps avoid some unpicklable objects. - MUST be called at start of script, otherwise changes will not be recorded. - """ - global _use_diff, diff - _use_diff = on - if _use_diff and diff is None: - try: - from . import diff as d - except: - import diff as d - diff = d - -def _create_typemap(): - import types - if PY3: - d = dict(list(__builtin__.__dict__.items()) + \ - list(types.__dict__.items())).items() - builtin = 'builtins' - else: - d = types.__dict__.iteritems() - builtin = '__builtin__' - for key, value in d: - if getattr(value, '__module__', None) == builtin \ - and type(value) is type: - yield key, value - return -_reverse_typemap = dict(_create_typemap()) -_reverse_typemap.update({ - 'CellType': CellType, - 'WrapperDescriptorType': WrapperDescriptorType, - 'MethodDescriptorType': MethodDescriptorType, - 'MethodWrapperType': MethodWrapperType, - 'PartialType': PartialType, - 'SuperType': SuperType, - 'ItemGetterType': ItemGetterType, - 'AttrGetterType': AttrGetterType, - 'FileType': FileType, - 'BufferedRandomType': BufferedRandomType, - 'BufferedReaderType': BufferedReaderType, - 'BufferedWriterType': BufferedWriterType, - 'TextWrapperType': TextWrapperType, - 'PyBufferedRandomType': PyBufferedRandomType, - 'PyBufferedReaderType': PyBufferedReaderType, - 'PyBufferedWriterType': PyBufferedWriterType, - 'PyTextWrapperType': PyTextWrapperType, -}) -if ExitType: - _reverse_typemap['ExitType'] = ExitType -if InputType: - _reverse_typemap['InputType'] = InputType - _reverse_typemap['OutputType'] = OutputType -if PY3: - _typemap = dict((v, k) for k, v in _reverse_typemap.items()) -else: - _typemap = dict((v, k) for k, v in _reverse_typemap.iteritems()) - -def _unmarshal(string): - return marshal.loads(string) - -def _load_type(name): - return _reverse_typemap[name] - -def _create_type(typeobj, *args): - return typeobj(*args) - -def _create_function(fcode, fglobals, fname=None, fdefaults=None, \ - fclosure=None, fdict=None): - # same as FunctionType, but enable passing __dict__ to new function, - # __dict__ is the storehouse for attributes added after function creation - if fdict is None: fdict = dict() - func = FunctionType(fcode, fglobals, fname, fdefaults, fclosure) - func.__dict__.update(fdict) #XXX: better copy? option to copy? - return func - -def _create_ftype(ftypeobj, func, args, kwds): - if kwds is None: - kwds = {} - if args is None: - args = () - return ftypeobj(func, *args, **kwds) - -def _create_lock(locked, *args): - from threading import Lock - lock = Lock() - if locked: - if not lock.acquire(False): - raise UnpicklingError("Cannot acquire lock") - return lock - -# thanks to matsjoyce for adding all the different file modes -def _create_filehandle(name, mode, position, closed, open, strictio, fmode, fdata): # buffering=0 - # only pickles the handle, not the file contents... good? or StringIO(data)? - # (for file contents see: http://effbot.org/librarybook/copy-reg.htm) - # NOTE: handle special cases first (are there more special cases?) - names = {'':sys.__stdin__, '':sys.__stdout__, - '':sys.__stderr__} #XXX: better fileno=(0,1,2) ? - if name in list(names.keys()): - f = names[name] #XXX: safer "f=sys.stdin" - elif name == '': - f = os.tmpfile() - elif name == '': - import tempfile - f = tempfile.TemporaryFile(mode) - else: - # treat x mode as w mode - if "x" in mode and sys.hexversion < 0x03030000: - raise ValueError("invalid mode: '%s'" % mode) - - if not os.path.exists(name): - if strictio: - raise FileNotFoundError("[Errno 2] No such file or directory: '%s'" % name) - elif "r" in mode and fmode != FILE_FMODE: - name = '' # or os.devnull? - current_size = 0 # or maintain position? - else: - current_size = os.path.getsize(name) - - if position > current_size: - if strictio: - raise ValueError("invalid buffer size") - elif fmode == CONTENTS_FMODE: - position = current_size - # try to open the file by name - # NOTE: has different fileno - try: - #FIXME: missing: *buffering*, encoding, softspace - if fmode == FILE_FMODE: - f = open(name, mode if "w" in mode else "w") - f.write(fdata) - if "w" not in mode: - f.close() - f = open(name, mode) - elif name == '': # file did not exist - import tempfile - f = tempfile.TemporaryFile(mode) - elif fmode == CONTENTS_FMODE \ - and ("w" in mode or "x" in mode): - # stop truncation when opening - flags = os.O_CREAT - if "+" in mode: - flags |= os.O_RDWR - else: - flags |= os.O_WRONLY - f = os.fdopen(os.open(name, flags), mode) - # set name to the correct value - if PY3: - r = getattr(f, "buffer", f) - r = getattr(r, "raw", r) - r.name = name - else: - class FILE(ctypes.Structure): - _fields_ = [("refcount", ctypes.c_long), - ("type_obj", ctypes.py_object), - ("file_pointer", ctypes.c_voidp), - ("name", ctypes.py_object)] - - class PyObject(ctypes.Structure): - _fields_ = [ - ("ob_refcnt", ctypes.c_int), - ("ob_type", ctypes.py_object) - ] - if not HAS_CTYPES: - raise ImportError("No module named 'ctypes'") - ctypes.cast(id(f), ctypes.POINTER(FILE)).contents.name = name - ctypes.cast(id(name), ctypes.POINTER(PyObject)).contents.ob_refcnt += 1 - assert f.name == name - else: - f = open(name, mode) - except (IOError, FileNotFoundError): - err = sys.exc_info()[1] - raise UnpicklingError(err) - if closed: - f.close() - elif position >= 0 and fmode != HANDLE_FMODE: - f.seek(position) - return f - -def _create_stringi(value, position, closed): - f = StringIO(value) - if closed: f.close() - else: f.seek(position) - return f - -def _create_stringo(value, position, closed): - f = StringIO() - if closed: f.close() - else: - f.write(value) - f.seek(position) - return f - -class _itemgetter_helper(object): - def __init__(self): - self.items = [] - def __getitem__(self, item): - self.items.append(item) - return - -class _attrgetter_helper(object): - def __init__(self, attrs, index=None): - self.attrs = attrs - self.index = index - def __getattribute__(self, attr): - attrs = object.__getattribute__(self, "attrs") - index = object.__getattribute__(self, "index") - if index is None: - index = len(attrs) - attrs.append(attr) - else: - attrs[index] = ".".join([attrs[index], attr]) - return type(self)(attrs, index) - -if HAS_CTYPES: - ctypes.pythonapi.PyCell_New.restype = ctypes.py_object - ctypes.pythonapi.PyCell_New.argtypes = [ctypes.py_object] - # thanks to Paul Kienzle for cleaning the ctypes CellType logic - def _create_cell(contents): - return ctypes.pythonapi.PyCell_New(contents) - -def _create_weakref(obj, *args): - from weakref import ref - if obj is None: # it's dead - if PY3: - from collections import UserDict - else: - from UserDict import UserDict - return ref(UserDict(), *args) - return ref(obj, *args) - -def _create_weakproxy(obj, callable=False, *args): - from weakref import proxy - if obj is None: # it's dead - if callable: return proxy(lambda x:x, *args) - if PY3: - from collections import UserDict - else: - from UserDict import UserDict - return proxy(UserDict(), *args) - return proxy(obj, *args) - -def _eval_repr(repr_str): - return eval(repr_str) - -def _create_array(f, args, state, npdict=None): - #array = numpy.core.multiarray._reconstruct(*args) - array = f(*args) - array.__setstate__(state) - if npdict is not None: # we also have saved state in __dict__ - array.__dict__.update(npdict) - return array - -def _getattr(objclass, name, repr_str): - # hack to grab the reference directly - try: #XXX: works only for __builtin__ ? - attr = repr_str.split("'")[3] - return eval(attr+'.__dict__["'+name+'"]') - except: - attr = getattr(objclass,name) - if name == '__dict__': - attr = attr[name] - return attr - -def _get_attr(self, name): - # stop recursive pickling - return getattr(self, name) - -def _dict_from_dictproxy(dictproxy): - _dict = dictproxy.copy() # convert dictproxy to dict - _dict.pop('__dict__', None) - _dict.pop('__weakref__', None) - return _dict - -def _import_module(import_name, safe=False): - try: - if '.' in import_name: - items = import_name.split('.') - module = '.'.join(items[:-1]) - obj = items[-1] - else: - return __import__(import_name) - return getattr(__import__(module, None, None, [obj]), obj) - except (ImportError, AttributeError): - if safe: - return None - raise - -def _locate_function(obj, session=False): - if obj.__module__ == '__main__': # and session: - return False - found = _import_module(obj.__module__ + '.' + obj.__name__, safe=True) - return found is obj - -@register(CodeType) -def save_code(pickler, obj): - log.info("Co: %s" % obj) - pickler.save_reduce(_unmarshal, (marshal.dumps(obj),), obj=obj) - return - -@register(FunctionType) -def save_function(pickler, obj): - if not _locate_function(obj): #, pickler._session): - log.info("F1: %s" % obj) - if PY3: - pickler.save_reduce(_create_function, (obj.__code__, - obj.__globals__, obj.__name__, - obj.__defaults__, obj.__closure__, - obj.__dict__), obj=obj) - else: - pickler.save_reduce(_create_function, (obj.func_code, - obj.func_globals, obj.func_name, - obj.func_defaults, obj.func_closure, - obj.__dict__), obj=obj) - else: - log.info("F2: %s" % obj) - StockPickler.save_global(pickler, obj) #NOTE: also takes name=... - return - -@register(dict) -def save_module_dict(pickler, obj): - if is_dill(pickler) and obj == pickler._main_module.__dict__ and not pickler._session: - log.info("D1: = 0x20500f0: - @register(MemberDescriptorType) - @register(GetSetDescriptorType) - @register(MethodDescriptorType) - @register(WrapperDescriptorType) - def save_wrapper_descriptor(pickler, obj): - log.info("Wr: %s" % obj) - pickler.save_reduce(_getattr, (obj.__objclass__, obj.__name__, - obj.__repr__()), obj=obj) - return - - @register(MethodWrapperType) - def save_instancemethod(pickler, obj): - log.info("Mw: %s" % obj) - pickler.save_reduce(getattr, (obj.__self__, obj.__name__), obj=obj) - return -else: - @register(MethodDescriptorType) - @register(WrapperDescriptorType) - def save_wrapper_descriptor(pickler, obj): - log.info("Wr: %s" % obj) - pickler.save_reduce(_getattr, (obj.__objclass__, obj.__name__, - obj.__repr__()), obj=obj) - return - -if HAS_CTYPES: - @register(CellType) - def save_cell(pickler, obj): - log.info("Ce: %s" % obj) - pickler.save_reduce(_create_cell, (obj.cell_contents,), obj=obj) - return - -# The following function is based on 'saveDictProxy' from spickle -# Copyright (c) 2011 by science+computing ag -# License: http://www.apache.org/licenses/LICENSE-2.0 -@register(DictProxyType) -def save_dictproxy(pickler, obj): - log.info("Dp: %s" % obj) - attr = obj.get('__dict__') - #pickler.save_reduce(_create_dictproxy, (attr,'nested'), obj=obj) - if type(attr) == GetSetDescriptorType and attr.__name__ == "__dict__" \ - and getattr(attr.__objclass__, "__dict__", None) == obj: - pickler.save_reduce(getattr, (attr.__objclass__, "__dict__"), obj=obj) - return - # all bad below... so throw ReferenceError or TypeError - from weakref import ReferenceError - raise ReferenceError("%s does not reference a class __dict__" % obj) - -@register(SliceType) -def save_slice(pickler, obj): - log.info("Sl: %s" % obj) - pickler.save_reduce(slice, (obj.start, obj.stop, obj.step), obj=obj) - return - -@register(XRangeType) -@register(EllipsisType) -@register(NotImplementedType) -def save_singleton(pickler, obj): - log.info("Si: %s" % obj) - pickler.save_reduce(_eval_repr, (obj.__repr__(),), obj=obj) - return - -# thanks to Paul Kienzle for pointing out ufuncs didn't pickle -if NumpyArrayType: - @register(NumpyUfuncType) - def save_numpy_ufunc(pickler, obj): - log.info("Nu: %s" % obj) - StockPickler.save_global(pickler, obj) - return -# NOTE: the above 'save' performs like: -# import copy_reg -# def udump(f): return f.__name__ -# def uload(name): return getattr(numpy, name) -# copy_reg.pickle(NumpyUfuncType, udump, uload) - -def _proxy_helper(obj): # a dead proxy returns a reference to None - """get memory address of proxy's reference object""" - try: #FIXME: has to be a smarter way to identify if it's a proxy - address = int(repr(obj).rstrip('>').split(' at ')[-1], base=16) - except ValueError: # has a repr... is thus probably not a proxy - address = id(obj) - return address - -def _locate_object(address, module=None): - """get object located at the given memory address (inverse of id(obj))""" - special = [None, True, False] #XXX: more...? - for obj in special: - if address == id(obj): return obj - if module: - if PY3: - objects = iter(module.__dict__.values()) - else: - objects = module.__dict__.itervalues() - else: objects = iter(gc.get_objects()) - for obj in objects: - if address == id(obj): return obj - # all bad below... nothing found so throw ReferenceError or TypeError - from weakref import ReferenceError - try: address = hex(address) - except TypeError: - raise TypeError("'%s' is not a valid memory address" % str(address)) - raise ReferenceError("Cannot reference object at '%s'" % address) - -@register(ReferenceType) -def save_weakref(pickler, obj): - refobj = obj() - log.info("R1: %s" % obj) - #refobj = ctypes.pythonapi.PyWeakref_GetObject(obj) # dead returns "None" - pickler.save_reduce(_create_weakref, (refobj,), obj=obj) - return - -@register(ProxyType) -@register(CallableProxyType) -def save_weakproxy(pickler, obj): - refobj = _locate_object(_proxy_helper(obj)) - try: log.info("R2: %s" % obj) - except ReferenceError: log.info("R3: %s" % sys.exc_info()[1]) - #callable = bool(getattr(refobj, '__call__', None)) - if type(obj) is CallableProxyType: callable = True - else: callable = False - pickler.save_reduce(_create_weakproxy, (refobj, callable), obj=obj) - return - -@register(ModuleType) -def save_module(pickler, obj): - if False: #_use_diff: - if obj.__name__ != "dill": - try: - changed = diff.whats_changed(obj, seen=pickler._diff_cache)[0] - except RuntimeError: # not memorised module, probably part of dill - pass - else: - log.info("M1: %s with diff" % obj) - log.info("Diff: %s", changed.keys()) - pickler.save_reduce(_import_module, (obj.__name__,), obj=obj, - state=changed) - return - - log.info("M2: %s" % obj) - pickler.save_reduce(_import_module, (obj.__name__,), obj=obj) - else: - # if a module file name starts with prefx, it should be a builtin - # module, so should be pickled as a reference - prefix = getattr(sys, "base_prefix", sys.prefix) - std_mod = getattr(obj, "__file__", prefix).startswith(prefix) - if obj.__name__ not in ("builtins", "dill") \ - and not std_mod or is_dill(pickler) and obj is pickler._main_module: - log.info("M1: %s" % obj) - _main_dict = obj.__dict__.copy() #XXX: better no copy? option to copy? - [_main_dict.pop(item, None) for item in singletontypes - + ["__builtins__", "__loader__"]] - pickler.save_reduce(_import_module, (obj.__name__,), obj=obj, - state=_main_dict) - else: - log.info("M2: %s" % obj) - pickler.save_reduce(_import_module, (obj.__name__,), obj=obj) - return - return - -@register(TypeType) -def save_type(pickler, obj): - if obj in _typemap: - log.info("T1: %s" % obj) - pickler.save_reduce(_load_type, (_typemap[obj],), obj=obj) - elif obj.__module__ == '__main__': - try: # use StockPickler for special cases [namedtuple,] - [getattr(obj, attr) for attr in ('_fields','_asdict', - '_make','_replace')] - log.info("T6: %s" % obj) - StockPickler.save_global(pickler, obj) - return - except AttributeError: pass - if type(obj) == type: - # try: # used when pickling the class as code (or the interpreter) - if is_dill(pickler) and not pickler._byref: - # thanks to Tom Stepleton pointing out pickler._session unneeded - log.info("T2: %s" % obj) - _dict = _dict_from_dictproxy(obj.__dict__) - # except: # punt to StockPickler (pickle by class reference) - else: - log.info("T5: %s" % obj) - StockPickler.save_global(pickler, obj) - return - else: - log.info("T3: %s" % obj) - _dict = obj.__dict__ - #print (_dict) - #print ("%s\n%s" % (type(obj), obj.__name__)) - #print ("%s\n%s" % (obj.__bases__, obj.__dict__)) - pickler.save_reduce(_create_type, (type(obj), obj.__name__, - obj.__bases__, _dict), obj=obj) - else: - log.info("T4: %s" % obj) - #print (obj.__dict__) - #print ("%s\n%s" % (type(obj), obj.__name__)) - #print ("%s\n%s" % (obj.__bases__, obj.__dict__)) - StockPickler.save_global(pickler, obj) - return - -# quick sanity checking -def pickles(obj,exact=False,safe=False,**kwds): - """quick check if object pickles with dill""" - if safe: exceptions = (Exception,) # RuntimeError, ValueError - else: - exceptions = (TypeError, AssertionError, PicklingError, UnpicklingError) - try: - pik = copy(obj, **kwds) - try: - result = bool(pik.all() == obj.all()) - except AttributeError: - result = pik == obj - if result: return True - if not exact: - return type(pik) == type(obj) - return False - except exceptions: - return False - -# use to protect against missing attributes -def is_dill(pickler): - "check the dill-ness of your pickler" - return 'dill' in pickler.__module__ - #return hasattr(pickler,'_main_module') - -def _extend(): - """extend pickle with all of dill's registered types""" - # need to have pickle not choke on _main_module? use is_dill(pickler) - for t,func in Pickler.dispatch.items(): - try: - StockPickler.dispatch[t] = func - except: #TypeError, PicklingError, UnpicklingError - log.info("skip: %s" % t) - else: pass - return - -del diff, _use_diff, use_diff - -# EOF diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/objtypes.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/objtypes.py deleted file mode 100644 index bf0b557a54948..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/objtypes.py +++ /dev/null @@ -1,27 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE -""" -all Python Standard Library object types (currently: CH 1-15 @ 2.7) -and some other common object types (i.e. numpy.ndarray) - -to load more objects and types, use dill.load_types() -""" - -from __future__ import absolute_import - -# non-local import of dill.objects -from dill import objects -for _type in objects.keys(): - exec("%s = type(objects['%s'])" % (_type,_type)) - -del objects -try: - del _type -except NameError: - pass - -del absolute_import diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/pointers.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/pointers.py deleted file mode 100644 index 25714ea6418dc..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/pointers.py +++ /dev/null @@ -1,122 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE - -from __future__ import absolute_import -__all__ = ['parent', 'reference', 'at', 'parents', 'children'] - -import gc -import sys - -from .dill import _proxy_helper as reference -from .dill import _locate_object as at - -def parent(obj, objtype, ignore=()): - """ ->>> listiter = iter([4,5,6,7]) ->>> obj = parent(listiter, list) ->>> obj == [4,5,6,7] # actually 'is', but don't have handle any longer -True - -NOTE: objtype can be a single type (e.g. int or list) or a tuple of types. - -WARNING: if obj is a sequence (e.g. list), may produce unexpected results. -Parent finds *one* parent (e.g. the last member of the sequence). - """ - depth = 1 #XXX: always looking for the parent (only, right?) - chain = parents(obj, objtype, depth, ignore) - parent = chain.pop() - if parent is obj: - return None - return parent - - -def parents(obj, objtype, depth=1, ignore=()): #XXX: objtype=object ? - """Find the chain of referents for obj. Chain will end with obj. - - objtype: an object type or tuple of types to search for - depth: search depth (e.g. depth=2 is 'grandparents') - ignore: an object or tuple of objects to ignore in the search - """ - edge_func = gc.get_referents # looking for refs, not back_refs - predicate = lambda x: isinstance(x, objtype) # looking for parent type - #if objtype is None: predicate = lambda x: True #XXX: in obj.mro() ? - ignore = (ignore,) if not hasattr(ignore, '__len__') else ignore - ignore = (id(obj) for obj in ignore) - chain = find_chain(obj, predicate, edge_func, depth)[::-1] - #XXX: should pop off obj... ? - return chain - - -def children(obj, objtype, depth=1, ignore=()): #XXX: objtype=object ? - """Find the chain of referrers for obj. Chain will start with obj. - - objtype: an object type or tuple of types to search for - depth: search depth (e.g. depth=2 is 'grandchildren') - ignore: an object or tuple of objects to ignore in the search - - NOTE: a common thing to ignore is all globals, 'ignore=(globals(),)' - - NOTE: repeated calls may yield different results, as python stores - the last value in the special variable '_'; thus, it is often good - to execute something to replace '_' (e.g. >>> 1+1). - """ - edge_func = gc.get_referrers # looking for back_refs, not refs - predicate = lambda x: isinstance(x, objtype) # looking for child type - #if objtype is None: predicate = lambda x: True #XXX: in obj.mro() ? - ignore = (ignore,) if not hasattr(ignore, '__len__') else ignore - ignore = (id(obj) for obj in ignore) - chain = find_chain(obj, predicate, edge_func, depth, ignore) - #XXX: should pop off obj... ? - return chain - - -# more generic helper function (cut-n-paste from objgraph) -# Source at http://mg.pov.lt/objgraph/ -# Copyright (c) 2008-2010 Marius Gedminas -# Copyright (c) 2010 Stefano Rivera -# Released under the MIT licence (see objgraph/objgrah.py) - -def find_chain(obj, predicate, edge_func, max_depth=20, extra_ignore=()): - queue = [obj] - depth = {id(obj): 0} - parent = {id(obj): None} - ignore = set(extra_ignore) - ignore.add(id(extra_ignore)) - ignore.add(id(queue)) - ignore.add(id(depth)) - ignore.add(id(parent)) - ignore.add(id(ignore)) - ignore.add(id(sys._getframe())) # this function - ignore.add(id(sys._getframe(1))) # find_chain/find_backref_chain, likely - gc.collect() - while queue: - target = queue.pop(0) - if predicate(target): - chain = [target] - while parent[id(target)] is not None: - target = parent[id(target)] - chain.append(target) - return chain - tdepth = depth[id(target)] - if tdepth < max_depth: - referrers = edge_func(target) - ignore.add(id(referrers)) - for source in referrers: - if id(source) in ignore: - continue - if id(source) not in depth: - depth[id(source)] = tdepth + 1 - parent[id(source)] = target - queue.append(source) - return [obj] # not found - - -# backward compatability -refobject = at - - -# EOF diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/source.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/source.py deleted file mode 100644 index b55ca55811bc6..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/source.py +++ /dev/null @@ -1,1010 +0,0 @@ -# #!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE -# -# inspired by inspect.py from Python-2.7.6 -# inspect.py author: 'Ka-Ping Yee ' -# inspect.py merged into original dill.source by Mike McKerns 4/13/14 -""" -Extensions to python's 'inspect' module, which can be used -to retrieve information from live python objects. The methods -defined in this module are augmented to facilitate access to -source code of interactively defined functions and classes, -as well as provide access to source code for objects defined -in a file. -""" - -from __future__ import absolute_import -__all__ = ['findsource', 'getsourcelines', 'getsource', 'indent', 'outdent', \ - '_wrap', 'dumpsource', 'getname', '_namespace', 'getimport', \ - '_importable', 'importable','isdynamic', 'isfrommain'] - -import re -import linecache -from tokenize import TokenError -from inspect import ismodule, isclass, ismethod, isfunction, istraceback -from inspect import isframe, iscode, getfile, getmodule, getsourcefile -from inspect import getblock, indentsize, isbuiltin -from .dill import PY3 - -def isfrommain(obj): - "check if object was built in __main__" - module = getmodule(obj) - if module and module.__name__ == '__main__': - return True - return False - - -def isdynamic(obj): - "check if object was built in the interpreter" - try: file = getfile(obj) - except TypeError: file = None - if file == '' and isfrommain(obj): - return True - return False - - -def _matchlambda(func, line): - """check if lambda object 'func' matches raw line of code 'line'""" - from dill.detect import code as getcode - from dill.detect import freevars, globalvars, varnames - dummy = lambda : '__this_is_a_big_dummy_function__' - # process the line (removing leading whitespace, etc) - lhs,rhs = line.split('lambda ',1)[-1].split(":", 1) #FIXME: if !1 inputs - try: #FIXME: unsafe - _ = eval("lambda %s : %s" % (lhs,rhs), globals(),locals()) - except: _ = dummy - # get code objects, for comparison - _, code = getcode(_).co_code, getcode(func).co_code - # check if func is in closure - _f = [line.count(i) for i in freevars(func).keys()] - if not _f: # not in closure - # check if code matches - if _ == code: return True - return False - # weak check on freevars - if not all(_f): return False #XXX: VERY WEAK - # weak check on varnames and globalvars - _f = varnames(func) - _f = [line.count(i) for i in _f[0]+_f[1]] - if _f and not all(_f): return False #XXX: VERY WEAK - _f = [line.count(i) for i in globalvars(func).keys()] - if _f and not all(_f): return False #XXX: VERY WEAK - # check if func is a double lambda - if (line.count('lambda ') > 1) and (lhs in freevars(func).keys()): - _lhs,_rhs = rhs.split('lambda ',1)[-1].split(":",1) #FIXME: if !1 inputs - try: #FIXME: unsafe - _f = eval("lambda %s : %s" % (_lhs,_rhs), globals(),locals()) - except: _f = dummy - # get code objects, for comparison - _, code = getcode(_f).co_code, getcode(func).co_code - if len(_) != len(code): return False - #NOTE: should be same code same order, but except for 't' and '\x88' - _ = set((i,j) for (i,j) in zip(_,code) if i != j) - if len(_) != 1: return False #('t','\x88') - return True - # check indentsize - if not indentsize(line): return False #FIXME: is this a good check??? - # check if code 'pattern' matches - #XXX: or pattern match against dis.dis(code)? (or use uncompyle2?) - _ = _.split(_[0]) # 't' #XXX: remove matching values if starts the same? - _f = code.split(code[0]) # '\x88' - #NOTE: should be same code different order, with different first element - _ = dict(re.match('([\W\D\S])(.*)', _[i]).groups() for i in range(1,len(_))) - _f = dict(re.match('([\W\D\S])(.*)', _f[i]).groups() for i in range(1,len(_f))) - if (_.keys() == _f.keys()) and (sorted(_.values()) == sorted(_f.values())): - return True - return False - - -def findsource(object): - """Return the entire source file and starting line number for an object. - For interactively-defined objects, the 'file' is the interpreter's history. - - The argument may be a module, class, method, function, traceback, frame, - or code object. The source code is returned as a list of all the lines - in the file and the line number indexes a line in that list. An IOError - is raised if the source code cannot be retrieved, while a TypeError is - raised for objects where the source code is unavailable (e.g. builtins).""" - - module = getmodule(object) - try: file = getfile(module) - except TypeError: file = None - # use readline when working in interpreter (i.e. __main__ and not file) - if module and module.__name__ == '__main__' and not file: - import readline - lbuf = readline.get_current_history_length() - lines = [readline.get_history_item(i)+'\n' for i in range(1,lbuf)] - else: - try: # special handling for class instances - if not isclass(object) and isclass(type(object)): # __class__ - file = getfile(module) - sourcefile = getsourcefile(module) - else: # builtins fail with a TypeError - file = getfile(object) - sourcefile = getsourcefile(object) - except (TypeError, AttributeError): # fail with better error - file = getfile(object) - sourcefile = getsourcefile(object) - if not sourcefile and file[:1] + file[-1:] != '<>': - raise IOError('source code not available') - file = sourcefile if sourcefile else file - - module = getmodule(object, file) - if module: - lines = linecache.getlines(file, module.__dict__) - else: - lines = linecache.getlines(file) - - if not lines: - raise IOError('could not get source code') - - #FIXME: all below may fail if exec used (i.e. exec('f = lambda x:x') ) - if ismodule(object): - return lines, 0 - - name = pat1 = obj = '' - pat2 = r'^(\s*@)' -# pat1b = r'^(\s*%s\W*=)' % name #FIXME: finds 'f = decorate(f)', not exec - if ismethod(object): - name = object.__name__ - if name == '': pat1 = r'(.*(?': - pat1 = r'(.*(?' - if stdin: - lnum = len(lines) - 1 # can't get lnum easily, so leverage pat - if not pat1: pat1 = r'^(\s*def\s)|(.*(? 0: #XXX: won't find decorators in ? - line = lines[lnum] - if pat1.match(line): - if not stdin: break # co_firstlineno does the job - if name == '': # hackery needed to confirm a match - if _matchlambda(obj, line): break - else: # not a lambda, just look for the name - if name in line: # need to check for decorator... - hats = 0 - for _lnum in range(lnum-1,-1,-1): - if pat2.match(lines[_lnum]): hats += 1 - else: break - lnum = lnum - hats - break - lnum = lnum - 1 - return lines, lnum - - try: # turn instances into classes - if not isclass(object) and isclass(type(object)): # __class__ - object = object.__class__ #XXX: sometimes type(class) is better? - #XXX: we don't find how the instance was built - except AttributeError: pass - if isclass(object): - name = object.__name__ - pat = re.compile(r'^(\s*)class\s*' + name + r'\b') - # make some effort to find the best matching class definition: - # use the one with the least indentation, which is the one - # that's most probably not inside a function definition. - candidates = [] - for i in range(len(lines)-1,-1,-1): - match = pat.match(lines[i]) - if match: - # if it's at toplevel, it's already the best one - if lines[i][0] == 'c': - return lines, i - # else add whitespace to candidate list - candidates.append((match.group(1), i)) - if candidates: - # this will sort by whitespace, and by line number, - # less whitespace first #XXX: should sort high lnum before low - candidates.sort() - return lines, candidates[0][1] - else: - raise IOError('could not find class definition') - raise IOError('could not find code object') - - -def getblocks(object, lstrip=False, enclosing=False, locate=False): - """Return a list of source lines and starting line number for an object. - Interactively-defined objects refer to lines in the interpreter's history. - - If enclosing=True, then also return any enclosing code. - If lstrip=True, ensure there is no indentation in the first line of code. - If locate=True, then also return the line number for the block of code. - - DEPRECATED: use 'getsourcelines' instead - """ - lines, lnum = findsource(object) - - if ismodule(object): - if lstrip: lines = _outdent(lines) - return ([lines], [0]) if locate is True else [lines] - - #XXX: 'enclosing' means: closures only? or classes and files? - indent = indentsize(lines[lnum]) - block = getblock(lines[lnum:]) #XXX: catch any TokenError here? - - if not enclosing or not indent: - if lstrip: block = _outdent(block) - return ([block], [lnum]) if locate is True else [block] - - pat1 = r'^(\s*def\s)|(.*(? indent: #XXX: should be >= ? - line += len(code) - skip - elif target in ''.join(code): - blocks.append(code) # save code block as the potential winner - _lnum.append(line - skip) # save the line number for the match - line += len(code) - skip - else: - line += 1 - skip = 0 - # find skip: the number of consecutive decorators - elif pat2.match(lines[line]): - try: code = getblock(lines[line:]) - except TokenError: code = [lines[line]] - skip = 1 - for _line in code[1:]: # skip lines that are decorators - if not pat2.match(_line): break - skip += 1 - line += skip - # no match: reset skip and go to the next line - else: - line +=1 - skip = 0 - - if not blocks: - blocks = [block] - _lnum = [lnum] - if lstrip: blocks = [_outdent(block) for block in blocks] - # return last match - return (blocks, _lnum) if locate is True else blocks - - -def getsourcelines(object, lstrip=False, enclosing=False): - """Return a list of source lines and starting line number for an object. - Interactively-defined objects refer to lines in the interpreter's history. - - The argument may be a module, class, method, function, traceback, frame, - or code object. The source code is returned as a list of the lines - corresponding to the object and the line number indicates where in the - original source file the first line of code was found. An IOError is - raised if the source code cannot be retrieved, while a TypeError is - raised for objects where the source code is unavailable (e.g. builtins). - - If lstrip=True, ensure there is no indentation in the first line of code. - If enclosing=True, then also return any enclosing code.""" - code, n = getblocks(object, lstrip=lstrip, enclosing=enclosing, locate=True) - return code[-1], n[-1] - - -#NOTE: broke backward compatibility 4/16/14 (was lstrip=True, force=True) -def getsource(object, alias='', lstrip=False, enclosing=False, \ - force=False, builtin=False): - """Return the text of the source code for an object. The source code for - interactively-defined objects are extracted from the interpreter's history. - - The argument may be a module, class, method, function, traceback, frame, - or code object. The source code is returned as a single string. An - IOError is raised if the source code cannot be retrieved, while a - TypeError is raised for objects where the source code is unavailable - (e.g. builtins). - - If alias is provided, then add a line of code that renames the object. - If lstrip=True, ensure there is no indentation in the first line of code. - If enclosing=True, then also return any enclosing code. - If force=True, catch (TypeError,IOError) and try to use import hooks. - If builtin=True, force an import for any builtins - """ - # hascode denotes a callable - hascode = _hascode(object) - # is a class instance type (and not in builtins) - instance = _isinstance(object) - - # get source lines; if fail, try to 'force' an import - try: # fails for builtins, and other assorted object types - lines, lnum = getsourcelines(object, enclosing=enclosing) - except (TypeError, IOError): # failed to get source, resort to import hooks - if not force: # don't try to get types that findsource can't get - raise - if not getmodule(object): # get things like 'None' and '1' - if not instance: return getimport(object, alias, builtin=builtin) - # special handling (numpy arrays, ...) - _import = getimport(object, builtin=builtin) - name = getname(object, force=True) - _alias = "%s = " % alias if alias else "" - if alias == name: _alias = "" - return _import+_alias+"%s\n" % name - else: #FIXME: could use a good bit of cleanup, since using getimport... - if not instance: return getimport(object, alias, builtin=builtin) - # now we are dealing with an instance... - name = object.__class__.__name__ - module = object.__module__ - if module in ['builtins','__builtin__']: - return getimport(object, alias, builtin=builtin) - else: #FIXME: leverage getimport? use 'from module import name'? - lines, lnum = ["%s = __import__('%s', fromlist=['%s']).%s\n" % (name,module,name,name)], 0 - obj = eval(lines[0].lstrip(name + ' = ')) - lines, lnum = getsourcelines(obj, enclosing=enclosing) - - # strip leading indent (helps ensure can be imported) - if lstrip or alias: - lines = _outdent(lines) - - # instantiate, if there's a nice repr #XXX: BAD IDEA??? - if instance: #and force: #XXX: move into findsource or getsourcelines ? - if '(' in repr(object): lines.append('%r\n' % object) - #else: #XXX: better to somehow to leverage __reduce__ ? - # reconstructor,args = object.__reduce__() - # _ = reconstructor(*args) - else: # fall back to serialization #XXX: bad idea? - #XXX: better not duplicate work? #XXX: better new/enclose=True? - lines = dumpsource(object, alias='', new=force, enclose=False) - lines, lnum = [line+'\n' for line in lines.split('\n')][:-1], 0 - #else: object.__code__ # raise AttributeError - - # add an alias to the source code - if alias: - if hascode: - skip = 0 - for line in lines: # skip lines that are decorators - if not line.startswith('@'): break - skip += 1 - #XXX: use regex from findsource / getsourcelines ? - if lines[skip].lstrip().startswith('def '): # we have a function - if alias != object.__name__: - lines.append('\n%s = %s\n' % (alias, object.__name__)) - elif 'lambda ' in lines[skip]: # we have a lambda - if alias != lines[skip].split('=')[0].strip(): - lines[skip] = '%s = %s' % (alias, lines[skip]) - else: # ...try to use the object's name - if alias != object.__name__: - lines.append('\n%s = %s\n' % (alias, object.__name__)) - else: # class or class instance - if instance: - if alias != lines[-1].split('=')[0].strip(): - lines[-1] = ('%s = ' % alias) + lines[-1] - else: - name = getname(object, force=True) or object.__name__ - if alias != name: - lines.append('\n%s = %s\n' % (alias, name)) - return ''.join(lines) - - -def _hascode(object): - '''True if object has an attribute that stores it's __code__''' - return getattr(object,'__code__',None) or getattr(object,'func_code',None) - -def _isinstance(object): - '''True if object is a class instance type (and is not a builtin)''' - if _hascode(object) or isclass(object) or ismodule(object): - return False - if istraceback(object) or isframe(object) or iscode(object): - return False - # special handling (numpy arrays, ...) - if not getmodule(object) and getmodule(type(object)).__name__ in ['numpy']: - return True -# # check if is instance of a builtin -# if not getmodule(object) and getmodule(type(object)).__name__ in ['__builtin__','builtins']: -# return False - _types = ('") - if not repr(type(object)).startswith(_types): #FIXME: weak hack - return False - if not getmodule(object) or object.__module__ in ['builtins','__builtin__'] or getname(object, force=True) in ['array']: - return False - return True # by process of elimination... it's what we want - - -def _intypes(object): - '''check if object is in the 'types' module''' - import types - # allow user to pass in object or object.__name__ - if type(object) is not type(''): - object = getname(object, force=True) - if object == 'ellipsis': object = 'EllipsisType' - return True if hasattr(types, object) else False - - -def _isstring(object): #XXX: isstringlike better? - '''check if object is a string-like type''' - if PY3: return isinstance(object, (str, bytes)) - return isinstance(object, basestring) - - -def indent(code, spaces=4): - '''indent a block of code with whitespace (default is 4 spaces)''' - indent = indentsize(code) - if type(spaces) is int: spaces = ' '*spaces - # if '\t' is provided, will indent with a tab - nspaces = indentsize(spaces) - # blank lines (etc) need to be ignored - lines = code.split('\n') -## stq = "'''"; dtq = '"""' -## in_stq = in_dtq = False - for i in range(len(lines)): - #FIXME: works... but shouldn't indent 2nd+ lines of multiline doc - _indent = indentsize(lines[i]) - if indent > _indent: continue - lines[i] = spaces+lines[i] -## #FIXME: may fail when stq and dtq in same line (depends on ordering) -## nstq, ndtq = lines[i].count(stq), lines[i].count(dtq) -## if not in_dtq and not in_stq: -## lines[i] = spaces+lines[i] # we indent -## # entering a comment block -## if nstq%2: in_stq = not in_stq -## if ndtq%2: in_dtq = not in_dtq -## # leaving a comment block -## elif in_dtq and ndtq%2: in_dtq = not in_dtq -## elif in_stq and nstq%2: in_stq = not in_stq -## else: pass - if lines[-1].strip() == '': lines[-1] = '' - return '\n'.join(lines) - - -def _outdent(lines, spaces=None, all=True): - '''outdent lines of code, accounting for docs and line continuations''' - indent = indentsize(lines[0]) - if spaces is None or spaces > indent or spaces < 0: spaces = indent - for i in range(len(lines) if all else 1): - #FIXME: works... but shouldn't outdent 2nd+ lines of multiline doc - _indent = indentsize(lines[i]) - if spaces > _indent: _spaces = _indent - else: _spaces = spaces - lines[i] = lines[i][_spaces:] - return lines - -def outdent(code, spaces=None, all=True): - '''outdent a block of code (default is to strip all leading whitespace)''' - indent = indentsize(code) - if spaces is None or spaces > indent or spaces < 0: spaces = indent - #XXX: will this delete '\n' in some cases? - if not all: return code[spaces:] - return '\n'.join(_outdent(code.split('\n'), spaces=spaces, all=all)) - - -#XXX: not sure what the point of _wrap is... -#exec_ = lambda s, *a: eval(compile(s, '', 'exec'), *a) -__globals__ = globals() -__locals__ = locals() -wrap2 = ''' -def _wrap(f): - """ encapsulate a function and it's __import__ """ - def func(*args, **kwds): - try: - #_ = eval(getsource(f, force=True)) #FIXME: safer, but not as robust - exec getimportable(f, alias='_') in %s, %s - except: - raise ImportError('cannot import name ' + f.__name__) - return _(*args, **kwds) - func.__name__ = f.__name__ - func.__doc__ = f.__doc__ - return func -''' % ('__globals__', '__locals__') -wrap3 = ''' -def _wrap(f): - """ encapsulate a function and it's __import__ """ - def func(*args, **kwds): - try: - #_ = eval(getsource(f, force=True)) #FIXME: safer, but not as robust - exec(getimportable(f, alias='_'), %s, %s) - except: - raise ImportError('cannot import name ' + f.__name__) - return _(*args, **kwds) - func.__name__ = f.__name__ - func.__doc__ = f.__doc__ - return func -''' % ('__globals__', '__locals__') -if PY3: - exec(wrap3) -else: - exec(wrap2) -del wrap2, wrap3 - - -def _enclose(object, alias=''): #FIXME: needs alias to hold returned object - """create a function enclosure around the source of some object""" - #XXX: dummy and stub should append a random string - dummy = '__this_is_a_big_dummy_enclosing_function__' - stub = '__this_is_a_stub_variable__' - code = 'def %s():\n' % dummy - code += indent(getsource(object, alias=stub, lstrip=True, force=True)) - code += indent('return %s\n' % stub) - if alias: code += '%s = ' % alias - code += '%s(); del %s\n' % (dummy, dummy) - #code += "globals().pop('%s',lambda :None)()\n" % dummy - return code - - -def dumpsource(object, alias='', new=False, enclose=True): - """'dump to source', where the code includes a pickled object. - - If new=True and object is a class instance, then create a new - instance using the unpacked class source code. If enclose, then - create the object inside a function enclosure (thus minimizing - any global namespace pollution). - """ - from dill import dumps - pik = repr(dumps(object)) - code = 'import dill\n' - if enclose: - stub = '__this_is_a_stub_variable__' #XXX: *must* be same _enclose.stub - pre = '%s = ' % stub - new = False #FIXME: new=True doesn't work with enclose=True - else: - stub = alias - pre = '%s = ' % stub if alias else alias - - # if a 'new' instance is not needed, then just dump and load - if not new or not _isinstance(object): - code += pre + 'dill.loads(%s)\n' % pik - else: #XXX: other cases where source code is needed??? - code += getsource(object.__class__, alias='', lstrip=True, force=True) - mod = repr(object.__module__) # should have a module (no builtins here) - if PY3: - code += pre + 'dill.loads(%s.replace(b%s,bytes(__name__,"UTF-8")))\n' % (pik,mod) - else: - code += pre + 'dill.loads(%s.replace(%s,__name__))\n' % (pik,mod) - #code += 'del %s' % object.__class__.__name__ #NOTE: kills any existing! - - if enclose: - # generation of the 'enclosure' - dummy = '__this_is_a_big_dummy_object__' - dummy = _enclose(dummy, alias=alias) - # hack to replace the 'dummy' with the 'real' code - dummy = dummy.split('\n') - code = dummy[0]+'\n' + indent(code) + '\n'.join(dummy[-3:]) - - return code #XXX: better 'dumpsourcelines', returning list of lines? - - -def getname(obj, force=False): #XXX: allow 'throw'(?) to raise error on fail? - """get the name of the object. for lambdas, get the name of the pointer """ - module = getmodule(obj) - if not module: # things like "None" and "1" - if not force: return None - return repr(obj) - try: - #XXX: 'wrong' for decorators and curried functions ? - # if obj.func_closure: ...use logic from getimportable, etc ? - name = obj.__name__ - if name == '': - return getsource(obj).split('=',1)[0].strip() - # handle some special cases - if module.__name__ in ['builtins','__builtin__']: - if name == 'ellipsis': name = 'EllipsisType' - return name - except AttributeError: #XXX: better to just throw AttributeError ? - if not force: return None - name = repr(obj) - if name.startswith('<'): # or name.split('('): - return None - return name - - -def _namespace(obj): - """_namespace(obj); return namespace hierarchy (as a list of names) - for the given object. For an instance, find the class hierarchy. - - For example: - - >>> from functools import partial - >>> p = partial(int, base=2) - >>> _namespace(p) - [\'functools\', \'partial\'] - """ - # mostly for functions and modules and such - #FIXME: 'wrong' for decorators and curried functions - try: #XXX: needs some work and testing on different types - module = qual = str(getmodule(obj)).split()[1].strip('"').strip("'") - qual = qual.split('.') - if ismodule(obj): - return qual - # get name of a lambda, function, etc - name = getname(obj) or obj.__name__ # failing, raise AttributeError - # check special cases (NoneType, ...) - if module in ['builtins','__builtin__']: # BuiltinFunctionType - if _intypes(name): return ['types'] + [name] - return qual + [name] #XXX: can be wrong for some aliased objects - except: pass - # special case: numpy.inf and numpy.nan (we don't want them as floats) - if str(obj) in ['inf','nan','Inf','NaN']: # is more, but are they needed? - return ['numpy'] + [str(obj)] - # mostly for classes and class instances and such - module = getattr(obj.__class__, '__module__', None) - qual = str(obj.__class__) - try: qual = qual[qual.index("'")+1:-2] - except ValueError: pass # str(obj.__class__) made the 'try' unnecessary - qual = qual.split(".") - if module in ['builtins','__builtin__']: - # check special cases (NoneType, Ellipsis, ...) - if qual[-1] == 'ellipsis': qual[-1] = 'EllipsisType' - if _intypes(qual[-1]): module = 'types' #XXX: BuiltinFunctionType - qual = [module] + qual - return qual - - -#NOTE: 05/25/14 broke backward compatability: added 'alias' as 3rd argument -def _getimport(head, tail, alias='', verify=True, builtin=False): - """helper to build a likely import string from head and tail of namespace. - ('head','tail') are used in the following context: "from head import tail" - - If verify=True, then test the import string before returning it. - If builtin=True, then force an import for builtins where possible. - If alias is provided, then rename the object on import. - """ - # special handling for a few common types - if tail in ['Ellipsis', 'NotImplemented'] and head in ['types']: - head = len.__module__ - elif tail in ['None'] and head in ['types']: - _alias = '%s = ' % alias if alias else '' - if alias == tail: _alias = '' - return _alias+'%s\n' % tail - # we don't need to import from builtins, so return '' -# elif tail in ['NoneType','int','float','long','complex']: return '' #XXX: ? - if head in ['builtins','__builtin__']: - # special cases (NoneType, Ellipsis, ...) #XXX: BuiltinFunctionType - if tail == 'ellipsis': tail = 'EllipsisType' - if _intypes(tail): head = 'types' - elif not builtin: - _alias = '%s = ' % alias if alias else '' - if alias == tail: _alias = '' - return _alias+'%s\n' % tail - else: pass # handle builtins below - # get likely import string - if not head: _str = "import %s" % tail - else: _str = "from %s import %s" % (head, tail) - _alias = " as %s\n" % alias if alias else "\n" - if alias == tail: _alias = "\n" - _str += _alias - # FIXME: fails on most decorators, currying, and such... - # (could look for magic __wrapped__ or __func__ attr) - # (could fix in 'namespace' to check obj for closure) - if verify and not head.startswith('dill.'):# weird behavior for dill - #print(_str) - try: exec(_str) #XXX: check if == obj? (name collision) - except ImportError: #XXX: better top-down or bottom-up recursion? - _head = head.rsplit(".",1)[0] #(or get all, then compare == obj?) - if not _head: raise - if _head != head: - _str = _getimport(_head, tail, alias, verify) - return _str - - -#XXX: rename builtin to force? vice versa? verify to force? (as in getsource) -#NOTE: 05/25/14 broke backward compatability: added 'alias' as 2nd argument -def getimport(obj, alias='', verify=True, builtin=False, enclosing=False): - """get the likely import string for the given object - - obj is the object to inspect - If verify=True, then test the import string before returning it. - If builtin=True, then force an import for builtins where possible. - If enclosing=True, get the import for the outermost enclosing callable. - If alias is provided, then rename the object on import. - """ - if enclosing: - from dill.detect import outermost - _obj = outermost(obj) - obj = _obj if _obj else obj - # get the namespace - qual = _namespace(obj) - head = '.'.join(qual[:-1]) - tail = qual[-1] - # for named things... with a nice repr #XXX: move into _namespace? - try: # look for '<...>' and be mindful it might be in lists, dicts, etc... - name = repr(obj).split('<',1)[1].split('>',1)[1] - name = None # we have a 'object'-style repr - except: # it's probably something 'importable' - if head in ['builtins','__builtin__']: - name = repr(obj) #XXX: catch [1,2], (1,2), set([1,2])... others? - else: - name = repr(obj).split('(')[0] - #if not repr(obj).startswith('<'): name = repr(obj).split('(')[0] - #else: name = None - if name: # try using name instead of tail - try: return _getimport(head, name, alias, verify, builtin) - except ImportError: pass - except SyntaxError: - if head in ['builtins','__builtin__']: - _alias = '%s = ' % alias if alias else '' - if alias == name: _alias = '' - return _alias+'%s\n' % name - else: pass - try: - #if type(obj) is type(abs): _builtin = builtin # BuiltinFunctionType - #else: _builtin = False - return _getimport(head, tail, alias, verify, builtin) - except ImportError: - raise # could do some checking against obj - except SyntaxError: - if head in ['builtins','__builtin__']: - _alias = '%s = ' % alias if alias else '' - if alias == tail: _alias = '' - return _alias+'%s\n' % tail - raise # could do some checking against obj - - -def _importable(obj, alias='', source=None, enclosing=False, force=True, \ - builtin=True, lstrip=True): - """get an import string (or the source code) for the given object - - This function will attempt to discover the name of the object, or the repr - of the object, or the source code for the object. To attempt to force - discovery of the source code, use source=True, to attempt to force the - use of an import, use source=False; otherwise an import will be sought - for objects not defined in __main__. The intent is to build a string - that can be imported from a python file. obj is the object to inspect. - If alias is provided, then rename the object with the given alias. - - If source=True, use these options: - If enclosing=True, then also return any enclosing code. - If force=True, catch (TypeError,IOError) and try to use import hooks. - If lstrip=True, ensure there is no indentation in the first line of code. - - If source=False, use these options: - If enclosing=True, get the import for the outermost enclosing callable. - If force=True, then don't test the import string before returning it. - If builtin=True, then force an import for builtins where possible. - """ - if source is None: - source = True if isfrommain(obj) else False - if source: # first try to get the source - try: - return getsource(obj, alias, enclosing=enclosing, \ - force=force, lstrip=lstrip, builtin=builtin) - except: pass - try: - if not _isinstance(obj): - return getimport(obj, alias, enclosing=enclosing, \ - verify=(not force), builtin=builtin) - # first 'get the import', then 'get the instance' - _import = getimport(obj, enclosing=enclosing, \ - verify=(not force), builtin=builtin) - name = getname(obj, force=True) - if not name: - raise AttributeError("object has no atribute '__name__'") - _alias = "%s = " % alias if alias else "" - if alias == name: _alias = "" - return _import+_alias+"%s\n" % name - - except: pass - if not source: # try getsource, only if it hasn't been tried yet - try: - return getsource(obj, alias, enclosing=enclosing, \ - force=force, lstrip=lstrip, builtin=builtin) - except: pass - # get the name (of functions, lambdas, and classes) - # or hope that obj can be built from the __repr__ - #XXX: what to do about class instances and such? - obj = getname(obj, force=force) - # we either have __repr__ or __name__ (or None) - if not obj or obj.startswith('<'): - raise AttributeError("object has no atribute '__name__'") - _alias = '%s = ' % alias if alias else '' - if alias == obj: _alias = '' - return _alias+'%s\n' % obj - #XXX: possible failsafe... (for example, for instances when source=False) - # "import dill; result = dill.loads(); # repr()" - -def _closuredimport(func, alias='', builtin=False): - """get import for closured objects; return a dict of 'name' and 'import'""" - import re - from dill.detect import freevars, outermost - free_vars = freevars(func) - func_vars = {} - # split into 'funcs' and 'non-funcs' - for name,obj in list(free_vars.items()): - if not isfunction(obj): continue - # get import for 'funcs' - fobj = free_vars.pop(name) - src = getsource(fobj) - if src.lstrip().startswith('@'): # we have a decorator - src = getimport(fobj, alias=alias, builtin=builtin) - else: # we have to "hack" a bit... and maybe be lucky - encl = outermost(func) - # pattern: 'func = enclosing(fobj' - pat = '.*[\w\s]=\s*'+getname(encl)+'\('+getname(fobj) - mod = getname(getmodule(encl)) - #HACK: get file containing 'outer' function; is func there? - lines,_ = findsource(encl) - candidate = [line for line in lines if getname(encl) in line and \ - re.match(pat, line)] - if not candidate: - mod = getname(getmodule(fobj)) - #HACK: get file containing 'inner' function; is func there? - lines,_ = findsource(fobj) - candidate = [line for line in lines \ - if getname(fobj) in line and re.match(pat, line)] - if not len(candidate): raise TypeError('import could not be found') - candidate = candidate[-1] - name = candidate.split('=',1)[0].split()[-1].strip() - src = _getimport(mod, name, alias=alias, builtin=builtin) - func_vars[name] = src - if not func_vars: - name = outermost(func) - mod = getname(getmodule(name)) - if not mod or name is func: # then it can be handled by getimport - name = getname(func, force=True) #XXX: better key? - src = getimport(func, alias=alias, builtin=builtin) - else: - lines,_ = findsource(name) - # pattern: 'func = enclosing(' - candidate = [line for line in lines if getname(name) in line and \ - re.match('.*[\w\s]=\s*'+getname(name)+'\(', line)] - if not len(candidate): raise TypeError('import could not be found') - candidate = candidate[-1] - name = candidate.split('=',1)[0].split()[-1].strip() - src = _getimport(mod, name, alias=alias, builtin=builtin) - func_vars[name] = src - return func_vars - -#XXX: should be able to use __qualname__ -def _closuredsource(func, alias=''): - """get source code for closured objects; return a dict of 'name' - and 'code blocks'""" - #FIXME: this entire function is a messy messy HACK - # - pollutes global namespace - # - fails if name of freevars are reused - # - can unnecessarily duplicate function code - from dill.detect import freevars - free_vars = freevars(func) - func_vars = {} - # split into 'funcs' and 'non-funcs' - for name,obj in list(free_vars.items()): - if not isfunction(obj): - # get source for 'non-funcs' - free_vars[name] = getsource(obj, force=True, alias=name) - continue - # get source for 'funcs' - fobj = free_vars.pop(name) - src = getsource(fobj, alias) # DO NOT include dependencies - # if source doesn't start with '@', use name as the alias - if not src.lstrip().startswith('@'): #FIXME: 'enclose' in dummy; - src = importable(fobj,alias=name)# wrong ref 'name' - org = getsource(func, alias, enclosing=False, lstrip=True) - src = (src, org) # undecorated first, then target - else: #NOTE: reproduces the code! - org = getsource(func, enclosing=True, lstrip=False) - src = importable(fobj, alias, source=True) # include dependencies - src = (org, src) # target first, then decorated - func_vars[name] = src - src = ''.join(free_vars.values()) - if not func_vars: #FIXME: 'enclose' in dummy; wrong ref 'name' - org = getsource(func, alias, force=True, enclosing=False, lstrip=True) - src = (src, org) # variables first, then target - else: - src = (src, None) # just variables (better '' instead of None?) - func_vars[None] = src - # FIXME: remove duplicates (however, order is important...) - return func_vars - -def importable(obj, alias='', source=None, builtin=True): - """get an importable string (i.e. source code or the import string) - for the given object, including any required objects from the enclosing - and global scope - - This function will attempt to discover the name of the object, or the repr - of the object, or the source code for the object. To attempt to force - discovery of the source code, use source=True, to attempt to force the - use of an import, use source=False; otherwise an import will be sought - for objects not defined in __main__. The intent is to build a string - that can be imported from a python file. - - obj is the object to inspect. If alias is provided, then rename the - object with the given alias. If builtin=True, then force an import for - builtins where possible. - """ - #NOTE: we always 'force', and 'lstrip' as necessary - #NOTE: for 'enclosing', use importable(outermost(obj)) - if source is None: - source = True if isfrommain(obj) else False - elif builtin and isbuiltin(obj): - source = False - tried_source = tried_import = False - while True: - if not source: # we want an import - try: - if _isinstance(obj): # for instances, punt to _importable - return _importable(obj, alias, source=False, builtin=builtin) - src = _closuredimport(obj, alias=alias, builtin=builtin) - if len(src) == 0: - raise NotImplementedError('not implemented') - if len(src) > 1: - raise NotImplementedError('not implemented') - return list(src.values())[0] - except: - if tried_source: raise - tried_import = True - # we want the source - try: - src = _closuredsource(obj, alias=alias) - if len(src) == 0: - raise NotImplementedError('not implemented') - # groan... an inline code stitcher - def _code_stitcher(block): - "stitch together the strings in tuple 'block'" - if block[0] and block[-1]: block = '\n'.join(block) - elif block[0]: block = block[0] - elif block[-1]: block = block[-1] - else: block = '' - return block - # get free_vars first - _src = _code_stitcher(src.pop(None)) - _src = [_src] if _src else [] - # get func_vars - for xxx in src.values(): - xxx = _code_stitcher(xxx) - if xxx: _src.append(xxx) - # make a single source string - if not len(_src): - src = '' - elif len(_src) == 1: - src = _src[0] - else: - src = '\n'.join(_src) - # get source code of objects referred to by obj in global scope - from dill.detect import globalvars - obj = globalvars(obj) #XXX: don't worry about alias? - obj = list(getsource(_obj,name,force=True) for (name,_obj) in obj.items()) - obj = '\n'.join(obj) if obj else '' - # combine all referred-to source (global then enclosing) - if not obj: return src - if not src: return obj - return obj + src - except: - if tried_import: raise - tried_source = True - source = not source - # should never get here - return - - -# backward compatability -def getimportable(obj, alias='', byname=True, explicit=False): - return importable(obj,alias,source=(not byname),builtin=explicit) - #return outdent(_importable(obj,alias,source=(not byname),builtin=explicit)) -def likely_import(obj, passive=False, explicit=False): - return getimport(obj, verify=(not passive), builtin=explicit) -def _likely_import(first, last, passive=False, explicit=True): - return _getimport(first, last, verify=(not passive), builtin=explicit) -_get_name = getname -getblocks_from_history = getblocks - - - -# EOF diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/temp.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/temp.py deleted file mode 100644 index 9dedb411950ca..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/temp.py +++ /dev/null @@ -1,236 +0,0 @@ -#!/usr/bin/env python -# -# Author: Mike McKerns (mmckerns @caltech and @uqfoundation) -# Copyright (c) 2008-2014 California Institute of Technology. -# License: 3-clause BSD. The full license text is available at: -# - http://trac.mystic.cacr.caltech.edu/project/pathos/browser/dill/LICENSE -""" -Methods for serialized objects (or source code) stored in temporary files -and file-like objects. -""" -#XXX: better instead to have functions write to any given file-like object ? -#XXX: currently, all file-like objects are created by the function... - -from __future__ import absolute_import -__all__ = ['dump_source', 'dump', 'dumpIO_source', 'dumpIO',\ - 'load_source', 'load', 'loadIO_source', 'loadIO'] - -from .dill import PY3 - -def b(x): # deal with b'foo' versus 'foo' - import codecs - return codecs.latin_1_encode(x)[0] - -def load_source(file, **kwds): - """load an object that was stored with dill.temp.dump_source - - file: filehandle - alias: string name of stored object - mode: mode to open the file, one of: {'r', 'rb'} - - >>> f = lambda x: x**2 - >>> pyfile = dill.temp.dump_source(f, alias='_f') - >>> _f = dill.temp.load_source(pyfile) - >>> _f(4) - 16 - """ - alias = kwds.pop('alias', None) - mode = kwds.pop('mode', 'r') - fname = getattr(file, 'name', file) # fname=file.name or fname=file (if str) - source = open(fname, mode=mode, **kwds).read() - if not alias: - tag = source.strip().splitlines()[-1].split() - if tag[0] != '#NAME:': - stub = source.splitlines()[0] - raise IOError("unknown name for code: %s" % stub) - alias = tag[-1] - local = {} - exec(source, local) - _ = eval("%s" % alias, local) - return _ - -def dump_source(object, **kwds): - """write object source to a NamedTemporaryFile (instead of dill.dump) -Loads with "import" or "dill.temp.load_source". Returns the filehandle. - - >>> f = lambda x: x**2 - >>> pyfile = dill.temp.dump_source(f, alias='_f') - >>> _f = dill.temp.load_source(pyfile) - >>> _f(4) - 16 - - >>> f = lambda x: x**2 - >>> pyfile = dill.temp.dump_source(f, dir='.') - >>> modulename = os.path.basename(pyfile.name).split('.py')[0] - >>> exec('from %s import f as _f' % modulename) - >>> _f(4) - 16 - -Optional kwds: - If 'alias' is specified, the object will be renamed to the given string. - - If 'prefix' is specified, the file name will begin with that prefix, - otherwise a default prefix is used. - - If 'dir' is specified, the file will be created in that directory, - otherwise a default directory is used. - - If 'text' is specified and true, the file is opened in text - mode. Else (the default) the file is opened in binary mode. On - some operating systems, this makes no difference. - -NOTE: Keep the return value for as long as you want your file to exist ! - """ #XXX: write a "load_source"? - from .source import importable, getname - import tempfile - kwds.pop('suffix', '') # this is *always* '.py' - alias = kwds.pop('alias', '') #XXX: include an alias so a name is known - name = str(alias) or getname(object) - name = "\n#NAME: %s\n" % name - #XXX: assumes kwds['dir'] is writable and on $PYTHONPATH - file = tempfile.NamedTemporaryFile(suffix='.py', **kwds) - file.write(b(''.join([importable(object, alias=alias),name]))) - file.flush() - return file - -def load(file, **kwds): - """load an object that was stored with dill.temp.dump - - file: filehandle - mode: mode to open the file, one of: {'r', 'rb'} - - >>> dumpfile = dill.temp.dump([1, 2, 3, 4, 5]) - >>> dill.temp.load(dumpfile) - [1, 2, 3, 4, 5] - """ - import dill as pickle - mode = kwds.pop('mode', 'rb') - name = getattr(file, 'name', file) # name=file.name or name=file (if str) - return pickle.load(open(name, mode=mode, **kwds)) - -def dump(object, **kwds): - """dill.dump of object to a NamedTemporaryFile. -Loads with "dill.temp.load". Returns the filehandle. - - >>> dumpfile = dill.temp.dump([1, 2, 3, 4, 5]) - >>> dill.temp.load(dumpfile) - [1, 2, 3, 4, 5] - -Optional kwds: - If 'suffix' is specified, the file name will end with that suffix, - otherwise there will be no suffix. - - If 'prefix' is specified, the file name will begin with that prefix, - otherwise a default prefix is used. - - If 'dir' is specified, the file will be created in that directory, - otherwise a default directory is used. - - If 'text' is specified and true, the file is opened in text - mode. Else (the default) the file is opened in binary mode. On - some operating systems, this makes no difference. - -NOTE: Keep the return value for as long as you want your file to exist ! - """ - import dill as pickle - import tempfile - file = tempfile.NamedTemporaryFile(**kwds) - pickle.dump(object, file) - file.flush() - return file - -def loadIO(buffer, **kwds): - """load an object that was stored with dill.temp.dumpIO - - buffer: buffer object - - >>> dumpfile = dill.temp.dumpIO([1, 2, 3, 4, 5]) - >>> dill.temp.loadIO(dumpfile) - [1, 2, 3, 4, 5] - """ - import dill as pickle - if PY3: - from io import BytesIO as StringIO - else: - from StringIO import StringIO - value = getattr(buffer, 'getvalue', buffer) # value or buffer.getvalue - if value != buffer: value = value() # buffer.getvalue() - return pickle.load(StringIO(value)) - -def dumpIO(object, **kwds): - """dill.dump of object to a buffer. -Loads with "dill.temp.loadIO". Returns the buffer object. - - >>> dumpfile = dill.temp.dumpIO([1, 2, 3, 4, 5]) - >>> dill.temp.loadIO(dumpfile) - [1, 2, 3, 4, 5] - """ - import dill as pickle - if PY3: - from io import BytesIO as StringIO - else: - from StringIO import StringIO - file = StringIO() - pickle.dump(object, file) - file.flush() - return file - -def loadIO_source(buffer, **kwds): - """load an object that was stored with dill.temp.dumpIO_source - - buffer: buffer object - alias: string name of stored object - - >>> f = lambda x:x**2 - >>> pyfile = dill.temp.dumpIO_source(f, alias='_f') - >>> _f = dill.temp.loadIO_source(pyfile) - >>> _f(4) - 16 - """ - alias = kwds.pop('alias', None) - source = getattr(buffer, 'getvalue', buffer) # source or buffer.getvalue - if source != buffer: source = source() # buffer.getvalue() - if PY3: source = source.decode() # buffer to string - if not alias: - tag = source.strip().splitlines()[-1].split() - if tag[0] != '#NAME:': - stub = source.splitlines()[0] - raise IOError("unknown name for code: %s" % stub) - alias = tag[-1] - local = {} - exec(source, local) - _ = eval("%s" % alias, local) - return _ - -def dumpIO_source(object, **kwds): - """write object source to a buffer (instead of dill.dump) -Loads by with dill.temp.loadIO_source. Returns the buffer object. - - >>> f = lambda x:x**2 - >>> pyfile = dill.temp.dumpIO_source(f, alias='_f') - >>> _f = dill.temp.loadIO_source(pyfile) - >>> _f(4) - 16 - -Optional kwds: - If 'alias' is specified, the object will be renamed to the given string. - """ - from .source import importable, getname - if PY3: - from io import BytesIO as StringIO - else: - from StringIO import StringIO - alias = kwds.pop('alias', '') #XXX: include an alias so a name is known - name = str(alias) or getname(object) - name = "\n#NAME: %s\n" % name - #XXX: assumes kwds['dir'] is writable and on $PYTHONPATH - file = StringIO() - file.write(b(''.join([importable(object, alias=alias),name]))) - file.flush() - return file - - -del absolute_import - - -# EOF diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/executor.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/executor.py deleted file mode 100644 index 2cfb9d370aad9..0000000000000 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/executor.py +++ /dev/null @@ -1,54 +0,0 @@ -################################################################################ -# 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. -################################################################################ -import sys -import socket -import struct -#argv[1] = port - -s = None -try: - import dill - port = int(sys.argv[1]) - - s = socket.socket(family=socket.AF_INET, type=socket.SOCK_STREAM) - s.connect((socket.gethostbyname("localhost"), port)) - - size = struct.unpack(">i", s.recv(4, socket.MSG_WAITALL))[0] - serialized_operator = s.recv(size, socket.MSG_WAITALL) - - size = struct.unpack(">i", s.recv(4, socket.MSG_WAITALL))[0] - import_string = s.recv(size, socket.MSG_WAITALL).decode("utf-8") - - size = struct.unpack(">i", s.recv(4, socket.MSG_WAITALL))[0] - input_file = s.recv(size, socket.MSG_WAITALL).decode("utf-8") - - size = struct.unpack(">i", s.recv(4, socket.MSG_WAITALL))[0] - output_file = s.recv(size, socket.MSG_WAITALL).decode("utf-8") - - exec(import_string) - - operator = dill.loads(serialized_operator) - operator._configure(input_file, output_file, s) - operator._go() - sys.stdout.flush() - sys.stderr.flush() -except: - sys.stdout.flush() - sys.stderr.flush() - s.send(struct.pack(">i", -2)) - raise \ No newline at end of file diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Collector.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Collector.py index 88ada8843c167..bf35756b54902 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Collector.py +++ b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Collector.py @@ -120,10 +120,6 @@ class TypedCollector(object): def __init__(self, con): self._connection = con - def collectBytes(self, value): - size = pack(">I", len(value)) - self._connection.write(b"".join([Types.TYPE_BYTES, size, value])) - def collect(self, value): if not isinstance(value, (list, tuple)): self._send_field(value) diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Connection.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Connection.py index 7ccc9953016b2..988bf25eff287 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Connection.py +++ b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/connection/Connection.py @@ -17,7 +17,6 @@ ################################################################################ import mmap import socket as SOCKET -import tempfile from struct import pack, unpack from collections import deque import sys @@ -38,8 +37,8 @@ class OneWayBusyBufferingMappedFileConnection(object): - def __init__(self): - self._output_file = open(tempfile.gettempdir() + "/flink_data/output", "rb+") + def __init__(self, output_path): + self._output_file = open(output_path, "rb+") self._file_output_buffer = mmap.mmap(self._output_file.fileno(), MAPPED_FILE_SIZE, mmap.MAP_SHARED, mmap.ACCESS_WRITE) self._out = deque() @@ -61,12 +60,13 @@ def _write_buffer(self): class BufferingTCPMappedFileConnection(object): - def __init__(self, input_file=tempfile.gettempdir() + "/flink_data/input", output_file=tempfile.gettempdir() + "/flink_data/output", socket=None): + def __init__(self, input_file, output_file, port): self._input_file = open(input_file, "rb+") self._output_file = open(output_file, "rb+") self._file_input_buffer = mmap.mmap(self._input_file.fileno(), MAPPED_FILE_SIZE, mmap.MAP_SHARED, mmap.ACCESS_READ) self._file_output_buffer = mmap.mmap(self._output_file.fileno(), MAPPED_FILE_SIZE, mmap.MAP_SHARED, mmap.ACCESS_WRITE) - self._socket = socket + self._socket = SOCKET.socket(family=SOCKET.AF_INET, type=SOCKET.SOCK_STREAM) + self._socket.connect((SOCKET.gethostbyname("localhost"), port)) self._out = deque() self._out_size = 0 @@ -128,8 +128,8 @@ def reset(self): class TwinBufferingTCPMappedFileConnection(BufferingTCPMappedFileConnection): - def __init__(self, input_file=tempfile.gettempdir() + "/flink/data/input", output_file=tempfile.gettempdir() + "/flink/data/output", socket=None): - super(TwinBufferingTCPMappedFileConnection, self).__init__(input_file, output_file, socket) + def __init__(self, input_file, output_file, port): + super(TwinBufferingTCPMappedFileConnection, self).__init__(input_file, output_file, port) self._input = [b"", b""] self._input_offset = [0, 0] self._input_size = [0, 0] diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/functions/Function.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/functions/Function.py index dfbb5c50f38e7..532346220e82a 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/functions/Function.py +++ b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/functions/Function.py @@ -16,7 +16,6 @@ # limitations under the License. ################################################################################ from abc import ABCMeta, abstractmethod -import dill import sys from collections import deque from flink.connection import Connection, Iterator, Collector @@ -32,7 +31,6 @@ def __init__(self): self._collector = None self.context = None self._chain_operator = None - self._meta = None def _configure(self, input_file, output_file, port): self._connection = Connection.BufferingTCPMappedFileConnection(input_file, output_file, port) @@ -42,22 +40,15 @@ def _configure(self, input_file, output_file, port): def _configure_chain(self, collector): if self._chain_operator is not None: - frag = self._meta.split("|") - if "flink/functions" in frag[0]:#lambda function - exec("from flink.functions." + frag[1] + " import " + frag[1]) - else: - self._chain_operator = self._chain_operator.replace(b"__main__", b"plan") - exec("from plan import " + frag[1]) - self._collector = dill.loads(self._chain_operator) + self._collector = self._chain_operator self._collector.context = self.context self._collector._configure_chain(collector) self._collector._open() else: self._collector = collector - def _chain(self, operator, meta): + def _chain(self, operator): self._chain_operator = operator - self._meta = meta @abstractmethod def _run(self): diff --git a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/plan/Environment.py b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/plan/Environment.py index 61c077f400ac8..236eda4821102 100644 --- a/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/plan/Environment.py +++ b/flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/flink/plan/Environment.py @@ -15,14 +15,13 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import inspect from flink.connection import Connection from flink.connection import Collector from flink.plan.DataSet import DataSet from flink.plan.Constants import _Fields, _Identifier from flink.utilities import Switch -import dill import copy +import sys def get_environment(): @@ -34,15 +33,9 @@ def get_environment(): return Environment() -def _dump(function): - return dill.dumps(function, protocol=0, byref=True) - - class Environment(object): def __init__(self): # util - self._connection = Connection.OneWayBusyBufferingMappedFileConnection() - self._collector = Collector.TypedCollector(self._connection) self._counter = 0 #parameters @@ -128,8 +121,41 @@ def execute(self, local=False, debug=False): self._parameters.append(("mode", local)) self._parameters.append(("debug", debug)) self._optimize_plan() - self._send_plan() - self._connection._write_buffer() + + plan_mode = sys.stdin.readline().rstrip('\n') == "plan" + + if plan_mode: + output_path = sys.stdin.readline().rstrip('\n') + self._connection = Connection.OneWayBusyBufferingMappedFileConnection(output_path) + self._collector = Collector.TypedCollector(self._connection) + self._send_plan() + self._connection._write_buffer() + else: + import struct + operator = None + try: + port = int(sys.stdin.readline().rstrip('\n')) + + id = int(sys.stdin.readline().rstrip('\n')) + input_path = sys.stdin.readline().rstrip('\n') + output_path = sys.stdin.readline().rstrip('\n') + + operator = None + for set in self._sets: + if set[_Fields.ID] == id: + operator = set[_Fields.OPERATOR] + if set[_Fields.ID] == -id: + operator = set[_Fields.COMBINEOP] + operator._configure(input_path, output_path, port) + operator._go() + sys.stdout.flush() + sys.stderr.flush() + except: + sys.stdout.flush() + sys.stderr.flush() + if operator is not None: + operator._connection._socket.send(struct.pack(">i", -2)) + raise def _optimize_plan(self): self._find_chains() @@ -157,8 +183,7 @@ def _find_chains(self): if parent_type in udf and len(parent[_Fields.CHILDREN]) == 1: if parent[_Fields.OPERATOR] is not None: function = child[_Fields.COMBINEOP] - meta = str(inspect.getmodule(function)) + "|" + str(function.__class__.__name__) - parent[_Fields.OPERATOR]._chain(_dump(function), meta) + parent[_Fields.OPERATOR]._chain(function) child[_Fields.COMBINE] = False parent[_Fields.NAME] += " -> PythonCombine" for bcvar in child[_Fields.BCVARS]: @@ -170,8 +195,7 @@ def _find_chains(self): parent_op = parent[_Fields.OPERATOR] if parent_op is not None: function = child[_Fields.OPERATOR] - meta = str(inspect.getmodule(function)) + "|" + str(function.__class__.__name__) - parent_op._chain(_dump(function), meta) + parent_op._chain(function) parent[_Fields.NAME] += " -> " + child[_Fields.NAME] parent[_Fields.TYPES] = child[_Fields.TYPES] for grand_child in child[_Fields.CHILDREN]: @@ -233,7 +257,6 @@ def _send_sources(self): def _send_operations(self): collect = self._collector.collect - collectBytes = self._collector.collectBytes for set in self._sets: identifier = set.get(_Fields.IDENTIFIER) collect(set[_Fields.IDENTIFIER]) @@ -251,18 +274,11 @@ def _send_operations(self): collect(set[_Fields.OTHER][_Fields.ID]) collect(set[_Fields.KEY1]) collect(set[_Fields.KEY2]) - collectBytes(_dump(set[_Fields.OPERATOR])) - collect(set[_Fields.META]) collect(set[_Fields.TYPES]) collect(set[_Fields.NAME]) break if case(_Identifier.CROSS, _Identifier.CROSSH, _Identifier.CROSST): collect(set[_Fields.OTHER][_Fields.ID]) - if set[_Fields.OPERATOR] is None: - collect(set[_Fields.OPERATOR]) - else: - collectBytes(_dump(set[_Fields.OPERATOR])) - collect(set[_Fields.META]) collect(set[_Fields.TYPES]) collect(len(set[_Fields.PROJECTIONS])) for p in set[_Fields.PROJECTIONS]: @@ -271,9 +287,6 @@ def _send_operations(self): collect(set[_Fields.NAME]) break if case(_Identifier.REDUCE, _Identifier.GROUPREDUCE): - collectBytes(_dump(set[_Fields.OPERATOR])) - collectBytes(_dump(set[_Fields.COMBINEOP])) - collect(set[_Fields.META]) collect(set[_Fields.TYPES]) collect(set[_Fields.COMBINE]) collect(set[_Fields.NAME]) @@ -282,11 +295,6 @@ def _send_operations(self): collect(set[_Fields.KEY1]) collect(set[_Fields.KEY2]) collect(set[_Fields.OTHER][_Fields.ID]) - if set[_Fields.OPERATOR] is None: - collect(set[_Fields.OPERATOR]) - else: - collectBytes(_dump(set[_Fields.OPERATOR])) - collect(set[_Fields.META]) collect(set[_Fields.TYPES]) collect(len(set[_Fields.PROJECTIONS])) for p in set[_Fields.PROJECTIONS]: @@ -295,8 +303,6 @@ def _send_operations(self): collect(set[_Fields.NAME]) break if case(_Identifier.MAP, _Identifier.MAPPARTITION, _Identifier.FLATMAP, _Identifier.FILTER): - collectBytes(_dump(set[_Fields.OPERATOR])) - collect(set[_Fields.META]) collect(set[_Fields.TYPES]) collect(set[_Fields.NAME]) break diff --git a/pom.xml b/pom.xml index fd3a3cab28d13..455153aee6a74 100644 --- a/pom.xml +++ b/pom.xml @@ -761,8 +761,6 @@ under the License. flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/*.java flink-staging/flink-language-binding/flink-python/src/test/python/org/apache/flink/languagebinding/api/python/flink/test/data_csv flink-staging/flink-language-binding/flink-python/src/test/python/org/apache/flink/languagebinding/api/python/flink/test/data_text - - flink-staging/flink-language-binding/flink-python/src/main/python/org/apache/flink/languagebinding/api/python/dill/** **/flink-bin/conf/slaves **/flink-bin/conf/masters From 3c236b2c89ed84cded0d056ac40fc728089c6f67 Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Thu, 30 Jul 2015 13:54:26 +0530 Subject: [PATCH 042/175] [FLINK-2238][api] Generalized fromCollection(Seq) to fromCollection(Iterable) This closes #956 --- .../org/apache/flink/api/scala/ExecutionEnvironment.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index 28e84584a4bfd..d53c54cdaae6c 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -455,13 +455,13 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { } /** - * Creates a DataSet from the given non-empty [[Seq]]. + * Creates a DataSet from the given non-empty [[Iterable]]. * * Note that this operation will result in a non-parallel data source, i.e. a data source with * a parallelism of one. */ def fromCollection[T: ClassTag : TypeInformation]( - data: Seq[T]): DataSet[T] = { + data: Iterable[T]): DataSet[T] = { require(data != null, "Data must not be null.") val typeInfo = implicitly[TypeInformation[T]] From a56aad74eb35a19623d58b6dd0228ec5bc1bb422 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Thu, 30 Jul 2015 13:49:52 +0200 Subject: [PATCH 043/175] [FLINK-2412] [runtime] Check if parent released before querying in-memory buffer in SpillableSubpartitionView --- .../partition/SpillableSubpartition.java | 2 +- .../partition/SpillableSubpartitionView.java | 4 +++ .../partition/SpillableSubpartitionTest.java | 33 +++++++++++++++++-- 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java index 91f20426a6303..4a18691b7b0c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -59,7 +59,7 @@ class SpillableSubpartition extends ResultSubpartition { private boolean isFinished; /** Flag indicating whether the subpartition has been released. */ - private boolean isReleased; + boolean isReleased; /** The read view to consume this subpartition. */ private ResultSubpartitionView readView; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index d37f04290e16d..972e34bc07535 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -73,6 +73,10 @@ public Buffer getNextBuffer() throws IOException, InterruptedException { // 1) In-memory synchronized (parent.buffers) { + if (parent.isReleased) { + return null; + } + if (parent.spillWriter == null) { if (currentQueuePosition < numberOfBuffers) { Buffer buffer = parent.buffers.get(currentQueuePosition); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index c530eff8996ec..d7e56c8515e80 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider; import org.junit.AfterClass; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; @@ -35,9 +35,12 @@ import java.util.concurrent.Future; import static org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode.SYNC; +import static org.junit.Assert.assertNull; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; public class SpillableSubpartitionTest extends SubpartitionTestBase { @@ -59,7 +62,6 @@ ResultSubpartition createSubpartition() { return new SpillableSubpartition(0, mock(ResultPartition.class), ioManager, SYNC); } - /** * Tests a fix for FLINK-2384. * @@ -118,4 +120,31 @@ public Void call() throws Exception { doneLatch.countDown(); blockingFinish.get(); } + + /** + * Tests a fix for FLINK-2412. + * + * @see FLINK-2412 + */ + @Test + public void testReleasePartitionAndGetNext() throws Exception { + // Create partition and add some buffers + SpillableSubpartition partition = new SpillableSubpartition( + 0, mock(ResultPartition.class), ioManager, SYNC); + + partition.finish(); + + // Create the read view + ResultSubpartitionView readView = spy(partition + .createReadView(new TestInfiniteBufferProvider())); + + // The released state check (of the parent) needs to be independent + // of the released state of the view. + doNothing().when(readView).releaseAllResources(); + + // Release the partition, but the view does not notice yet. + partition.release(); + + assertNull(readView.getNextBuffer()); + } } From ce622aa99f6ec6b4325f11dbd6211a40d97c48ed Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Thu, 30 Jul 2015 14:43:03 +0530 Subject: [PATCH 044/175] [FLINK-2248] add flag to disable sysout logging from cli This closes #957. --- docs/apis/cli.md | 4 + .../org/apache/flink/client/CliFrontend.java | 2 +- .../flink/client/cli/CliFrontendParser.java | 8 ++ .../flink/client/cli/ProgramOptions.java | 13 ++ .../flink/client/CliFrontendLoggingTest.java | 113 ++++++++++++++++++ .../flink/client/testjar/WordCount.java | 5 +- 6 files changed, 141 insertions(+), 4 deletions(-) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java diff --git a/docs/apis/cli.md b/docs/apis/cli.md index 1f550f39e989b..c0ae9b15b3fb0 100644 --- a/docs/apis/cli.md +++ b/docs/apis/cli.md @@ -58,6 +58,10 @@ The command line can be used to ./bin/flink run -p 16 ./examples/flink-java-examples-{{ site.version }}-WordCount.jar \ file:///home/user/hamlet.txt file:///home/user/wordcount_out +- Run example program with flink log output disabled + + ./bin/flink run -q ./examples/flink-java-examples-{{ site.version }}-WordCount.jar + - Run example program on a specific JobManager: ./bin/flink run -m myJMHost:6123 \ diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 71b78bd9570b5..9ef2d5f432ea9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -276,6 +276,7 @@ protected int run(String[] args) { LOG.debug("User parallelism is set to {}", userParallelism); Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName(), userParallelism); + client.setPrintStatusDuringExecution(options.getStdoutLogging()); LOG.debug("Client slots is set to {}", client.getMaxSlots()); if(client.getMaxSlots() != -1 && userParallelism == -1) { logAndSysout("Using the parallelism provided by the remote cluster ("+client.getMaxSlots()+"). " + @@ -604,7 +605,6 @@ protected int executeProgram(PackagedProgram program, Client client, int paralle LOG.info("Starting execution of program"); JobSubmissionResult execResult; try { - client.setPrintStatusDuringExecution(true); execResult = client.run(program, parallelism, wait); } catch (ProgramInvocationException e) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index 2e66a97fb1f6f..ae8499b4d4769 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -51,6 +51,10 @@ public class CliFrontendParser { "The parallelism with which to run the program. Optional flag to override the default value " + "specified in the configuration."); + static final Option LOGGING_OPTION = new Option("q", "sysoutLogging", false, "Whether sysout" + + " " + + "logging is required or not"); + static final Option ARGS_OPTION = new Option("a", "arguments", true, "Program arguments. Arguments can also be added without -a, simply as trailing parameters."); @@ -81,6 +85,8 @@ public class CliFrontendParser { PARALLELISM_OPTION.setRequired(false); PARALLELISM_OPTION.setArgName("parallelism"); + LOGGING_OPTION.setRequired(false); + ARGS_OPTION.setRequired(false); ARGS_OPTION.setArgName("programArgs"); ARGS_OPTION.setArgs(Option.UNLIMITED_VALUES); @@ -107,6 +113,7 @@ public static Options getProgramSpecificOptions(Options options) { options.addOption(CLASS_OPTION); options.addOption(PARALLELISM_OPTION); options.addOption(ARGS_OPTION); + options.addOption(LOGGING_OPTION); // also add the YARN options so that the parser can parse them yarnSessionCLi.getYARNSessionCLIOptions(options); @@ -116,6 +123,7 @@ public static Options getProgramSpecificOptions(Options options) { private static Options getProgramSpecificOptionsWithoutDeprecatedOptions(Options options) { options.addOption(CLASS_OPTION); options.addOption(PARALLELISM_OPTION); + options.addOption(LOGGING_OPTION); return options; } diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java index 5b24a41adf3a6..c45da133cc072 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java @@ -25,6 +25,7 @@ import static org.apache.flink.client.cli.CliFrontendParser.JAR_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.CLASS_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.PARALLELISM_OPTION; +import static org.apache.flink.client.cli.CliFrontendParser.LOGGING_OPTION; /** * Base class for command line options that refer to a JAR file program. @@ -39,6 +40,8 @@ public abstract class ProgramOptions extends CommandLineOptions { private final int parallelism; + private final boolean stdoutLogging; + protected ProgramOptions(CommandLine line) throws CliArgsException { super(line); @@ -77,6 +80,12 @@ else if (args.length > 0) { else { parallelism = -1; } + + if(line.hasOption(LOGGING_OPTION.getOpt())){ + stdoutLogging = false; + } else{ + stdoutLogging = true; + } } public String getJarFilePath() { @@ -94,4 +103,8 @@ public String[] getProgramArgs() { public int getParallelism() { return parallelism; } + + public boolean getStdoutLogging() { + return stdoutLogging; + } } \ No newline at end of file diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java new file mode 100644 index 0000000000000..157e0705860cd --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java @@ -0,0 +1,113 @@ +/* + * 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.client; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; + +import static org.apache.flink.client.CliFrontendTestUtils.getTestJarPath; +import static org.apache.flink.client.CliFrontendTestUtils.getConfigDir; +import static org.junit.Assert.fail; + +public class CliFrontendLoggingTest { + + private static LocalFlinkMiniCluster cluster; + private static Configuration config; + private static String hostPort; + private ByteArrayOutputStream stream = new ByteArrayOutputStream(); + private CliFrontend cli; + private PrintStream output; + + @Before + public void setUp() throws Exception { + stream.reset(); + output = System.out; + System.setOut(new PrintStream(stream)); + + config = new Configuration(); + config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1); + config.setBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, false); + hostPort = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null) + ":" + + config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); + + try { + cluster = new LocalFlinkMiniCluster(config, false, StreamingMode.BATCH_ONLY); + } + catch (Exception e) { + e.printStackTrace(); + fail("Setup of test actor system failed."); + } + + cli = new CliFrontend(getConfigDir()); + } + + @After + public void shutDownActorSystem() { + System.setOut(output); + if(cluster != null){ + cluster.shutdown(); + } + } + + @Test + public void verifyLogging(){ + try { + int ret = cli.run(new String[]{"-m", hostPort, getTestJarPath()}); + System.out.flush(); + assert(ret == 0 && checkForLogs(stream.toString())); + } catch(Exception e){ + e.printStackTrace(); + fail(e.getMessage()); + } finally { + if(cluster != null){ + cluster.shutdown(); + } + } + } + + @Test + public void verifyNoLogging(){ + try { + int ret = cli.run(new String[]{"-q", "-m", hostPort, getTestJarPath()}); + System.out.flush(); + assert(ret == 0 && !checkForLogs(stream.toString())); + } catch(Exception e){ + e.printStackTrace(); + fail(e.getMessage()); + } finally { + if(cluster != null){ + cluster.shutdown(); + } + } + } + + private boolean checkForLogs(String output){ + return output.indexOf("RUNNING") >= 0; + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java index 827bc7777dfbd..b4ff6165a0ae7 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java +++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java @@ -57,12 +57,11 @@ public static void main(String[] args) throws Exception { // emit result if(fileOutput) { counts.writeAsCsv(outputPath, "\n", " "); + // execute program + env.execute("WordCount Example"); } else { counts.print(); } - - // execute program - env.execute("WordCount Example"); } // ************************************************************************* From d3b75e415a76f723c582fc12a98e8c659540bb92 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 31 Jul 2015 17:37:48 +0300 Subject: [PATCH 045/175] Rearranged the Snapshot code. --- .../accumulators/AccumulatorRegistry.java | 8 +- ...Snapshot.java => AccumulatorSnapshot.java} | 38 +- .../SmallAccumulatorSnapshot.java | 55 --- ...torSnapshot.java => UserAccumulators.java} | 48 ++- .../runtime/executiongraph/Execution.java | 12 +- .../executiongraph/ExecutionGraph.java | 370 ++++++++++-------- .../taskmanager/TaskExecutionState.java | 8 +- .../flink/runtime/jobmanager/JobManager.scala | 4 +- .../messages/TaskManagerMessages.scala | 4 +- .../runtime/taskmanager/TaskManager.scala | 4 +- 10 files changed, 289 insertions(+), 262 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/{BaseAccumulatorSnapshot.java => AccumulatorSnapshot.java} (62%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java rename flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/{LargeAccumulatorSnapshot.java => UserAccumulators.java} (57%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java index 555f860ddf7a9..67e4666ada620 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java @@ -90,8 +90,8 @@ public AccumulatorRegistry(Configuration jobConfig, JobID jobID, ExecutionAttemp * * @return a serialized accumulator map */ - public BaseAccumulatorSnapshot getSnapshot() { - BaseAccumulatorSnapshot snapshot; + public AccumulatorSnapshot getSnapshot() { + AccumulatorSnapshot snapshot; Map> largeAccumulatorBlobKeys; SerializedValue>> serializedAccumulators; @@ -102,10 +102,10 @@ public BaseAccumulatorSnapshot getSnapshot() { largeAccumulatorBlobKeys = LargeAccumulatorHelper. storeAccumulatorsToBlobCache(blobServerAddress, userAccumulators); - snapshot = new LargeAccumulatorSnapshot(jobID, taskID, + snapshot = new AccumulatorSnapshot(jobID, taskID, flinkAccumulators, largeAccumulatorBlobKeys); } else { - snapshot = new SmallAccumulatorSnapshot(jobID, taskID, + snapshot = new AccumulatorSnapshot(jobID, taskID, flinkAccumulators, serializedAccumulators); } return snapshot; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java similarity index 62% rename from flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java index 758525ddaff50..d6996dc9d52fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/BaseAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java @@ -20,11 +20,13 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.util.SerializedValue; import java.io.IOException; import java.io.Serializable; +import java.util.List; import java.util.Map; /** @@ -32,21 +34,39 @@ * encapsulate a map of accumulators (user- and system- defined) for a single task. It is used for the * transfer from TaskManagers to the JobManager and from the JobManager to the Client. */ -public class BaseAccumulatorSnapshot implements Serializable { +public class AccumulatorSnapshot implements Serializable { private static final long serialVersionUID = 42L; private final JobID jobID; private final ExecutionAttemptID executionAttemptID; - /** Flink internal accumulators which can be deserialized using the system class loader. */ + /** + * Flink internal accumulators which can be deserialized using the system class loader. + */ private final SerializedValue>> flinkAccumulators; - public BaseAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, - Map> flinkAccumulators) throws IOException { + /** + * User defined accumulators that may require user defined classloader. + */ + private final UserAccumulators userAccumulators; + + public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + Map> oversizedUserAccumulatorBlobKeys) throws IOException { + this.jobID = jobID; + this.executionAttemptID = executionAttemptID; + this.flinkAccumulators = new SerializedValue>>(flinkAccumulators); + this.userAccumulators = new UserAccumulators(oversizedUserAccumulatorBlobKeys); + } + + public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + SerializedValue>> smallUserAccumulators) throws IOException { this.jobID = jobID; this.executionAttemptID = executionAttemptID; this.flinkAccumulators = new SerializedValue>>(flinkAccumulators); + this.userAccumulators = new UserAccumulators(smallUserAccumulators); } public JobID getJobID() { @@ -59,9 +79,19 @@ public ExecutionAttemptID getExecutionAttemptID() { /** * Gets the Flink (internal) accumulators values. + * * @return the serialized map */ public Map> deserializeFlinkAccumulators() throws IOException, ClassNotFoundException { return flinkAccumulators.deserializeValue(ClassLoader.getSystemClassLoader()); } + + /** + * Gets the User-defined accumulators. + * + * @return the user accumulators. + */ + public UserAccumulators getUserAccumulators() { + return userAccumulators; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java deleted file mode 100644 index d8dd2cb94f49a..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/SmallAccumulatorSnapshot.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.runtime.accumulators; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.util.SerializedValue; - -import java.io.IOException; -import java.util.Map; - -/** - * This is a subclass of the BaseAccumulatorSnapshot that serves at storing the task user-defined - * accumulators that are small enough to be sent to the JobManager using akka. It is used for the - * transfer from TaskManagers to the JobManager and from the JobManager to the Client. - * */ -public class SmallAccumulatorSnapshot extends BaseAccumulatorSnapshot { - - /** - * Serialized user accumulators which may require the custom user class loader. - */ - private final SerializedValue>> userAccumulators; - - public SmallAccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, - Map> flinkAccumulators, - SerializedValue>> userAccumulators) throws IOException { - super(jobID, executionAttemptID, flinkAccumulators); - this.userAccumulators = userAccumulators; - } - - /** - * Gets the user-defined accumulators values. - * @return the serialized map - */ - public Map> deserializeSmallUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { - return userAccumulators.deserializeValue(classLoader); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java similarity index 57% rename from flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java index 1710e6d6b621e..243e5163eb99f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java @@ -18,24 +18,19 @@ package org.apache.flink.runtime.accumulators; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.runtime.blob.BlobKey; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.util.SerializedValue; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; -/** - * In case some user-defined accumulators do not fit in an Akka message payload, we store them in the - * blobCache, and put in the snapshot only the mapping between the name of the accumulator, - * and its blobKey in the cache. This clase is a subclass of the BaseAccumulatorSnapshot - * and holds the (potential) references to blobs stored in the BlobCache and containing - * these oversized accumulators. It is used for the transfer from TaskManagers to the - * JobManager and from the JobManager to the Client. - */ -public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { +public class UserAccumulators implements java.io.Serializable { + + /** Serialized user accumulators which may require the custom user class loader. */ + private final SerializedValue>> smallUserAccumulators; /** * In case some accumulators do not fit in an Akka message payload, we store them in the blobCache and put @@ -44,20 +39,37 @@ public class LargeAccumulatorSnapshot extends BaseAccumulatorSnapshot { * */ private final Map> largeUserAccumulatorBlobs; - public LargeAccumulatorSnapshot( - JobID jobID, ExecutionAttemptID executionAttemptID, - Map> flinkAccumulators, - Map> oversizedUserAccumulatorBlobKeys) throws IOException { - super(jobID, executionAttemptID, flinkAccumulators); + public UserAccumulators(Map> oversizedUserAccumulatorBlobKeys) throws IOException { + this.smallUserAccumulators = null; this.largeUserAccumulatorBlobs = oversizedUserAccumulatorBlobKeys; } + + public UserAccumulators(SerializedValue>> userAccumulators) throws IOException { + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulatorBlobs = null; + } + + /** + * Gets the user-defined accumulators values that fit in akka payload. + * @return the serialized map + */ + public Map> deserializeSmallUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { + if(largeUserAccumulatorBlobs != null) { + return Collections.emptyMap(); + } + return smallUserAccumulators.deserializeValue(classLoader); + } + /** - * Gets the BlobKeys of the oversized accumulators that were too big to be sent through akka, and - * had to be stored in the BlobCache. + * Gets the BlobKeys of the oversized accumulators that were too big to be sent through akka. + * These accumulators had to be stored in the BlobCache and their blobKeys are returned here. * @return the maping between accumulator and its blobKeys. */ public Map> getLargeAccumulatorBlobKeys() { + if(smallUserAccumulators != null) { + return Collections.emptyMap(); + } return largeUserAccumulatorBlobs; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 4c08a44462a04..4c5f9339e1641 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -965,18 +965,18 @@ public String getVertexWithAttempt() { /** * Update accumulators (discarded when the Execution has already been terminated). * @param flinkAccumulators the flink internal accumulators - * @param userAccumulators the user accumulators - * @param userLargeAccumulatorBlobKeys the keys to the blobs storing the oversized accumulators sent by + * @param smallUserAccumulators the user accumulators + * @param largeUserAccumulatorBlobKeys the keys to the blobs storing the oversized accumulators sent by * (some) tasks. */ public void setAccumulators(Map> flinkAccumulators, - Map> userAccumulators, - Map> userLargeAccumulatorBlobKeys) { + Map> smallUserAccumulators, + Map> largeUserAccumulatorBlobKeys) { synchronized (accumulatorLock) { if (!state.isTerminal()) { this.flinkAccumulators = flinkAccumulators; - this.smallUserAccumulators = userAccumulators; - this.largeUserAccumulators = userLargeAccumulatorBlobKeys; + this.smallUserAccumulators = smallUserAccumulators; + this.largeUserAccumulators = largeUserAccumulatorBlobKeys; } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 7372ad1ecde22..9f9c7e9b65c2c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -25,10 +25,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot; -import org.apache.flink.runtime.accumulators.LargeAccumulatorSnapshot; -import org.apache.flink.runtime.accumulators.SmallAccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.accumulators.UserAccumulators; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.execution.ExecutionState; @@ -79,24 +78,24 @@ * The execution graph is the central data structure that coordinates the distributed * execution of a data flow. It keeps representations of each parallel task, each * intermediate result, and the communication between them. - * + *

* The execution graph consists of the following constructs: *

    - *
  • The {@link ExecutionJobVertex} represents one vertex from the JobGraph (usually one operation like - * "map" or "join") during execution. It holds the aggregated state of all parallel subtasks. - * The ExecutionJobVertex is identified inside the graph by the {@link JobVertexID}, which it takes - * from the JobGraph's corresponding JobVertex.
  • - *
  • The {@link ExecutionVertex} represents one parallel subtask. For each ExecutionJobVertex, there are - * as many ExecutionVertices as the parallelism. The ExecutionVertex is identified by - * the ExecutionJobVertex and the number of the parallel subtask
  • - *
  • The {@link Execution} is one attempt to execute a ExecutionVertex. There may be multiple Executions - * for the ExecutionVertex, in case of a failure, or in the case where some data needs to be recomputed - * because it is no longer available when requested by later operations. An Execution is always - * identified by an {@link ExecutionAttemptID}. All messages between the JobManager and the TaskManager - * about deployment of tasks and updates in the task status always use the ExecutionAttemptID to - * address the message receiver.
  • + *
  • The {@link ExecutionJobVertex} represents one vertex from the JobGraph (usually one operation like + * "map" or "join") during execution. It holds the aggregated state of all parallel subtasks. + * The ExecutionJobVertex is identified inside the graph by the {@link JobVertexID}, which it takes + * from the JobGraph's corresponding JobVertex.
  • + *
  • The {@link ExecutionVertex} represents one parallel subtask. For each ExecutionJobVertex, there are + * as many ExecutionVertices as the parallelism. The ExecutionVertex is identified by + * the ExecutionJobVertex and the number of the parallel subtask
  • + *
  • The {@link Execution} is one attempt to execute a ExecutionVertex. There may be multiple Executions + * for the ExecutionVertex, in case of a failure, or in the case where some data needs to be recomputed + * because it is no longer available when requested by later operations. An Execution is always + * identified by an {@link ExecutionAttemptID}. All messages between the JobManager and the TaskManager + * about deployment of tasks and updates in the task status always use the ExecutionAttemptID to + * address the message receiver.
  • *
- * + *

*

The ExecutionGraph implements {@link java.io.Serializable}, because it can be archived by * sending it to an archive actor via an actor message. The execution graph does contain some * non-serializable fields. These fields are not required in the archived form and are cleared @@ -109,62 +108,77 @@ public class ExecutionGraph implements Serializable { private static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ExecutionGraph.class, JobStatus.class, "state"); - /** The log object used for debugging. */ + /** + * The log object used for debugging. + */ static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class); - + private static final int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1; // -------------------------------------------------------------------------------------------- - /** The lock used to secure all access to mutable fields, especially the tracking of progress - * within the job. */ + /** + * The lock used to secure all access to mutable fields, especially the tracking of progress + * within the job. + */ private final SerializableObject progressLock = new SerializableObject(); - - /** The ID of the job this graph has been built for. */ + + /** + * The ID of the job this graph has been built for. + */ private final JobID jobID; - /** The name of the original job graph. */ + /** + * The name of the original job graph. + */ private final String jobName; - /** The job configuration that was originally attached to the JobGraph. */ + /** + * The job configuration that was originally attached to the JobGraph. + */ private final Configuration jobConfiguration; - /** All job vertices that are part of this graph */ + /** + * All job vertices that are part of this graph + */ private final ConcurrentHashMap tasks; - /** All vertices, in the order in which they were created **/ + /** + * All vertices, in the order in which they were created + **/ private final List verticesInCreationOrder; - /** All intermediate results that are part of this graph */ + /** + * All intermediate results that are part of this graph + */ private final ConcurrentHashMap intermediateResults; - /** The currently executed tasks, for callbacks */ + /** + * The currently executed tasks, for callbacks + */ private final ConcurrentHashMap currentExecutions; /** * Updates the accumulators during the runtime of a job. Final accumulator results are transferred * through the UpdateTaskExecutionState message. - * @param accumulatorSnapshot The serialized flink and user-defined accumulators + * + * @param snapshot The serialized flink and user-defined accumulators */ - public void updateAccumulators(BaseAccumulatorSnapshot accumulatorSnapshot) { + public void updateAccumulators(AccumulatorSnapshot snapshot) { Map> flinkAccumulators; - Map> smallUserAccumulators = null; - Map> largeUserAccumulators = null; + UserAccumulators userAccumulators = snapshot.getUserAccumulators(); try { - flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - - if(accumulatorSnapshot instanceof SmallAccumulatorSnapshot) { - smallUserAccumulators = ((SmallAccumulatorSnapshot) accumulatorSnapshot). - deserializeSmallUserAccumulators(userClassLoader); - } else if(accumulatorSnapshot instanceof LargeAccumulatorSnapshot) { - largeUserAccumulators = ((LargeAccumulatorSnapshot) accumulatorSnapshot). - getLargeAccumulatorBlobKeys(); - } + flinkAccumulators = snapshot.deserializeFlinkAccumulators(); - ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); + ExecutionAttemptID execID = snapshot.getExecutionAttemptID(); Execution execution = currentExecutions.get(execID); if (execution != null) { - execution.setAccumulators(flinkAccumulators, smallUserAccumulators, largeUserAccumulators); + Map> largeAccumulators = + userAccumulators.getLargeAccumulatorBlobKeys(); + Map> smallAccumulators = + userAccumulators.deserializeSmallUserAccumulators(userClassLoader); + + execution.setAccumulators(flinkAccumulators, smallAccumulators, largeAccumulators); } else { LOG.warn("Received accumulator result for unknown execution {}.", execID); } @@ -173,76 +187,110 @@ public void updateAccumulators(BaseAccumulatorSnapshot accumulatorSnapshot) { } } - /** A list of all libraries required during the job execution. Libraries have to be stored - * inside the BlobService and are referenced via the BLOB keys. */ + /** + * A list of all libraries required during the job execution. Libraries have to be stored + * inside the BlobService and are referenced via the BLOB keys. + */ private final List requiredJarFiles; - /** Listeners that receive messages when the entire job switches it status (such as from - * RUNNING to FINISHED) */ + /** + * Listeners that receive messages when the entire job switches it status (such as from + * RUNNING to FINISHED) + */ private final List jobStatusListenerActors; - /** Listeners that receive messages whenever a single task execution changes its status */ + /** + * Listeners that receive messages whenever a single task execution changes its status + */ private final List executionListenerActors; - /** Timestamps (in milliseconds as returned by {@code System.currentTimeMillis()} when + /** + * Timestamps (in milliseconds as returned by {@code System.currentTimeMillis()} when * the execution graph transitioned into a certain state. The index into this array is the * ordinal of the enum value, i.e. the timestamp when the graph went into state "RUNNING" is - * at {@code stateTimestamps[RUNNING.ordinal()]}. */ + * at {@code stateTimestamps[RUNNING.ordinal()]}. + */ private final long[] stateTimestamps; - /** The timeout for all messages that require a response/acknowledgement */ + /** + * The timeout for all messages that require a response/acknowledgement + */ private final FiniteDuration timeout; // ------ Configuration of the Execution ------- - /** The number of times failed executions should be retried. */ + /** + * The number of times failed executions should be retried. + */ private int numberOfRetriesLeft; - /** The delay that the system should wait before restarting failed executions. */ + /** + * The delay that the system should wait before restarting failed executions. + */ private long delayBeforeRetrying; - /** Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able - * to deploy them immediately. */ + /** + * Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able + * to deploy them immediately. + */ private boolean allowQueuedScheduling = false; - /** The mode of scheduling. Decides how to select the initial set of tasks to be deployed. + /** + * The mode of scheduling. Decides how to select the initial set of tasks to be deployed. * May indicate to deploy all sources, or to deploy everything, or to deploy via backtracking - * from results than need to be materialized. */ + * from results than need to be materialized. + */ private ScheduleMode scheduleMode = ScheduleMode.FROM_SOURCES; - /** Flag that indicate whether the executed dataflow should be periodically snapshotted */ + /** + * Flag that indicate whether the executed dataflow should be periodically snapshotted + */ private boolean snapshotCheckpointsEnabled; - + // ------ Execution status and progress. These values are volatile, and accessed under the lock ------- - /** Current status of the job execution */ + /** + * Current status of the job execution + */ private volatile JobStatus state = JobStatus.CREATED; - /** The exception that caused the job to fail. This is set to the first root exception - * that was not recoverable and triggered job failure */ + /** + * The exception that caused the job to fail. This is set to the first root exception + * that was not recoverable and triggered job failure + */ private volatile Throwable failureCause; - /** The number of job vertices that have reached a terminal state */ + /** + * The number of job vertices that have reached a terminal state + */ private volatile int numFinishedJobVertices; - - + + // ------ Fields that are relevant to the execution and need to be cleared before archiving ------- - /** The scheduler to use for scheduling new tasks as they are needed */ + /** + * The scheduler to use for scheduling new tasks as they are needed + */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private Scheduler scheduler; - /** The classloader for the user code. Needed for calls into user code classes */ + /** + * The classloader for the user code. Needed for calls into user code classes + */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private ClassLoader userClassLoader; - - /** The coordinator for checkpoints, if snapshot checkpoints are enabled */ + + /** + * The coordinator for checkpoints, if snapshot checkpoints are enabled + */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private CheckpointCoordinator checkpointCoordinator; - /** The execution context which is used to execute futures. */ + /** + * The execution context which is used to execute futures. + */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private ExecutionContext executionContext; @@ -265,13 +313,13 @@ public void updateAccumulators(BaseAccumulatorSnapshot accumulatorSnapshot) { Configuration jobConfig, FiniteDuration timeout) { this( - executionContext, - jobId, - jobName, - jobConfig, - timeout, - new ArrayList(), - ExecutionGraph.class.getClassLoader() + executionContext, + jobId, + jobName, + jobConfig, + timeout, + new ArrayList(), + ExecutionGraph.class.getClassLoader() ); } @@ -300,7 +348,7 @@ public ExecutionGraph( this.verticesInCreationOrder = new ArrayList(); this.currentExecutions = new ConcurrentHashMap(); - this.jobStatusListenerActors = new CopyOnWriteArrayList(); + this.jobStatusListenerActors = new CopyOnWriteArrayList(); this.executionListenerActors = new CopyOnWriteArrayList(); this.stateTimestamps = new long[JobStatus.values().length]; @@ -372,10 +420,10 @@ public void enableSnapshotCheckpointing( ExecutionVertex[] tasksToTrigger = collectExecutionVertices(verticesToTrigger); ExecutionVertex[] tasksToWaitFor = collectExecutionVertices(verticesToWaitFor); ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo); - + // disable to make sure existing checkpoint coordinators are cleared disableSnaphotCheckpointing(); - + // create the coordinator that triggers and commits checkpoints and holds the state snapshotCheckpointsEnabled = true; checkpointCoordinator = new CheckpointCoordinator( @@ -386,7 +434,7 @@ public void enableSnapshotCheckpointing( tasksToWaitFor, tasksToCommitTo, userClassLoader); - + // the periodic checkpoint scheduler is activated and deactivated as a result of // job status changes (running -> on, all other states -> off) registerJobStatusListener( @@ -395,19 +443,19 @@ public void enableSnapshotCheckpointing( interval, leaderSessionID)); } - + public void disableSnaphotCheckpointing() { if (state != JobStatus.CREATED) { throw new IllegalStateException("Job must be in CREATED state"); } - + snapshotCheckpointsEnabled = false; if (checkpointCoordinator != null) { checkpointCoordinator.shutdown(); checkpointCoordinator = null; } } - + public boolean isSnapshotCheckpointsEnabled() { return snapshotCheckpointsEnabled; } @@ -423,8 +471,7 @@ private ExecutionVertex[] collectExecutionVertices(List jobV throw new IllegalArgumentException("Can only use ExecutionJobVertices of this ExecutionGraph"); } return jv.getTaskVertices(); - } - else { + } else { ArrayList all = new ArrayList(); for (ExecutionJobVertex jv : jobVertices) { if (jv.getGraph() != this) { @@ -439,9 +486,10 @@ private ExecutionVertex[] collectExecutionVertices(List jobV // -------------------------------------------------------------------------------------------- // Properties and Status of the Execution Graph // -------------------------------------------------------------------------------------------- - + /** * Returns a list of BLOB keys referring to the JAR files required to run this job + * * @return list of BLOB keys referring to the JAR files required to run this job */ public List getRequiredJarFiles() { @@ -499,7 +547,7 @@ public Iterable getVerticesTopologically() { // we return a specific iterator that does not fail with concurrent modifications // the list is append only, so it is safe for that final int numElements = this.verticesInCreationOrder.size(); - + return new Iterable() { @Override public Iterator iterator() { @@ -557,9 +605,10 @@ public ExecutionContext getExecutionContext() { /** * Gets the internal flink accumulator map of maps which contains some metrics. + * * @return A map of accumulators for every executed task. */ - public Map>> getFlinkAccumulators() { + public Map>> getFlinkAccumulators() { Map>> flinkAccumulators = new HashMap>>(); @@ -574,14 +623,15 @@ public Map> /** * This works as cache for already merged accumulators, as, in some cases, * we do not want to remerge accumulators as this may lead to duplicate entries. - * */ + */ private Map> mergedSmallUserAccumulators; /** * Merges all accumulator results from the tasks previously executed in the Executions. + * * @return The accumulator map */ - public Map> aggregateSmallUserAccumulators() { + public Map> aggregateSmallUserAccumulators() { return aggregateSmallUserAccumulators(true); } @@ -589,12 +639,13 @@ public Map> aggregateSmallUserAccumulators() { * Merges all accumulator results from the tasks previously executed in the Executions. * If reaggregate is set to false, then no aggregation is performed, and * the cache merge result is returned. Otherwise accumulators are merged. - * @param reaggregate true if we want to aggregate accumulators, - * false otherwise. + * + * @param reaggregate true if we want to aggregate accumulators, + * false otherwise. * @return The accumulator map */ - public Map> aggregateSmallUserAccumulators(boolean reaggregate) { - if(!reaggregate) { + public Map> aggregateSmallUserAccumulators(boolean reaggregate) { + if (!reaggregate) { return mergedSmallUserAccumulators; } this.mergedSmallUserAccumulators = new HashMap>(); @@ -612,6 +663,7 @@ public Map> aggregateSmallUserAccumulators(boolean reag * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, * thus had to be sent through the BlobCache. + * * @return The accumulator map */ public Map> aggregateLargeUserAccumulatorBlobKeys() { @@ -628,22 +680,23 @@ public Map> aggregateLargeUserAccumulatorBlobKeys() { * Adds new blobKeys referring to blobs of large accumulators to the already existing ones. * These refer to blobs in the blobCache holding accumulators (results of tasks) that did not * fit in an akka frame, thus had to be sent through the BlobCache. - * @param target the initial blobKey map + * + * @param target the initial blobKey map * @param toMerge the new keys to add to the initial map * @return The resulting accumulator map */ - public Map> addLargeUserAccumulatorBlobKeys(Map> target, - Map> toMerge) { - if(target == null) { + public Map> addLargeUserAccumulatorBlobKeys( + Map> target, Map> toMerge) { + if (target == null) { target = new HashMap>(); } mergeLargeUserAccumulatorBlobKeys(target, toMerge); return target; } - private void mergeLargeUserAccumulatorBlobKeys(Map> target, - Map> toMerge) { - if(toMerge == null || toMerge.isEmpty()) { + private void mergeLargeUserAccumulatorBlobKeys( + Map> target, Map> toMerge) { + if (toMerge == null || toMerge.isEmpty()) { return; } @@ -661,6 +714,7 @@ private void mergeLargeUserAccumulatorBlobKeys(Map> target * Merges the partial accumulators referring to the same global accumulator received from the tasks, * and serializes the final result. Each of the partial accumulators contains the partial result * produced by each task, for that specific accumulator. + * * @return The accumulator map with serialized accumulator values. * @throws IOException */ @@ -680,6 +734,7 @@ public Map> getSmallAccumulatorsContentSerialize * objects are serialized, thus merging can still be applied after deserialization. * This method assumes that partial accumulators received from the tasks that refer to * the same global accumulator have already been merged. + * * @return The accumulator map with serialized accumulator objects. * @throws IOException */ @@ -721,6 +776,7 @@ public Map> getSmallAccumulatorsSerialized() thr /** * Returns the a stringified version of the user-defined accumulators. + * * @return an Array containing the StringifiedAccumulatorResult objects */ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { @@ -750,7 +806,7 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { for (Map.Entry> entry : largeAccumulatorMap.entrySet()) { - if(!smallAccumulatorMap.containsKey(entry.getKey())) { + if (!smallAccumulatorMap.containsKey(entry.getKey())) { StringBuilder str = new StringBuilder(); str.append("BlobKeys=[ "); for (BlobKey bk : entry.getValue()) { @@ -801,16 +857,16 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti this.verticesInCreationOrder.add(ejv); } } - + public void scheduleForExecution(Scheduler scheduler) throws JobException { if (scheduler == null) { throw new IllegalArgumentException("Scheduler must not be null."); } - + if (this.scheduler != null && this.scheduler != scheduler) { throw new IllegalArgumentException("Cannot use different schedulers for the same job"); } - + if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) { this.scheduler = scheduler; @@ -835,8 +891,7 @@ public void scheduleForExecution(Scheduler scheduler) throws JobException { // go back from vertices that need computation to the ones we need to run throw new JobException("BACKTRACKING is currently not supported as schedule mode."); } - } - else { + } else { throw new IllegalStateException("Job may only be scheduled from state " + JobStatus.CREATED); } } @@ -844,7 +899,7 @@ public void scheduleForExecution(Scheduler scheduler) throws JobException { public void cancel() { while (true) { JobStatus current = state; - + if (current == JobStatus.RUNNING || current == JobStatus.CREATED) { if (transitionState(current, JobStatus.CANCELLING)) { for (ExecutionJobVertex ejv : verticesInCreationOrder) { @@ -852,8 +907,7 @@ public void cancel() { } return; } - } - else { + } else { // no need to treat other states return; } @@ -865,8 +919,7 @@ public void fail(Throwable t) { JobStatus current = state; if (current == JobStatus.FAILED || current == JobStatus.FAILING) { return; - } - else if (transitionState(current, JobStatus.FAILING, t)) { + } else if (transitionState(current, JobStatus.FAILING, t)) { this.failureCause = t; if (!verticesInCreationOrder.isEmpty()) { @@ -878,10 +931,10 @@ else if (transitionState(current, JobStatus.FAILING, t)) { // set the state of the job to failed transitionState(JobStatus.FAILING, JobStatus.FAILED, t); } - + return; } - + // no need to treat other states } } @@ -913,7 +966,7 @@ public void restart() { } numFinishedJobVertices = 0; transitionState(JobStatus.RESTARTING, JobStatus.CREATED); - + // if we have checkpointed state, reload it into the executions if (checkpointCoordinator != null) { checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), false, false); @@ -921,8 +974,7 @@ public void restart() { } scheduleForExecution(scheduler); - } - catch (Throwable t) { + } catch (Throwable t) { fail(t); } } @@ -936,7 +988,7 @@ public void prepareForArchiving() { } // "unpack" execution config before we throw away the usercode classloader. try { - executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(jobConfiguration, ExecutionConfig.CONFIG_KEY,userClassLoader); + executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(jobConfiguration, ExecutionConfig.CONFIG_KEY, userClassLoader); } catch (Exception e) { LOG.warn("Error deserializing the execution config while archiving the execution graph", e); } @@ -963,6 +1015,7 @@ public ExecutionConfig getExecutionConfig() { /** * For testing: This waits until the job execution has finished. + * * @throws InterruptedException */ public void waitUntilFinished() throws InterruptedException { @@ -972,7 +1025,7 @@ public void waitUntilFinished() throws InterruptedException { } } } - + private boolean transitionState(JobStatus current, JobStatus newState) { return transitionState(current, newState, null); } @@ -986,8 +1039,7 @@ private boolean transitionState(JobStatus current, JobStatus newState, Throwable stateTimestamps[newState.ordinal()] = System.currentTimeMillis(); notifyJobStatusChange(newState, error); return true; - } - else { + } else { return false; } } @@ -999,27 +1051,25 @@ void jobVertexInFinalState(ExecutionJobVertex ev) { } numFinishedJobVertices++; - + if (numFinishedJobVertices == verticesInCreationOrder.size()) { - + // we are done, transition to the final state JobStatus current; while (true) { current = this.state; - + if (current == JobStatus.RUNNING) { if (transitionState(current, JobStatus.FINISHED)) { postRunCleanup(); break; } - } - else if (current == JobStatus.CANCELLING) { + } else if (current == JobStatus.CANCELLING) { if (transitionState(current, JobStatus.CANCELED)) { postRunCleanup(); break; } - } - else if (current == JobStatus.FAILING) { + } else if (current == JobStatus.FAILING) { if (numberOfRetriesLeft > 0 && transitionState(current, JobStatus.RESTARTING)) { numberOfRetriesLeft--; future(new Callable() { @@ -1028,8 +1078,7 @@ public Object call() throws Exception { try { LOG.info("Delaying retry of job execution for {} ms ...", delayBeforeRetrying); Thread.sleep(delayBeforeRetrying); - } - catch(InterruptedException e){ + } catch (InterruptedException e) { // should only happen on shutdown } restart(); @@ -1037,13 +1086,11 @@ public Object call() throws Exception { } }, executionContext); break; - } - else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, failureCause)) { + } else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, failureCause)) { postRunCleanup(); break; } - } - else { + } else { fail(new Exception("ExecutionGraph went into final state from state " + current)); } } @@ -1054,7 +1101,7 @@ else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, } } } - + private void postRunCleanup() { try { CheckpointCoordinator coord = this.checkpointCoordinator; @@ -1062,8 +1109,7 @@ private void postRunCleanup() { if (coord != null) { coord.shutdown(); } - } - catch (Exception e) { + } catch (Exception e) { LOG.error("Error while cleaning up after execution", e); } } @@ -1074,6 +1120,7 @@ private void postRunCleanup() { /** * Updates the state of the Task and sets the final accumulator results. + * * @param state * @return */ @@ -1085,27 +1132,23 @@ public boolean updateState(TaskExecutionState state) { case RUNNING: return attempt.switchToRunning(); case FINISHED: - BaseAccumulatorSnapshot accumulatorSnapshot = state.getAccumulators(); + AccumulatorSnapshot accumulatorSnapshot = state.getAccumulators(); Map> flinkAccumulators = null; - Map> smallUserAccumulators = null; - Map> largeUserAccumulators = null; + UserAccumulators userAccumulators = accumulatorSnapshot.getUserAccumulators(); + Map> smallAccumulators = null; + Map> largeAccumulators = null; try { flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - if(accumulatorSnapshot instanceof SmallAccumulatorSnapshot) { - smallUserAccumulators = ((SmallAccumulatorSnapshot) accumulatorSnapshot). - deserializeSmallUserAccumulators(userClassLoader); - } else if(accumulatorSnapshot instanceof LargeAccumulatorSnapshot) { - largeUserAccumulators = ((LargeAccumulatorSnapshot) accumulatorSnapshot). - getLargeAccumulatorBlobKeys(); - } + largeAccumulators = userAccumulators.getLargeAccumulatorBlobKeys(); + smallAccumulators = userAccumulators.deserializeSmallUserAccumulators(userClassLoader); } catch (Exception e) { // Exceptions would be thrown in the future here LOG.error("Failed to deserialize final accumulator results.", e); } - attempt.markFinished(flinkAccumulators, smallUserAccumulators, largeUserAccumulators); + attempt.markFinished(flinkAccumulators, smallAccumulators, largeAccumulators); return true; case CANCELED: attempt.cancelingComplete(); @@ -1119,8 +1162,7 @@ public boolean updateState(TaskExecutionState state) { attempt.fail(new Exception("TaskManager sent illegal state update: " + state.getExecutionState())); return false; } - } - else { + } else { return false; } } @@ -1132,8 +1174,7 @@ public void scheduleOrUpdateConsumers(ResultPartitionID partitionId) { if (execution == null) { fail(new IllegalStateException("Cannot find execution for execution ID " + partitionId.getPartitionId())); - } - else if (execution.getVertex() == null){ + } else if (execution.getVertex() == null) { fail(new IllegalStateException("Execution with execution ID " + partitionId.getPartitionId() + " has no vertex assigned.")); } else { @@ -1175,31 +1216,30 @@ public void registerExecutionListener(ActorGateway listener) { this.executionListenerActors.add(listener); } } - - + + private void notifyJobStatusChange(JobStatus newState, Throwable error) { if (jobStatusListenerActors.size() > 0) { ExecutionGraphMessages.JobStatusChanged message = new ExecutionGraphMessages.JobStatusChanged(jobID, newState, System.currentTimeMillis(), error); - for (ActorGateway listener: jobStatusListenerActors) { + for (ActorGateway listener : jobStatusListenerActors) { listener.tell(message); } } } - + void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionAttemptID executionID, ExecutionState - newExecutionState, Throwable error) - { + newExecutionState, Throwable error) { ExecutionJobVertex vertex = getJobVertex(vertexId); if (executionListenerActors.size() > 0) { String message = error == null ? null : ExceptionUtils.stringifyException(error); ExecutionGraphMessages.ExecutionStateChanged actorMessage = - new ExecutionGraphMessages.ExecutionStateChanged(jobID, vertexId, vertex.getJobVertex().getName(), - vertex.getParallelism(), subtask, - executionID, newExecutionState, - System.currentTimeMillis(), message); + new ExecutionGraphMessages.ExecutionStateChanged(jobID, vertexId, vertex.getJobVertex().getName(), + vertex.getParallelism(), subtask, + executionID, newExecutionState, + System.currentTimeMillis(), message); for (ActorGateway listener : executionListenerActors) { listener.tell(actorMessage); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java index fac573aa861eb..063701752c175 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java @@ -20,7 +20,7 @@ import java.util.Arrays; -import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.api.common.JobID; @@ -54,7 +54,7 @@ public class TaskExecutionState implements java.io.Serializable { private transient Throwable cachedError; /** Serialized flink and user-defined accumulators */ - private final BaseAccumulatorSnapshot accumulators; + private final AccumulatorSnapshot accumulators; /** * Creates a new task execution state update, with no attached exception and no accumulators. @@ -102,7 +102,7 @@ public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, */ public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState executionState, Throwable error, - BaseAccumulatorSnapshot accumulators) { + AccumulatorSnapshot accumulators) { if (jobID == null || executionId == null || executionState == null) { @@ -205,7 +205,7 @@ public JobID getJobID() { /** * Gets flink and user-defined accumulators in serialized form. */ - public BaseAccumulatorSnapshot getAccumulators() { + public AccumulatorSnapshot getAccumulators() { return accumulators; } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 55ac7fa16dd4e..040a7c948ae4a 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -30,7 +30,7 @@ import grizzled.slf4j.Logger import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.runtime.accumulators.{BaseAccumulatorSnapshot, LargeAccumulatorHelper} +import org.apache.flink.runtime.accumulators.{AccumulatorSnapshot, LargeAccumulatorHelper} import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.blob.{BlobKey, BlobServer} import org.apache.flink.runtime.client._ @@ -970,7 +970,7 @@ class JobManager( * Updates the accumulators reported from a task manager via the Heartbeat message. * @param accumulators list of accumulator snapshots */ - private def updateAccumulators(accumulators : Seq[BaseAccumulatorSnapshot]) = { + private def updateAccumulators(accumulators : Seq[AccumulatorSnapshot]) = { accumulators foreach { case accumulatorEvent => currentJobs.get(accumulatorEvent.getJobID) match { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala index 482cb6995d3de..6cb571ce7c63e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala @@ -18,7 +18,7 @@ package org.apache.flink.runtime.messages -import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.instance.InstanceID /** @@ -56,7 +56,7 @@ object TaskManagerMessages { * @param accumulators Accumulators of tasks serialized as Tuple2[internal, user-defined] */ case class Heartbeat(instanceID: InstanceID, metricsReport: Array[Byte], - accumulators: Seq[BaseAccumulatorSnapshot]) + accumulators: Seq[AccumulatorSnapshot]) // -------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index ec0bb105e3edf..4263da72719bc 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -34,7 +34,7 @@ import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration, IllegalConfigurationException} -import org.apache.flink.runtime.accumulators.BaseAccumulatorSnapshot +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.blob.{BlobCache, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager @@ -1040,7 +1040,7 @@ class TaskManager( val metricsReport: Array[Byte] = metricRegistryMapper.writeValueAsBytes(metricRegistry) val accumulatorEvents = - scala.collection.mutable.Buffer[BaseAccumulatorSnapshot]() + scala.collection.mutable.Buffer[AccumulatorSnapshot]() runningTasks foreach { case (execID, task) => From b0f237990cc52729ff1eb1de29b65e118bf59aa3 Mon Sep 17 00:00:00 2001 From: ffbin <869218239@qq.com> Date: Fri, 31 Jul 2015 11:51:25 +0800 Subject: [PATCH 046/175] [FLINK-2446] [streaming] Fix SocketTextStreamFunction memory leak on reconnect Closes #965 --- .../streaming/api/functions/source/SocketTextStreamFunction.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java index fb66f16c4759f..a55a56d7b0f1f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java @@ -102,6 +102,7 @@ public void streamFromSocket(SourceContext ctx, Socket socket) throws Ex success = true; } catch (ConnectException ce) { Thread.sleep(CONNECTION_RETRY_SLEEP); + socket.close(); } } From c58ba3da90884846a52b174098c782eb08308c25 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 2 Aug 2015 15:51:14 +0200 Subject: [PATCH 047/175] [hotfix] Fix SourceStreamTaskTest to switch source to running before checkpointing --- .../runtime/tasks/SourceStreamTaskTest.java | 72 ++++++++++--------- 1 file changed, 39 insertions(+), 33 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index 0f6e5f1e11166..232485dfe5a9f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -15,21 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks; +package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.util.TestHarnessUtil; + import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -98,38 +98,46 @@ public void testCheckpointing() throws Exception { final int SOURCE_CHECKPOINT_DELAY = 1000; // how many random values we sum up in storeCheckpoint final int SOURCE_READ_DELAY = 1; // in ms - - final TupleTypeInfo> typeInfo = new TupleTypeInfo>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - final SourceStreamTask> sourceTask = new SourceStreamTask>(); - final StreamTaskTestHarness> testHarness = new StreamTaskTestHarness>(sourceTask, typeInfo); - - StreamConfig streamConfig = testHarness.getStreamConfig(); - StreamSource> sourceOperator = new StreamSource>(new MockSource(NUM_ELEMENTS, SOURCE_CHECKPOINT_DELAY, SOURCE_READ_DELAY)); - streamConfig.setStreamOperator(sourceOperator); - - ExecutorService executor = Executors.newFixedThreadPool(10); - Future[] checkpointerResults = new Future[NUM_CHECKPOINTERS]; - for (int i = 0; i < NUM_CHECKPOINTERS; i++) { - checkpointerResults[i] = executor.submit(new Checkpointer(NUM_CHECKPOINTS, CHECKPOINT_INTERVAL, sourceTask)); - } - - testHarness.invoke(); - testHarness.waitForTaskCompletion(); - - // Get the result from the checkpointers, if these threw an exception it - // will be rethrown here - for (int i = 0; i < NUM_CHECKPOINTERS; i++) { - if (!checkpointerResults[i].isDone()) { - checkpointerResults[i].cancel(true); + try { + final TupleTypeInfo> typeInfo = new TupleTypeInfo>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + final SourceStreamTask> sourceTask = new SourceStreamTask>(); + final StreamTaskTestHarness> testHarness = new StreamTaskTestHarness>(sourceTask, typeInfo); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamSource> sourceOperator = new StreamSource>(new MockSource(NUM_ELEMENTS, SOURCE_CHECKPOINT_DELAY, SOURCE_READ_DELAY)); + streamConfig.setStreamOperator(sourceOperator); + + // prepare the + + Future[] checkpointerResults = new Future[NUM_CHECKPOINTERS]; + + // invoke this first, so the tasks are actually running when the checkpoints are scheduled + testHarness.invoke(); + + for (int i = 0; i < NUM_CHECKPOINTERS; i++) { + checkpointerResults[i] = executor.submit(new Checkpointer(NUM_CHECKPOINTS, CHECKPOINT_INTERVAL, sourceTask)); } - if (!checkpointerResults[i].isCancelled()) { - checkpointerResults[i].get(); + + testHarness.waitForTaskCompletion(); + + // Get the result from the checkpointers, if these threw an exception it + // will be rethrown here + for (int i = 0; i < NUM_CHECKPOINTERS; i++) { + if (!checkpointerResults[i].isDone()) { + checkpointerResults[i].cancel(true); + } + if (!checkpointerResults[i].isCancelled()) { + checkpointerResults[i].get(); + } } + + List> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + Assert.assertEquals(NUM_ELEMENTS, resultElements.size()); + } + finally { + executor.shutdown(); } - - List> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); - Assert.assertEquals(NUM_ELEMENTS, resultElements.size()); } private static class MockSource implements SourceFunction>, Checkpointed { @@ -267,9 +275,7 @@ public void run(SourceContext ctx) throws Exception { } @Override - public void cancel() { - - } + public void cancel() {} } } From 645d7cd9e437d3de301ca0ce5c9cdc87bcce494b Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 2 Aug 2015 15:54:44 +0200 Subject: [PATCH 048/175] [FLINK-2461] [tests] Guard tests that rely on unresolvable host names with the appropriate assumption. --- .../RemoteExecutorHostnameResolutionTest.java | 24 +++++++++++++++++++ .../program/ClientHostnameResolutionTest.java | 24 +++++++++++++++++++ 2 files changed, 48 insertions(+) diff --git a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java index a1bd0e2dd839e..929314815eead 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java @@ -23,11 +23,13 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.junit.Test; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.Collections; import static org.junit.Assert.fail; +import static org.junit.Assume.assumeTrue; public class RemoteExecutorHostnameResolutionTest { @@ -37,6 +39,9 @@ public class RemoteExecutorHostnameResolutionTest { @Test public void testUnresolvableHostname1() { + + checkPreconditions(); + try { RemoteExecutor exec = new RemoteExecutor(nonExistingHostname, port); exec.executePlan(getProgram()); @@ -54,6 +59,9 @@ public void testUnresolvableHostname1() { @Test public void testUnresolvableHostname2() { + + checkPreconditions(); + try { InetSocketAddress add = new InetSocketAddress(nonExistingHostname, port); RemoteExecutor exec = new RemoteExecutor(add, Collections.emptyList()); @@ -75,4 +83,20 @@ private static Plan getProgram() { env.fromElements(1, 2, 3).output(new DiscardingOutputFormat()); return env.createProgramPlan(); } + + private static void checkPreconditions() { + // the test can only work if the invalid URL cannot be resolves + // some internet providers resolve unresolvable URLs to navigational aid servers, + // voiding this test. + boolean throwsException; + try { + //noinspection ResultOfMethodCallIgnored + InetAddress.getByName(nonExistingHostname); + throwsException = false; + } + catch (UnknownHostException e) { + throwsException = true; + } + assumeTrue(throwsException); + } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java index 2cdb1a0a93254..41294e62b364d 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java @@ -22,10 +22,12 @@ import org.apache.flink.configuration.Configuration; import org.junit.Test; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; import static org.junit.Assert.*; +import static org.junit.Assume.*; /** * Tests that verify that the client correctly handles non-resolvable host names and does not @@ -37,6 +39,9 @@ public class ClientHostnameResolutionTest { @Test public void testUnresolvableHostname1() { + + checkPreconditions(); + try { InetSocketAddress addr = new InetSocketAddress(nonExistingHostname, 17234); new Client(addr, new Configuration(), getClass().getClassLoader(), 1); @@ -54,6 +59,9 @@ public void testUnresolvableHostname1() { @Test public void testUnresolvableHostname2() { + + checkPreconditions(); + try { Configuration config = new Configuration(); config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname); @@ -71,4 +79,20 @@ public void testUnresolvableHostname2() { fail(e.getMessage()); } } + + private static void checkPreconditions() { + // the test can only work if the invalid URL cannot be resolves + // some internet providers resolve unresolvable URLs to navigational aid servers, + // voiding this test. + boolean throwsException; + try { + //noinspection ResultOfMethodCallIgnored + InetAddress.getByName(nonExistingHostname); + throwsException = false; + } + catch (UnknownHostException e) { + throwsException = true; + } + assumeTrue(throwsException); + } } From 9311b9a9da57796e1eb91aa0ec5fa8948b732a47 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 29 Jul 2015 13:09:03 +0200 Subject: [PATCH 049/175] [FLINK-2427] [streaming] Make the BarrierBuffer more robust against lost/missing checkpoint barriers. Checkpoint barriers are now tolerated to be lost (as may happen if the checkpoint triggering actor messages are lost). This is realized by allowing the BarrierBuffer to maintain multiple queues of blocked inputs. The patch also reworks the buffer spilling logic, to increase I/O efficiency, and reduce the main memory footprint in cases where the buffers have little contents (low flush timeouts). --- .../network/partition/consumer/InputGate.java | 1 + .../partition/consumer/SingleInputGate.java | 10 + .../partition/consumer/UnionInputGate.java | 13 + .../runtime/util/DataInputDeserializer.java | 8 +- .../streaming/runtime/io/BarrierBuffer.java | 108 ++-- .../streaming/runtime/io/BarrierTracker.java | 19 +- .../streaming/runtime/io/BufferSpiller.java | 371 ++++++++++++-- .../runtime/io/FreeingBufferRecycler.java} | 11 +- .../streaming/runtime/io/SpillReader.java | 78 --- .../runtime/io/SpillingBufferOrEvent.java | 66 --- .../consumer/StreamTestSingleInputGate.java | 2 + .../io/BarrierBufferMassiveRandomTest.java | 11 +- .../runtime/io/BarrierBufferTest.java | 205 +++++++- .../runtime/io/BarrierTrackerTest.java | 9 +- .../runtime/io/BufferSpillerTest.java | 390 ++++++++++++++ .../io/SpilledBufferOrEventSequenceTest.java | 482 ++++++++++++++++++ .../runtime/io/SpillingBufferOrEventTest.java | 115 ----- .../runtime/io/StreamRecordWriterTest.java | 6 +- .../flink/streaming/runtime/io/TestEvent.java | 88 ++++ 19 files changed, 1620 insertions(+), 373 deletions(-) rename flink-staging/flink-streaming/flink-streaming-core/src/{test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java => main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java} (73%) delete mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillReader.java delete mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEvent.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java delete mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java index af089fcb0190b..c4f9dc4bb73ea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java @@ -37,4 +37,5 @@ public interface InputGate { void registerListener(EventListener listener); + int getPageSize(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 0aebcae64ffd9..80a79d26b8b66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -211,6 +211,16 @@ BufferProvider getBufferProvider() { return bufferPool; } + @Override + public int getPageSize() { + if (bufferPool != null) { + return bufferPool.getMemorySegmentSize(); + } + else { + throw new IllegalStateException("Input gate has not been initialized with buffers."); + } + } + // ------------------------------------------------------------------------ // Setup/Life-cycle // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 1f974de5818ea..730ead21e461e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -182,6 +182,19 @@ public void registerListener(EventListener listener) { inputGateListener.registerListener(listener); } + @Override + public int getPageSize() { + int pageSize = -1; + for (InputGate gate : inputGates) { + if (pageSize == -1) { + pageSize = gate.getPageSize(); + } else if (gate.getPageSize() != pageSize) { + throw new IllegalStateException("Found input gates with different page sizes."); + } + } + return pageSize; + } + /** * Data availability listener at all unioned input gates. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/DataInputDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/DataInputDeserializer.java index 9915abaf18786..e8e8f6d52517a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/DataInputDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/DataInputDeserializer.java @@ -109,7 +109,7 @@ public byte readByte() throws IOException { @Override public char readChar() throws IOException { if (this.position < this.end - 1) { - return (char) (((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0)); + return (char) (((this.buffer[this.position++] & 0xff) << 8) | (this.buffer[this.position++] & 0xff)); } else { throw new EOFException(); } @@ -205,7 +205,7 @@ public long readLong() throws IOException { @Override public short readShort() throws IOException { if (position >= 0 && position < this.end - 1) { - return (short) ((((this.buffer[position++]) & 0xff) << 8) | (((this.buffer[position++]) & 0xff) << 0)); + return (short) ((((this.buffer[position++]) & 0xff) << 8) | ((this.buffer[position++]) & 0xff)); } else { throw new EOFException(); } @@ -271,7 +271,7 @@ public String readUTF() throws IOException { if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { throw new UTFDataFormatException("malformed input around byte " + (count - 1)); } - chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0)); + chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F)); break; default: /* 10xx xxxx, 1111 xxxx */ @@ -294,7 +294,7 @@ public int readUnsignedByte() throws IOException { @Override public int readUnsignedShort() throws IOException { if (this.position < this.end - 1) { - return ((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0); + return ((this.buffer[this.position++] & 0xff) << 8) | (this.buffer[this.position++] & 0xff); } else { throw new EOFException(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index 466b8f75f9287..0441937eb5594 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -17,7 +17,6 @@ package org.apache.flink.streaming.runtime.io; -import java.io.File; import java.io.IOException; import java.util.ArrayDeque; @@ -50,12 +49,17 @@ public class BarrierBuffer implements CheckpointBarrierHandler { /** The total number of channels that this buffer handles data from */ private final int totalNumberOfInputChannels; - - private final SpillReader spillReader; - private final BufferSpiller bufferSpiller; - private ArrayDeque nonProcessed; - private ArrayDeque blockedNonProcessed; + /** To utility to write blocked data to a file channel */ + private final BufferSpiller bufferSpiller; + + /** The pending blocked buffer/event sequences. Must be consumed before requesting + * further data from the input gate. */ + private final ArrayDeque queuedBuffered; + + /** The sequence of buffers/events that has been unblocked and must now be consumed + * before requesting further data from the input gate */ + private BufferSpiller.SpilledBufferOrEventSequence currentBuffered; /** Handler that receives the checkpoint notifications */ private EventListener checkpointHandler; @@ -69,17 +73,21 @@ public class BarrierBuffer implements CheckpointBarrierHandler { /** Flag to indicate whether we have drawn all available input */ private boolean endOfStream; - + + /** + * + * @param inputGate Teh input gate to draw the buffers and events from. + * @param ioManager The I/O manager that gives access to the temp directories. + * + * @throws IOException Thrown, when the spilling to temp files cannot be initialized. + */ public BarrierBuffer(InputGate inputGate, IOManager ioManager) throws IOException { this.inputGate = inputGate; this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); this.blockedChannels = new boolean[this.totalNumberOfInputChannels]; - this.nonProcessed = new ArrayDeque(); - this.blockedNonProcessed = new ArrayDeque(); - - this.bufferSpiller = new BufferSpiller(ioManager); - this.spillReader = new SpillReader(); + this.bufferSpiller = new BufferSpiller(ioManager, inputGate.getPageSize()); + this.queuedBuffered = new ArrayDeque(); } // ------------------------------------------------------------------------ @@ -90,15 +98,25 @@ public BarrierBuffer(InputGate inputGate, IOManager ioManager) throws IOExceptio public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException { while (true) { // process buffered BufferOrEvents before grabbing new ones - final SpillingBufferOrEvent nextBuffered = nonProcessed.pollFirst(); - final BufferOrEvent next = nextBuffered == null ? - inputGate.getNextBufferOrEvent() : - nextBuffered.getBufferOrEvent(); + BufferOrEvent next; + if (currentBuffered != null) { + next = currentBuffered.getNext(); + if (next == null) { + currentBuffered = queuedBuffered.pollFirst(); + if (currentBuffered != null) { + currentBuffered.open(); + } + return getNextNonBlocked(); + } + } + else { + next = inputGate.getNextBufferOrEvent(); + } if (next != null) { if (isBlocked(next.getChannelIndex())) { // if the channel is blocked we, we just store the BufferOrEvent - blockedNonProcessed.add(new SpillingBufferOrEvent(next, bufferSpiller, spillReader)); + bufferSpiller.add(next); } else if (next.isBuffer() || next.getEvent().getClass() != CheckpointBarrier.class) { return next; @@ -181,25 +199,17 @@ public void registerCheckpointEventHandler(EventListener chec @Override public boolean isEmpty() { - return nonProcessed.isEmpty() && blockedNonProcessed.isEmpty(); + return currentBuffered == null; } @Override public void cleanup() throws IOException { bufferSpiller.close(); - File spillfile1 = bufferSpiller.getSpillFile(); - if (spillfile1 != null) { - if (!spillfile1.delete()) { - LOG.warn("Cannot remove barrier buffer spill file: " + spillfile1.getAbsolutePath()); - } + if (currentBuffered != null) { + currentBuffered.cleanup(); } - - spillReader.close(); - File spillfile2 = spillReader.getSpillFile(); - if (spillfile2 != null) { - if (!spillfile2.delete()) { - LOG.warn("Cannot remove barrier buffer spill file: " + spillfile2.getAbsolutePath()); - } + for (BufferSpiller.SpilledBufferOrEventSequence seq : queuedBuffered) { + seq.cleanup(); } } @@ -233,7 +243,8 @@ private void onBarrier(int channelIndex) throws IOException { } /** - * Releases the blocks on all channels. + * Releases the blocks on all channels. Makes sure the just written data + * is the next to be consumed. */ private void releaseBlocks() throws IOException { if (LOG.isDebugEnabled()) { @@ -244,27 +255,36 @@ private void releaseBlocks() throws IOException { blockedChannels[i] = false; } numReceivedBarriers = 0; - - if (nonProcessed.isEmpty()) { - // swap the queues - ArrayDeque empty = nonProcessed; - nonProcessed = blockedNonProcessed; - blockedNonProcessed = empty; + + if (currentBuffered == null) { + // common case: no more buffered data + currentBuffered = bufferSpiller.rollOver(); + if (currentBuffered != null) { + currentBuffered.open(); + } } else { - throw new IllegalStateException("Unconsumed data from previous checkpoint alignment " + - "when starting next checkpoint alignment"); + // uncommon case: buffered data pending + // push back the pending data + queuedBuffered.addFirst(currentBuffered); + + // since we did not fully drain the previous sequence, we need to allocate a new buffer for this one + currentBuffered = bufferSpiller.rollOverWithNewBuffer(); + if (currentBuffered != null) { + currentBuffered.open(); + } } - - // roll over the spill files - spillReader.setSpillFile(bufferSpiller.getSpillFile()); - bufferSpiller.resetSpillFile(); } // ------------------------------------------------------------------------ // For Testing // ------------------------------------------------------------------------ + /** + * Gets the ID defining the current pending, or just completed, checkpoint. + * + * @return The ID of the pending of completed checkpoint. + */ public long getCurrentCheckpointId() { return this.currentCheckpointId; } @@ -275,6 +295,6 @@ public long getCurrentCheckpointId() { @Override public String toString() { - return "Non-Processed: " + nonProcessed + " | Blocked: " + blockedNonProcessed; + return String.format("last checkpoint: %d, current barriers: %d", currentCheckpointId, numReceivedBarriers); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java index 6b24556e6eae4..a0b924fafe77c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java @@ -28,26 +28,39 @@ /** * The BarrierTracker keeps track of what checkpoint barriers have been received from - * which input channels. + * which input channels. Once it has observed all checkpoint barriers for a checkpoint ID, + * it notifies its listener of a completed checkpoint. * *

Unlike the {@link BarrierBuffer}, the BarrierTracker does not block the input * channels that have sent barriers, so it cannot be used to gain "exactly-once" processing * guarantees. It can, however, be used to gain "at least once" processing guarantees.

+ * + *

NOTE: This implementation strictly assumes that newer checkpoints have higher checkpoint IDs.

*/ public class BarrierTracker implements CheckpointBarrierHandler { + /** The tracker tracks a maximum number of checkpoints, for which some, but not all + * barriers have yet arrived. */ private static final int MAX_CHECKPOINTS_TO_TRACK = 50; + /** The input gate, to draw the buffers and events from */ private final InputGate inputGate; + /** The number of channels. Once that many barriers have been received for a checkpoint, + * the checkpoint is considered complete. */ private final int totalNumberOfInputChannels; - + + /** All checkpoints for which some (but not all) barriers have been received, + * and that are not yet known to be subsumed by newer checkpoints */ private final ArrayDeque pendingCheckpoints; + /** The listener to be notified on complete checkpoints */ private EventListener checkpointHandler; + /** The highest checkpoint ID encountered so far */ private long latestPendingCheckpointID = -1; + public BarrierTracker(InputGate inputGate) { this.inputGate = inputGate; this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); @@ -148,8 +161,6 @@ private void processBarrier(CheckpointBarrier receivedBarrier) { } } - // ------------------------------------------------------------------------ - // // ------------------------------------------------------------------------ /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java index fda612e7247d0..5f9a162b547d6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java @@ -20,80 +20,389 @@ import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.channels.FileChannel; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.event.task.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.util.StringUtils; +/** + * The buffer spiller takes the buffers and events from a data stream and adds them to a spill file. + * After a number of elements have been spilled, the spiller can "roll over": It presents the spilled + * elements as a readable sequence, and opens a new spill file. + * + *

This implementation buffers data effectively in the OS cache, which gracefully extends to the + * disk. Most data is written and re-read milliseconds later. The file is deleted after the read. + * Consequently, in most cases, the data will never actually hit the physical disks.

+ * + *

IMPORTANT: The SpilledBufferOrEventSequences created by this spiller all reuse the same + * reading memory (to reduce overhead) and can consequently not be read concurrently.

+ */ public class BufferSpiller { - - /** The random number generator for temp file names */ - private static final Random RND = new Random(); /** The counter that selects the next directory to spill into */ private static final AtomicInteger DIRECTORY_INDEX = new AtomicInteger(0); + /** The size of the buffer with which data is read back in */ + private static final int READ_BUFFER_SIZE = 1024 * 1024; /** The directories to spill to */ private final File tempDir; - - private File spillFile; - private FileChannel spillingChannel; + /** The name prefix for spill files */ + private final String spillFilePrefix; + + /** The buffer used for bulk reading data (used in the SpilledBufferOrEventSequence) */ + private final ByteBuffer readBuffer; + /** The buffer that encodes the spilled header */ + private final ByteBuffer headBuffer; + /** The reusable array that holds header and contents buffers */ + private final ByteBuffer[] sources; + + /** The file that we currently spill to */ + private File currentSpillFile; + + /** The channel of the file we currently spill to */ + private FileChannel currentChannel; - public BufferSpiller(IOManager ioManager) throws IOException { + /** The page size, to let this reader instantiate properly sized memory segments */ + private final int pageSize; + + /** A counter, to created numbered spill files */ + private int fileCounter; + + /** A flag to check whether the spiller has written since the last roll over */ + private boolean hasWritten; + + /** + * Creates a new buffer spiller, spilling to one of the I/O manager's temp directories. + * + * @param ioManager The I/O manager for access to teh temp directories. + * @param pageSize The page size used to re-create spilled buffers. + * @throws IOException Thrown if the temp files for spilling cannot be initialized. + */ + public BufferSpiller(IOManager ioManager, int pageSize) throws IOException { + this.pageSize = pageSize; + + this.readBuffer = ByteBuffer.allocateDirect(READ_BUFFER_SIZE); + this.readBuffer.order(ByteOrder.LITTLE_ENDIAN); + + this.headBuffer = ByteBuffer.allocateDirect(16); + this.headBuffer.order(ByteOrder.LITTLE_ENDIAN); + + this.sources = new ByteBuffer[] { this.headBuffer, null }; + File[] tempDirs = ioManager.getSpillingDirectories(); this.tempDir = tempDirs[DIRECTORY_INDEX.getAndIncrement() % tempDirs.length]; + + byte[] rndBytes = new byte[32]; + new Random().nextBytes(rndBytes); + this.spillFilePrefix = StringUtils.byteToHexString(rndBytes) + '.'; + + // prepare for first contents createSpillingChannel(); } /** - * Dumps the contents of the buffer to disk and recycles the buffer. + * Adds a buffer or event to the sequence of spilled buffers and events. + * + * @param boe The buffer or event to add and spill. + * @throws IOException Thrown, if the buffer of event could not be spilled. */ - public void spill(Buffer buffer) throws IOException { + public void add(BufferOrEvent boe) throws IOException { + hasWritten = true; try { - spillingChannel.write(buffer.getNioBuffer()); - buffer.recycle(); + ByteBuffer contents; + if (boe.isBuffer()) { + Buffer buf = boe.getBuffer(); + contents = buf.getMemorySegment().wrap(0, buf.getSize()); + } + else { + contents = EventSerializer.toSerializedEvent(boe.getEvent()); + } + + headBuffer.clear(); + headBuffer.putInt(boe.getChannelIndex()); + headBuffer.putInt(contents.remaining()); + headBuffer.put((byte) (boe.isBuffer() ? 0 : 1)); + headBuffer.flip(); + + sources[1] = contents; + currentChannel.write(sources); } - catch (IOException e) { - close(); - throw e; + finally { + if (boe.isBuffer()) { + boe.getBuffer().recycle(); + } } } - @SuppressWarnings("resource") - private void createSpillingChannel() throws IOException { - this.spillFile = new File(tempDir, randomString(RND) + ".buffer"); - this.spillingChannel = new RandomAccessFile(spillFile, "rw").getChannel(); + /** + * Starts a new sequence of spilled buffers and event and returns the current sequence of spilled buffers + * for reading. This method returns {@code null}, if nothing was added since the creation of the spiller, or the + * last call to this method. + * + *

NOTE: The SpilledBufferOrEventSequences created by this method all reuse the same + * reading memory (to reduce overhead) and can consequently not be read concurrently with each other. + * To create a sequence that can be read concurrently with the previous SpilledBufferOrEventSequence, use the + * {@link #rollOverWithNewBuffer()} method.

+ * + * @return The readable sequence of spilled buffers and events, or 'null', if nothing was added. + * @throws IOException Thrown, if the readable sequence could not be created, or no new spill + * file could be created. + */ + public SpilledBufferOrEventSequence rollOver() throws IOException { + return rollOverInternal(false); } + /** + * Starts a new sequence of spilled buffers and event and returns the current sequence of spilled buffers + * for reading. This method returns {@code null}, if nothing was added since the creation of the spiller, or the + * last call to this method. + * + *

The SpilledBufferOrEventSequence returned by this method is safe for concurrent consumption with + * any previously returned sequence.

+ * + * @return The readable sequence of spilled buffers and events, or 'null', if nothing was added. + * @throws IOException Thrown, if the readable sequence could not be created, or no new spill + * file could be created. + */ + public SpilledBufferOrEventSequence rollOverWithNewBuffer() throws IOException { + return rollOverInternal(true); + } + + private SpilledBufferOrEventSequence rollOverInternal(boolean newBuffer) throws IOException { + if (!hasWritten) { + return null; + } + + ByteBuffer buf; + if (newBuffer) { + buf = ByteBuffer.allocateDirect(READ_BUFFER_SIZE); + buf.order(ByteOrder.LITTLE_ENDIAN); + } else { + buf = readBuffer; + } + + // create a reader for the spilled data + currentChannel.position(0L); + SpilledBufferOrEventSequence seq = + new SpilledBufferOrEventSequence(currentSpillFile, currentChannel, buf, pageSize); + + // create ourselves a new spill file + createSpillingChannel(); + + hasWritten = false; + return seq; + } - + /** + * Cleans up the current spilling channel and file. + * + * Does not clean up the SpilledBufferOrEventSequences generated by calls to + * {@link #rollOver()}. + * + * @throws IOException Thrown if channel closing or file deletion fail. + */ public void close() throws IOException { - if (spillingChannel != null && spillingChannel.isOpen()) { - spillingChannel.close(); + currentChannel.close(); + if (!currentSpillFile.delete()) { + throw new IOException("Cannot delete spill file"); } } - public void resetSpillFile() throws IOException { - close(); - createSpillingChannel(); - } + // ------------------------------------------------------------------------ + // For testing + // ------------------------------------------------------------------------ - public File getSpillFile() { - return spillFile; + File getCurrentSpillFile() { + return currentSpillFile; + } + + FileChannel getCurrentChannel() { + return currentChannel; } // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + @SuppressWarnings("resource") + private void createSpillingChannel() throws IOException { + currentSpillFile = new File(tempDir, spillFilePrefix + (fileCounter++) +".buffer"); + currentChannel = new RandomAccessFile(currentSpillFile, "rw").getChannel(); + } + + // ------------------------------------------------------------------------ + + /** + * This class represents a sequence of spilled buffers and events, created by the + * {@link BufferSpiller}. The sequence of buffers and events can be read back using the + * method {@link #getNext()}. + */ + public static class SpilledBufferOrEventSequence { + + /** Header is "channel index" (4 bytes) + length (4 bytes) + buffer/event (1 byte) */ + private static final int HEADER_LENGTH = 9; + + /** The file containing the data */ + private final File file; + + /** The file channel to draw the data from */ + private final FileChannel fileChannel; + + /** The byte buffer for bulk reading */ + private final ByteBuffer buffer; - private static String randomString(Random random) { - final byte[] bytes = new byte[20]; - random.nextBytes(bytes); - return StringUtils.byteToHexString(bytes); + /** The page size to instantiate properly sized memory segments */ + private final int pageSize; + + /** Flag to track whether the sequence has been opened already */ + private boolean opened = false; + + /** + * Create a reader that reads a sequence of spilled buffers and events. + * + * @param file The file with the data. + * @param fileChannel The file channel to read the data from. + * @param buffer The buffer used for bulk reading. + * @param pageSize The page size to use for the created memory segments. + */ + SpilledBufferOrEventSequence(File file, FileChannel fileChannel, ByteBuffer buffer, int pageSize) { + this.file = file; + this.fileChannel = fileChannel; + this.buffer = buffer; + this.pageSize = pageSize; + } + + /** + * Initializes the sequence for reading. + * This method needs to be called before the first call to {@link #getNext()}. Otherwise + * the results of {@link #getNext()} are not predictable. + */ + public void open() { + if (!opened) { + opened = true; + buffer.position(0); + buffer.limit(0); + } + } + + /** + * Gets the next BufferOrEvent from the spilled sequence, or {@code null}, if the + * sequence is exhausted. + * + * @return The next BufferOrEvent from the spilled sequence, or {@code null} (end of sequence). + * @throws IOException Thrown, if the reads failed, of if the byte stream is corrupt. + */ + public BufferOrEvent getNext() throws IOException { + if (buffer.remaining() < HEADER_LENGTH) { + buffer.compact(); + + while (buffer.position() < HEADER_LENGTH) { + if (fileChannel.read(buffer) == -1) { + if (buffer.position() == 0) { + // no trailing data + return null; + } else { + throw new IOException("Found trailing incomplete buffer or event"); + } + } + } + + buffer.flip(); + } + + final int channel = buffer.getInt(); + final int length = buffer.getInt(); + final boolean isBuffer = buffer.get() == 0; + + + if (isBuffer) { + // deserialize buffer + if (length > pageSize) { + throw new IOException(String.format( + "Spilled buffer (%d bytes) is larger than page size of (%d bytes)", length, pageSize)); + } + + MemorySegment seg = new MemorySegment(new byte[pageSize]); + + int segPos = 0; + int bytesRemaining = length; + + while (true) { + int toCopy = Math.min(buffer.remaining(), bytesRemaining); + if (toCopy > 0) { + seg.put(segPos, buffer, toCopy); + segPos += toCopy; + bytesRemaining -= toCopy; + } + + if (bytesRemaining == 0) { + break; + } + else { + buffer.clear(); + if (fileChannel.read(buffer) == -1) { + throw new IOException("Found trailing incomplete buffer"); + } + buffer.flip(); + } + } + + + Buffer buf = new Buffer(seg, FreeingBufferRecycler.INSTANCE); + buf.setSize(length); + + return new BufferOrEvent(buf, channel); + } + else { + // deserialize event + if (length > buffer.capacity() - HEADER_LENGTH) { + throw new IOException("Event is too large"); + } + + if (buffer.remaining() < length) { + buffer.compact(); + + while (buffer.position() < length) { + if (fileChannel.read(buffer) == -1) { + throw new IOException("Found trailing incomplete event"); + } + } + + buffer.flip(); + } + + int oldLimit = buffer.limit(); + buffer.limit(buffer.position() + length); + AbstractEvent evt = EventSerializer.fromSerializedEvent(buffer, getClass().getClassLoader()); + buffer.limit(oldLimit); + + return new BufferOrEvent(evt, channel); + } + } + + /** + * Cleans up all file resources held by this spilled sequence. + * + * @throws IOException Thrown, if file channel closing or file deletion fail. + */ + public void cleanup() throws IOException { + fileChannel.close(); + if (!file.delete()) { + throw new IOException("Cannot remove temp file for stream alignment writer"); + } + } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java similarity index 73% rename from flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java index 3f815ef937ef4..27e37a5d215e1 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/DummyBufferRecycler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java @@ -22,13 +22,16 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler; /** - * A BufferRecycler that does nothing. + * A simple buffer recycler that only frees the memory segments. */ -public class DummyBufferRecycler implements BufferRecycler { +public class FreeingBufferRecycler implements BufferRecycler { - public static final BufferRecycler INSTANCE = new DummyBufferRecycler(); + public static final BufferRecycler INSTANCE = new FreeingBufferRecycler(); + // ------------------------------------------------------------------------ @Override - public void recycle(MemorySegment memorySegment) {} + public void recycle(MemorySegment memorySegment) { + memorySegment.free(); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillReader.java deleted file mode 100644 index 356b491b93cd6..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillReader.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.streaming.runtime.io; - -import java.io.File; -import java.io.IOException; -import java.io.RandomAccessFile; -import java.nio.channels.FileChannel; - -import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferRecycler; - -public class SpillReader { - - private FileChannel spillingChannel; - private File spillFile; - - /** - * Reads the next buffer from the spilled file. - */ - public Buffer readNextBuffer(int bufferSize) throws IOException { - try { - Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]), - new BufferRecycler() { - - @Override - public void recycle(MemorySegment memorySegment) { - memorySegment.free(); - } - }); - - spillingChannel.read(buffer.getMemorySegment().wrap(0, bufferSize)); - - return buffer; - } catch (Exception e) { - close(); - throw new IOException(e); - } - } - - @SuppressWarnings("resource") - public void setSpillFile(File nextSpillFile) throws IOException { - // We can close and delete the file now - close(); - if (spillFile != null) { - spillFile.delete(); - } - this.spillFile = nextSpillFile; - this.spillingChannel = new RandomAccessFile(spillFile, "rw").getChannel(); - } - - public File getSpillFile() { - return spillFile; - } - - public void close() throws IOException { - if (this.spillingChannel != null && this.spillingChannel.isOpen()) { - this.spillingChannel.close(); - } - } - -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEvent.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEvent.java deleted file mode 100644 index 368e3734fd645..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEvent.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.streaming.runtime.io; - -import java.io.IOException; - -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; - -public class SpillingBufferOrEvent { - - private BufferOrEvent boe; - private boolean isSpilled = false; - - private SpillReader spillReader; - - private int channelIndex; - private int bufferSize; - - public SpillingBufferOrEvent(BufferOrEvent boe, BufferSpiller spiller, SpillReader reader) - throws IOException { - - this.boe = boe; - this.channelIndex = boe.getChannelIndex(); - this.spillReader = reader; - - if (boe.isBuffer()) { - this.bufferSize = boe.getBuffer().getSize(); - spiller.spill(boe.getBuffer()); - this.boe = null; - this.isSpilled = true; - } - } - - /** - * If the buffer wasn't spilled simply returns the instance from the field, - * otherwise reads it from the spill reader - */ - public BufferOrEvent getBufferOrEvent() throws IOException { - if (isSpilled) { - boe = new BufferOrEvent(spillReader.readNextBuffer(bufferSize), channelIndex); - this.isSpilled = false; - return boe; - } else { - return boe; - } - } - - public boolean isSpilled() { - return isSpilled; - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index b59ad191ec5e8..4007da85fe2da 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.util.concurrent.ConcurrentLinkedQueue; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -75,6 +76,7 @@ public StreamTestSingleInputGate( inputQueues = new ConcurrentLinkedQueue[numInputChannels]; setupInputChannels(); + doReturn(bufferSize).when(inputGate).getPageSize(); } @SuppressWarnings("unchecked") diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java index c2df4d83e2bc4..7350516204de7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java @@ -42,14 +42,16 @@ */ public class BarrierBufferMassiveRandomTest { + private static final int PAGE_SIZE = 1024; + @Test public void testWithTwoChannelsAndRandomBarriers() { IOManager ioMan = null; try { ioMan = new IOManagerAsync(); - BufferPool pool1 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); - BufferPool pool2 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); + BufferPool pool1 = new NetworkBufferPool(100, PAGE_SIZE).createBufferPool(100, true); + BufferPool pool2 = new NetworkBufferPool(100, PAGE_SIZE).createBufferPool(100, true); RandomGeneratingInputGate myIG = new RandomGeneratingInputGate( new BufferPool[] { pool1, pool2 }, @@ -163,5 +165,10 @@ public void sendTaskEvent(TaskEvent event) {} @Override public void registerListener(EventListener listener) {} + + @Override + public int getPageSize() { + return PAGE_SIZE; + } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index ad61c6fb49b43..b8b3a8c3ec378 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -49,6 +49,8 @@ */ public class BarrierBufferTest { + private static final int PAGE_SIZE = 512; + private static int SIZE_COUNTER = 0; private static IOManager IO_MANAGER; @@ -89,6 +91,8 @@ public void testSingleChannelNoBarriers() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); } catch (Exception e) { e.printStackTrace(); @@ -118,6 +122,8 @@ public void testMultiChannelNoBarriers() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); } catch (Exception e) { e.printStackTrace(); @@ -157,6 +163,8 @@ public void testSingleChannelWithBarriers() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); } catch (Exception e) { e.printStackTrace(); @@ -272,6 +280,8 @@ public void testMultiChannelWithBarriers() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); } catch (Exception e) { e.printStackTrace(); @@ -327,6 +337,8 @@ public void testMultiChannelTrailingBlockedData() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + buffer.cleanup(); + } catch (Exception e) { e.printStackTrace(); @@ -441,6 +453,8 @@ public void testMultiChannelWithQueuedFutureBarriers() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); } catch (Exception e) { e.printStackTrace(); @@ -516,6 +530,95 @@ public void testMultiChannelSkippingCheckpoints() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + /** + * Validates that the buffer skips over the current checkpoint if it + * receives a barrier from a later checkpoint on a non-blocked input. + */ + @Test + public void testMultiChannelJumpingOverCheckpoint() { + try { + BufferOrEvent[] sequence = { + // checkpoint 1 - with blocked data + createBuffer(0), createBuffer(2), createBuffer(0), + createBarrier(1, 1), createBarrier(1, 2), + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(1, 0), + createBuffer(1), createBuffer(0), + + // checkpoint 2 will not complete: pre-mature barrier from checkpoint 3 + createBarrier(2, 1), + createBuffer(1), createBuffer(2), + createBarrier(2, 0), + createBuffer(2), createBuffer(0), + createBarrier(3, 1), + createBuffer(1), createBuffer(2), + createBarrier(3, 0), + createBuffer(2), createBuffer(0), + createBarrier(4, 2), + + createBuffer(2), + createBuffer(1), createEndOfPartition(1), + createBuffer(2), createEndOfPartition(2), + createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + // checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + check(sequence[7], buffer.getNextNonBlocked()); + assertEquals(1L, buffer.getCurrentCheckpointId()); + + check(sequence[5], buffer.getNextNonBlocked()); + check(sequence[6], buffer.getNextNonBlocked()); + check(sequence[9], buffer.getNextNonBlocked()); + check(sequence[10], buffer.getNextNonBlocked()); + + // alignment of checkpoint 2 + check(sequence[13], buffer.getNextNonBlocked()); + assertEquals(2L, buffer.getCurrentCheckpointId()); + check(sequence[15], buffer.getNextNonBlocked()); + check(sequence[19], buffer.getNextNonBlocked()); + check(sequence[21], buffer.getNextNonBlocked()); + + // checkpoint 2 aborted, checkpoint 4 started. replay buffered + check(sequence[12], buffer.getNextNonBlocked()); + assertEquals(4L, buffer.getCurrentCheckpointId()); + check(sequence[16], buffer.getNextNonBlocked()); + check(sequence[18], buffer.getNextNonBlocked()); + check(sequence[22], buffer.getNextNonBlocked()); + + // align remainder + check(sequence[25], buffer.getNextNonBlocked()); + check(sequence[26], buffer.getNextNonBlocked()); + check(sequence[29], buffer.getNextNonBlocked()); + check(sequence[30], buffer.getNextNonBlocked()); + + // end of input, emit remainder + check(sequence[24], buffer.getNextNonBlocked()); + check(sequence[27], buffer.getNextNonBlocked()); + check(sequence[28], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); } catch (Exception e) { e.printStackTrace(); @@ -526,15 +629,8 @@ public void testMultiChannelSkippingCheckpoints() { /** * Validates that the buffer skips over a later checkpoint if it * receives a barrier from an even later checkpoint on a blocked input. - * - * NOTE: This test currently fails, because the barrier buffer does not support - * to unblock inputs before all previously unblocked data is consumed. - * - * Since this test checks only that the buffer behaves "failsafe" in cases of - * corrupt checkpoint barrier propagation (a situation that does not occur - * under the current model), we ignore it for the moment. */ -// @Test + @Test public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { try { BufferOrEvent[] sequence = { @@ -551,18 +647,23 @@ public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { createBarrier(2, 0), createBuffer(1), createBuffer(0), + createBarrier(3, 0), // queued barrier on blocked input + createBuffer(0), + createBarrier(4, 1), // pre-mature barrier on blocked input - createBarrier(3, 0), // queued barrier, ignored on replay + createBuffer(1), + createBuffer(0), + createBuffer(2), // complete checkpoint 2 - createBarrier(2, 0), + createBarrier(2, 2), createBuffer(0), createBarrier(3, 2), // should be ignored createBuffer(2), createBarrier(4, 0), createBuffer(0), createBuffer(1), createBuffer(2), - createBarrier(4, 1), + createBarrier(4, 2), createBuffer(1), createEndOfPartition(1), createBuffer(2), createEndOfPartition(2), @@ -585,6 +686,7 @@ public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { // alignment of checkpoint 2 check(sequence[13], buffer.getNextNonBlocked()); + check(sequence[22], buffer.getNextNonBlocked()); assertEquals(2L, buffer.getCurrentCheckpointId()); // checkpoint 2 completed @@ -593,24 +695,79 @@ public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { check(sequence[16], buffer.getNextNonBlocked()); // checkpoint 3 skipped, alignment for 4 started - check(sequence[20], buffer.getNextNonBlocked()); + check(sequence[18], buffer.getNextNonBlocked()); assertEquals(4L, buffer.getCurrentCheckpointId()); - check(sequence[22], buffer.getNextNonBlocked()); + check(sequence[21], buffer.getNextNonBlocked()); + check(sequence[24], buffer.getNextNonBlocked()); check(sequence[26], buffer.getNextNonBlocked()); - + check(sequence[30], buffer.getNextNonBlocked()); + // checkpoint 4 completed - check(sequence[24], buffer.getNextNonBlocked()); - check(sequence[25], buffer.getNextNonBlocked()); - + check(sequence[20], buffer.getNextNonBlocked()); check(sequence[28], buffer.getNextNonBlocked()); check(sequence[29], buffer.getNextNonBlocked()); - check(sequence[30], buffer.getNextNonBlocked()); - check(sequence[31], buffer.getNextNonBlocked()); + check(sequence[32], buffer.getNextNonBlocked()); check(sequence[33], buffer.getNextNonBlocked()); + check(sequence[34], buffer.getNextNonBlocked()); + check(sequence[35], buffer.getNextNonBlocked()); + check(sequence[36], buffer.getNextNonBlocked()); + check(sequence[37], buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testEarlyCleanup() { + try { + BufferOrEvent[] sequence = { + createBuffer(0), createBuffer(1), createBuffer(2), + createBarrier(1, 1), createBarrier(1, 2), createBarrier(1, 0), + + createBuffer(2), createBuffer(1), createBuffer(0), + createBarrier(2, 1), + createBuffer(1), createBuffer(1), createEndOfPartition(1), createBuffer(0), createBuffer(2), + createBarrier(2, 2), + createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0) + }; + + MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); + buffer.registerCheckpointEventHandler(handler); + handler.setNextExpectedCheckpointId(1L); + + // pre-checkpoint 1 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + assertEquals(1L, handler.getNextExpectedCheckpointId()); + + // pre-checkpoint 2 + check(sequence[6], buffer.getNextNonBlocked()); + assertEquals(2L, handler.getNextExpectedCheckpointId()); + check(sequence[7], buffer.getNextNonBlocked()); + check(sequence[8], buffer.getNextNonBlocked()); + + // checkpoint 2 alignment + check(sequence[13], buffer.getNextNonBlocked()); + check(sequence[14], buffer.getNextNonBlocked()); + check(sequence[18], buffer.getNextNonBlocked()); + check(sequence[19], buffer.getNextNonBlocked()); + + // end of stream: remaining buffered contents + buffer.getNextNonBlocked(); + buffer.cleanup(); + } catch (Exception e) { e.printStackTrace(); @@ -629,8 +786,9 @@ private static BufferOrEvent createBarrier(long id, int channel) { private static BufferOrEvent createBuffer(int channel) { // since we have no access to the contents, we need to use the size as an // identifier to validate correctness here - return new BufferOrEvent( - new Buffer(new MemorySegment(new byte[SIZE_COUNTER++]), DummyBufferRecycler.INSTANCE), channel); + Buffer buf = new Buffer(new MemorySegment(new byte[PAGE_SIZE]), FreeingBufferRecycler.INSTANCE); + buf.setSize(SIZE_COUNTER++); + return new BufferOrEvent(buf, channel); } private static BufferOrEvent createEndOfPartition(int channel) { @@ -689,6 +847,11 @@ public void sendTaskEvent(TaskEvent event) {} @Override public void registerListener(EventListener listener) {} + + @Override + public int getPageSize() { + return PAGE_SIZE; + } } private static class ValidatingCheckpointHandler implements EventListener { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java index b2c570e409070..532078c5c8b3d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java @@ -39,7 +39,7 @@ * Tests for the behavior of the barrier tracker. */ public class BarrierTrackerTest { - + @Test public void testSingleChannelNoBarriers() { try { @@ -341,7 +341,7 @@ private static BufferOrEvent createBarrier(long id, int channel) { private static BufferOrEvent createBuffer(int channel) { return new BufferOrEvent( - new Buffer(new MemorySegment(new byte[] { 1 }), DummyBufferRecycler.INSTANCE), channel); + new Buffer(new MemorySegment(new byte[] { 1, 2 }), FreeingBufferRecycler.INSTANCE), channel); } // ------------------------------------------------------------------------ @@ -381,6 +381,11 @@ public void sendTaskEvent(TaskEvent event) {} @Override public void registerListener(EventListener listener) {} + + @Override + public int getPageSize() { + return 2; + } } private static class CheckpointSequenceValidator implements EventListener { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java new file mode 100644 index 0000000000000..ae384e16c65c8 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java @@ -0,0 +1,390 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Random; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.*; + +public class BufferSpillerTest { + + private static final int PAGE_SIZE = 4096; + + private static IOManager IO_MANAGER; + + private BufferSpiller spiller; + + + // ------------------------------------------------------------------------ + // Setup / Cleanup + // ------------------------------------------------------------------------ + + @BeforeClass + public static void setupIOManager() { + IO_MANAGER = new IOManagerAsync(); + } + + @AfterClass + public static void shutdownIOManager() { + IO_MANAGER.shutdown(); + } + + @Before + public void createSpiller() { + try { + spiller = new BufferSpiller(IO_MANAGER, PAGE_SIZE); + } + catch (Exception e) { + e.printStackTrace(); + fail("Cannot create BufferSpiller: " + e.getMessage()); + } + } + + @After + public void cleanupSpiller() { + if (spiller != null) { + try { + spiller.close(); + } + catch (Exception e) { + e.printStackTrace(); + fail("Cannot properly close the BufferSpiller: " + e.getMessage()); + } + + assertFalse(spiller.getCurrentChannel().isOpen()); + assertFalse(spiller.getCurrentSpillFile().exists()); + } + } + + // ------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------ + + @Test + public void testRollOverEmptySequences() { + try { + assertNull(spiller.rollOver()); + assertNull(spiller.rollOver()); + assertNull(spiller.rollOver()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testSpillAndRollOverSimple() { + try { + final Random rnd = new Random(); + final Random bufferRnd = new Random(); + + final int maxNumEventsAndBuffers = 3000; + final int maxNumChannels = 1656; + + // do multiple spilling / rolling over rounds + for (int round = 0; round < 5; round++) { + + final long bufferSeed = rnd.nextLong(); + bufferRnd.setSeed(bufferSeed); + + final int numEventsAndBuffers = rnd.nextInt(maxNumEventsAndBuffers) + 1; + final int numChannels = rnd.nextInt(maxNumChannels) + 1; + + final ArrayList events = new ArrayList(128); + + // generate sequence + for (int i = 0; i < numEventsAndBuffers; i++) { + boolean isEvent = rnd.nextDouble() < 0.05d; + if (isEvent) { + BufferOrEvent evt = generateRandomEvent(rnd, numChannels); + events.add(evt); + spiller.add(evt); + } + else { + BufferOrEvent evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); + spiller.add(evt); + } + } + + // reset and create reader + bufferRnd.setSeed(bufferSeed); + + BufferSpiller.SpilledBufferOrEventSequence seq = spiller.rollOver(); + seq.open(); + + // read and validate the sequence + + int numEvent = 0; + for (int i = 0; i < numEventsAndBuffers; i++) { + BufferOrEvent next = seq.getNext(); + assertNotNull(next); + if (next.isEvent()) { + BufferOrEvent expected = events.get(numEvent++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); + } + else { + validateBuffer(next, bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); + } + } + + // no further data + assertNull(seq.getNext()); + + // all events need to be consumed + assertEquals(events.size(), numEvent); + + seq.cleanup(); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testSpillWhileReading() { + try { + final int sequences = 10; + + final Random rnd = new Random(); + final AtomicReference error = new AtomicReference(); + + final SequenceConsumer consumer = new SequenceConsumer(error, sequences); + consumer.start(); + + final int maxNumEventsAndBuffers = 30000; + final int maxNumChannels = 1656; + + // do multiple spilling / rolling over rounds + for (int round = 0; round < 2*sequences; round++) { + + if (round % 2 == 1) { + // make this an empty sequence + assertNull(spiller.rollOver()); + } + else { + // proper spilled sequence + final long bufferSeed = rnd.nextLong(); + final Random bufferRnd = new Random(bufferSeed); + + final int numEventsAndBuffers = rnd.nextInt(maxNumEventsAndBuffers) + 1; + final int numChannels = rnd.nextInt(maxNumChannels) + 1; + + final ArrayList events = new ArrayList(128); + + // generate sequence + for (int i = 0; i < numEventsAndBuffers; i++) { + boolean isEvent = rnd.nextDouble() < 0.05d; + if (isEvent) { + BufferOrEvent evt = generateRandomEvent(rnd, numChannels); + events.add(evt); + spiller.add(evt); + } + else { + BufferOrEvent evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); + spiller.add(evt); + } + } + + // reset and create reader + bufferRnd.setSeed(bufferSeed); + BufferSpiller.SpilledBufferOrEventSequence seq = spiller.rollOver(); + + SequenceToConsume stc = new SequenceToConsume(bufferRnd, events, seq, numEventsAndBuffers, numChannels); + consumer.queue(stc); + } + } + + // wait for the consumer + consumer.join(180000); + assertFalse("sequence consumer did not finish its work in time", consumer.isAlive()); + + // validate there was no error in the consumer + if (error.get() != null) { + Throwable t = error.get(); + if (t instanceof Error) { + throw (Error) t; + } + else { + throw new Exception("Error while consuming the spilled records", t); + } + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // ------------------------------------------------------------------------ + // Utils + // ------------------------------------------------------------------------ + + private static BufferOrEvent generateRandomEvent(Random rnd, int numChannels) { + long magicNumber = rnd.nextLong(); + byte[] data = new byte[rnd.nextInt(1000)]; + rnd.nextBytes(data); + TestEvent evt = new TestEvent(magicNumber, data); + + int channelIndex = rnd.nextInt(numChannels); + + return new BufferOrEvent(evt, channelIndex); + } + + private static BufferOrEvent generateRandomBuffer(int size, int channelIndex) { + MemorySegment seg = new MemorySegment(new byte[PAGE_SIZE]); + for (int i = 0; i < size; i++) { + seg.put(i, (byte) i); + } + + Buffer buf = new Buffer(seg, FreeingBufferRecycler.INSTANCE); + buf.setSize(size); + return new BufferOrEvent(buf, channelIndex); + } + + private static void validateBuffer(BufferOrEvent boe, int expectedSize, int expectedChannelIndex) { + assertEquals("wrong channel index", expectedChannelIndex, boe.getChannelIndex()); + assertTrue("is not buffer", boe.isBuffer()); + + Buffer buf = boe.getBuffer(); + assertEquals("wrong buffer size", expectedSize, buf.getSize()); + + MemorySegment seg = buf.getMemorySegment(); + for (int i = 0; i < expectedSize; i++) { + assertEquals("wrong buffer contents", (byte) i, seg.get(i)); + } + } + + // ------------------------------------------------------------------------ + // Async Consumer + // ------------------------------------------------------------------------ + + private static class SequenceToConsume { + + final BufferSpiller.SpilledBufferOrEventSequence sequence; + final ArrayList events; + final Random bufferRnd; + final int numBuffersAndEvents; + final int numChannels; + + private SequenceToConsume(Random bufferRnd, ArrayList events, + BufferSpiller.SpilledBufferOrEventSequence sequence, + int numBuffersAndEvents, int numChannels) { + this.bufferRnd = bufferRnd; + this.events = events; + this.sequence = sequence; + this.numBuffersAndEvents = numBuffersAndEvents; + this.numChannels = numChannels; + } + } + + private static class SequenceConsumer extends Thread { + + private final AtomicReference error; + private final BlockingQueue sequences; + + private final int numSequencesToConsume; + + private int consumedSequences; + + private SequenceConsumer(AtomicReference error, int numSequencesToConsume) { + super("Sequence Consumer"); + setDaemon(true); + + this.error = error; + this.numSequencesToConsume = numSequencesToConsume; + this.sequences = new LinkedBlockingQueue(); + } + + + @Override + public void run() { + try { + while (consumedSequences < numSequencesToConsume) { + // get next sequence + SequenceToConsume nextSequence = sequences.take(); + + // wait a bit, allow some stuff to queue up + Thread.sleep(50); + + BufferSpiller.SpilledBufferOrEventSequence seq = nextSequence.sequence; + ArrayList events = nextSequence.events; + Random bufferRnd = nextSequence.bufferRnd; + int numBuffersAndEvents = nextSequence.numBuffersAndEvents; + int numChannels = nextSequence.numChannels; + + // consume sequence + seq.open(); + + int numEvent = 0; + for (int i = 0; i < numBuffersAndEvents; i++) { + BufferOrEvent next = seq.getNext(); + assertNotNull(next); + if (next.isEvent()) { + BufferOrEvent expected = events.get(numEvent++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); + } + else { + validateBuffer(next, bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); + } + } + + // no further data + assertNull(seq.getNext()); + + // all events need to be consumed + assertEquals(events.size(), numEvent); + + // remove all temp files + seq.cleanup(); + + consumedSequences++; + } + + } + catch (Throwable t) { + error.set(t); + } + } + + public void queue(SequenceToConsume next) { + sequences.add(next); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java new file mode 100644 index 0000000000000..991b03320ae83 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java @@ -0,0 +1,482 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.streaming.runtime.io.BufferSpiller.SpilledBufferOrEventSequence; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.util.ArrayList; +import java.util.Random; + +import static org.junit.Assert.*; + +/** + * Tests that validate the behavior of the {@link SpilledBufferOrEventSequence} in isolation, + * with respect to detecting corrupt sequences, trailing data, and interleaved buffers and events. + */ +public class SpilledBufferOrEventSequenceTest { + + private final ByteBuffer buffer = ByteBuffer.allocateDirect(128 * 1024).order(ByteOrder.LITTLE_ENDIAN); + private final int pageSize = 32*1024; + + private File tempFile; + private FileChannel fileChannel; + + + @Before + public void initTempChannel() { + try { + tempFile = File.createTempFile("testdata", "tmp"); + fileChannel = new RandomAccessFile(tempFile, "rw").getChannel(); + } + catch (Exception e) { + cleanup(); + } + } + + @After + public void cleanup() { + if (fileChannel != null) { + try { + fileChannel.close(); + } + catch (IOException e) { + // ignore + } + } + if (tempFile != null) { + //noinspection ResultOfMethodCallIgnored + tempFile.delete(); + } + } + + + // ------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------ + + @Test + public void testEmptyChannel() { + try { + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + + assertNull(seq.getNext()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testIncompleteHeaderOnFirstElement() { + try { + ByteBuffer buf = ByteBuffer.allocate(7); + buf.order(ByteOrder.LITTLE_ENDIAN); + + fileChannel.write(buf); + fileChannel.position(0); + + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + + try { + seq.getNext(); + fail("should fail with an exception"); + } + catch (IOException e) { + // expected + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testBufferSequence() { + try { + final Random rnd = new Random(); + final long seed = rnd.nextLong(); + + final int numBuffers = 325; + final int numChannels = 671; + + rnd.setSeed(seed); + + for (int i = 0; i < numBuffers; i++) { + writeBuffer(fileChannel, rnd.nextInt(pageSize) + 1, rnd.nextInt(numChannels)); + } + + fileChannel.position(0L); + rnd.setSeed(seed); + + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + + for (int i = 0; i < numBuffers; i++) { + validateBuffer(seq.getNext(), rnd.nextInt(pageSize) + 1, rnd.nextInt(numChannels)); + } + + // should have no more data + assertNull(seq.getNext()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testBufferSequenceWithIncompleteBuffer() { + try { + writeBuffer(fileChannel, 1672, 7); + + // write an incomplete buffer + ByteBuffer data = ByteBuffer.allocate(615); + data.order(ByteOrder.LITTLE_ENDIAN); + + data.putInt(2); + data.putInt(999); + data.put((byte) 0); + data.position(0); + data.limit(312); + fileChannel.write(data); + fileChannel.position(0L); + + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + + // first one is valid + validateBuffer(seq.getNext(), 1672, 7); + + // next one should fail + try { + seq.getNext(); + fail("should fail with an exception"); + } + catch (IOException e) { + // expected + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testEventSequence() { + try { + final Random rnd = new Random(); + final int numEvents = 3000; + final int numChannels = 1656; + + final ArrayList events = new ArrayList(numEvents); + + for (int i = 0; i < numEvents; i++) { + events.add(generateAndWriteEvent(fileChannel, rnd, numChannels)); + } + + fileChannel.position(0L); + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + + int i = 0; + BufferOrEvent boe; + while ((boe = seq.getNext()) != null) { + BufferOrEvent expected = events.get(i); + assertTrue(boe.isEvent()); + assertEquals(expected.getEvent(), boe.getEvent()); + assertEquals(expected.getChannelIndex(), boe.getChannelIndex()); + i++; + } + + assertEquals(numEvents, i); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testMixedSequence() { + try { + final Random rnd = new Random(); + final Random bufferRnd = new Random(); + + final long bufferSeed = rnd.nextLong(); + bufferRnd.setSeed(bufferSeed); + + final int numEventsAndBuffers = 3000; + final int numChannels = 1656; + + final ArrayList events = new ArrayList(128); + + // generate sequence + + for (int i = 0; i < numEventsAndBuffers; i++) { + boolean isEvent = rnd.nextDouble() < 0.05d; + if (isEvent) { + events.add(generateAndWriteEvent(fileChannel, rnd, numChannels)); + } + else { + writeBuffer(fileChannel, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels)); + } + } + + // reset and create reader + + fileChannel.position(0L); + bufferRnd.setSeed(bufferSeed); + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + + // read and validate the sequence + + int numEvent = 0; + for (int i = 0; i < numEventsAndBuffers; i++) { + BufferOrEvent next = seq.getNext(); + if (next.isEvent()) { + BufferOrEvent expected = events.get(numEvent++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); + } + else { + validateBuffer(next, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels)); + } + } + + // no further data + assertNull(seq.getNext()); + + // all events need to be consumed + assertEquals(events.size(), numEvent); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testMultipleSequences() { + File secondFile = null; + FileChannel secondChannel = null; + + try { + // create the second file channel + secondFile = File.createTempFile("testdata", "tmp"); + secondChannel = new RandomAccessFile(secondFile, "rw").getChannel(); + + final Random rnd = new Random(); + final Random bufferRnd = new Random(); + + final long bufferSeed = rnd.nextLong(); + bufferRnd.setSeed(bufferSeed); + + final int numEventsAndBuffers1 = 272; + final int numEventsAndBuffers2 = 151; + + final int numChannels = 1656; + + final ArrayList events1 = new ArrayList(128); + final ArrayList events2 = new ArrayList(128); + + // generate sequence 1 + + for (int i = 0; i < numEventsAndBuffers1; i++) { + boolean isEvent = rnd.nextDouble() < 0.05d; + if (isEvent) { + events1.add(generateAndWriteEvent(fileChannel, rnd, numChannels)); + } + else { + writeBuffer(fileChannel, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels)); + } + } + + // generate sequence 2 + + for (int i = 0; i < numEventsAndBuffers2; i++) { + boolean isEvent = rnd.nextDouble() < 0.05d; + if (isEvent) { + events2.add(generateAndWriteEvent(secondChannel, rnd, numChannels)); + } + else { + writeBuffer(secondChannel, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels)); + } + } + + // reset and create reader + + fileChannel.position(0L); + secondChannel.position(0L); + + bufferRnd.setSeed(bufferSeed); + + SpilledBufferOrEventSequence seq1 = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + SpilledBufferOrEventSequence seq2 = new SpilledBufferOrEventSequence(secondFile, secondChannel, buffer, pageSize); + + // read and validate the sequence 1 + seq1.open(); + + int numEvent = 0; + for (int i = 0; i < numEventsAndBuffers1; i++) { + BufferOrEvent next = seq1.getNext(); + if (next.isEvent()) { + BufferOrEvent expected = events1.get(numEvent++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); + } + else { + validateBuffer(next, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels)); + } + } + assertNull(seq1.getNext()); + assertEquals(events1.size(), numEvent); + + // read and validate the sequence 2 + seq2.open(); + + numEvent = 0; + for (int i = 0; i < numEventsAndBuffers2; i++) { + BufferOrEvent next = seq2.getNext(); + if (next.isEvent()) { + BufferOrEvent expected = events2.get(numEvent++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); + } + else { + validateBuffer(next, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels)); + } + } + assertNull(seq2.getNext()); + assertEquals(events2.size(), numEvent); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + finally { + if (secondChannel != null) { + try { + secondChannel.close(); + } + catch (IOException e) { + // ignore here + } + } + if (secondFile != null) { + //noinspection ResultOfMethodCallIgnored + secondFile.delete(); + } + } + } + + @Test + public void testCleanup() { + try { + ByteBuffer data = ByteBuffer.allocate(157); + data.order(ByteOrder.LITTLE_ENDIAN); + + fileChannel.write(data); + fileChannel.position(54); + + SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize); + seq.open(); + seq.cleanup(); + + assertFalse(fileChannel.isOpen()); + assertFalse(tempFile.exists()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // ------------------------------------------------------------------------ + // Utils + // ------------------------------------------------------------------------ + + private static BufferOrEvent generateAndWriteEvent(FileChannel fileChannel, Random rnd, int numChannels) throws IOException { + long magicNumber = rnd.nextLong(); + byte[] data = new byte[rnd.nextInt(1000)]; + rnd.nextBytes(data); + TestEvent evt = new TestEvent(magicNumber, data); + + int channelIndex = rnd.nextInt(numChannels); + + ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(evt); + ByteBuffer header = ByteBuffer.allocate(9); + header.order(ByteOrder.LITTLE_ENDIAN); + + header.putInt(channelIndex); + header.putInt(serializedEvent.remaining()); + header.put((byte) 1); + header.flip(); + + fileChannel.write(header); + fileChannel.write(serializedEvent); + return new BufferOrEvent(evt, channelIndex); + } + + private static void writeBuffer(FileChannel fileChannel, int size, int channelIndex) throws IOException { + ByteBuffer data = ByteBuffer.allocate(size + 9); + data.order(ByteOrder.LITTLE_ENDIAN); + + data.putInt(channelIndex); + data.putInt(size); + data.put((byte) 0); + for (int i = 0; i < size; i++) { + data.put((byte) i); + } + data.flip(); + fileChannel.write(data); + } + + private static void validateBuffer(BufferOrEvent boe, int expectedSize, int expectedChannelIndex) { + assertEquals("wrong channel index", expectedChannelIndex, boe.getChannelIndex()); + assertTrue("is not buffer", boe.isBuffer()); + + Buffer buf = boe.getBuffer(); + assertEquals("wrong buffer size", expectedSize, buf.getSize()); + + MemorySegment seg = buf.getMemorySegment(); + for (int i = 0; i < expectedSize; i++) { + assertEquals("wrong buffer contents", (byte) i, seg.get(i)); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java deleted file mode 100644 index b6cd656ccc116..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBufferOrEventTest.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.streaming.runtime.io; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; - -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; - -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -public class SpillingBufferOrEventTest { - - private static IOManager IO_MANAGER; - - @BeforeClass - public static void createIOManager() { - IO_MANAGER = new IOManagerAsync(); - } - - @AfterClass - public static void shutdownIOManager() { - IO_MANAGER.shutdown(); - } - - // ------------------------------------------------------------------------ - - @Test - public void testSpilling() throws IOException, InterruptedException { - BufferSpiller bsp = new BufferSpiller(IO_MANAGER); - SpillReader spr = new SpillReader(); - - BufferPool pool1 = new NetworkBufferPool(10, 256).createBufferPool(2, true); - BufferPool pool2 = new NetworkBufferPool(10, 256).createBufferPool(2, true); - - Buffer b1 = pool1.requestBuffer(); - b1.getMemorySegment().putInt(0, 10000); - BufferOrEvent boe1 = new BufferOrEvent(b1, 2); - SpillingBufferOrEvent sboe1 = new SpillingBufferOrEvent(boe1, bsp, spr); - - assertTrue(sboe1.isSpilled()); - - Buffer b2 = pool2.requestBuffer(); - b2.getMemorySegment().putInt(0, 10000); - BufferOrEvent boe2 = new BufferOrEvent(b2, 4); - SpillingBufferOrEvent sboe2 = new SpillingBufferOrEvent(boe2, bsp, spr); - - assertTrue(sboe2.isSpilled()); - - Buffer b3 = pool1.requestBuffer(); - b3.getMemorySegment().putInt(0, 50000); - BufferOrEvent boe3 = new BufferOrEvent(b3, 0); - SpillingBufferOrEvent sboe3 = new SpillingBufferOrEvent(boe3, bsp, spr); - - assertTrue(sboe3.isSpilled()); - - Buffer b4 = pool2.requestBuffer(); - b4.getMemorySegment().putInt(0, 60000); - BufferOrEvent boe4 = new BufferOrEvent(b4, 0); - SpillingBufferOrEvent sboe4 = new SpillingBufferOrEvent(boe4, bsp, spr); - - assertTrue(sboe4.isSpilled()); - - bsp.close(); - - spr.setSpillFile(bsp.getSpillFile()); - - Buffer b1ret = sboe1.getBufferOrEvent().getBuffer(); - assertEquals(10000, b1ret.getMemorySegment().getInt(0)); - assertEquals(2, sboe1.getBufferOrEvent().getChannelIndex()); - b1ret.recycle(); - - Buffer b2ret = sboe2.getBufferOrEvent().getBuffer(); - assertEquals(10000, b2ret.getMemorySegment().getInt(0)); - assertEquals(4, sboe2.getBufferOrEvent().getChannelIndex()); - b2ret.recycle(); - - Buffer b3ret = sboe3.getBufferOrEvent().getBuffer(); - assertEquals(50000, b3ret.getMemorySegment().getInt(0)); - assertEquals(0, sboe3.getBufferOrEvent().getChannelIndex()); - b3ret.recycle(); - - Buffer b4ret = sboe4.getBufferOrEvent().getBuffer(); - assertEquals(60000, b4ret.getMemorySegment().getInt(0)); - b4ret.recycle(); - - spr.close(); - bsp.getSpillFile().delete(); - - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java index b5bece7e1b2ce..fb3beeae7a1a6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java @@ -26,10 +26,13 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.types.LongValue; + import org.junit.Test; import org.junit.runner.RunWith; + import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; + import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -96,7 +99,7 @@ private static ResultPartitionWriter getMockWriter(int numPartitions) throws Exc when(mockProvider.requestBufferBlocking()).thenAnswer(new Answer() { @Override public Buffer answer(InvocationOnMock invocation) { - return new Buffer(new MemorySegment(new byte[4096]), DummyBufferRecycler.INSTANCE); + return new Buffer(new MemorySegment(new byte[4096]), FreeingBufferRecycler.INSTANCE); } }); @@ -108,7 +111,6 @@ public Buffer answer(InvocationOnMock invocation) { return mockWriter; } - // ------------------------------------------------------------------------ private static class FailingWriter extends StreamRecordWriter { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java new file mode 100644 index 0000000000000..4a7775717e5d7 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java @@ -0,0 +1,88 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.util.StringUtils; + +import java.io.IOException; +import java.util.Arrays; + +/** + * A simple task event, used for validation of buffer or event blocking/buffering. + */ +public class TestEvent extends AbstractEvent { + + private long magicNumber; + + private byte[] payload; + + public TestEvent() {} + + public TestEvent(long magicNumber, byte[] payload) { + this.magicNumber = magicNumber; + this.payload = payload; + } + + + // ------------------------------------------------------------------------ + // Serialization + // ------------------------------------------------------------------------ + + @Override + public void write(DataOutputView out) throws IOException { + out.writeLong(magicNumber); + out.writeInt(payload.length); + out.write(payload); + } + + @Override + public void read(DataInputView in) throws IOException { + this.magicNumber = in.readLong(); + this.payload = new byte[in.readInt()]; + in.read(this.payload); + } + + // ------------------------------------------------------------------------ + // Standard utilities + // ------------------------------------------------------------------------ + + @Override + public int hashCode() { + return Long.valueOf(magicNumber).hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj != null && obj.getClass() == TestEvent.class) { + TestEvent that = (TestEvent) obj; + return this.magicNumber == that.magicNumber && Arrays.equals(this.payload, that.payload); + } + else { + return false; + } + } + + @Override + public String toString() { + return String.format("TestEvent %d (%s)", magicNumber, StringUtils.byteToHexString(payload)); + } +} \ No newline at end of file From aa0105aa309da0dc8570b47880bb0624114e8231 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 30 Jul 2015 18:26:32 +0200 Subject: [PATCH 050/175] [runtime] Cleanup channel events. Add comments and move some classes to test scope. --- .../runtime/event/task/RuntimeEvent.java | 6 ++-- .../flink/runtime/event/task/TaskEvent.java | 7 +++-- .../io/network/api/EndOfPartitionEvent.java | 13 +++++++- .../io/network/api/EndOfSuperstepEvent.java | 31 ++++++++++++++++--- .../flink/runtime/event/task/EventList.java | 0 .../runtime/event/task/IntegerTaskEvent.java | 0 .../runtime/event/task/StringTaskEvent.java | 0 7 files changed, 47 insertions(+), 10 deletions(-) rename flink-runtime/src/{main => test}/java/org/apache/flink/runtime/event/task/EventList.java (100%) rename flink-runtime/src/{main => test}/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java (100%) rename flink-runtime/src/{main => test}/java/org/apache/flink/runtime/event/task/StringTaskEvent.java (100%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java index cd19bc4b2864e..8c44073e4b903 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java @@ -18,5 +18,7 @@ package org.apache.flink.runtime.event.task; -public abstract class RuntimeEvent extends AbstractEvent { -} +/** + * Subclasses of this event are recognized as events exchanged by the core runtime. + */ +public abstract class RuntimeEvent extends AbstractEvent {} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java index 9501b9557bb62..01ecce2733499 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java @@ -18,5 +18,8 @@ package org.apache.flink.runtime.event.task; -public abstract class TaskEvent extends AbstractEvent { -} +/** + * Subclasses of this event are recognized as custom events that are not part of the core + * flink runtime. + */ +public abstract class TaskEvent extends AbstractEvent {} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java index 3ecdb945841a4..c2b6fa4d7dfed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java @@ -22,11 +22,20 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.event.task.RuntimeEvent; - +/** + * This event marks a subpartition as fully consumed. + */ public class EndOfPartitionEvent extends RuntimeEvent { + /** The singleton instance of this event */ public static final EndOfPartitionEvent INSTANCE = new EndOfPartitionEvent(); + // ------------------------------------------------------------------------ + + // not instantiable + private EndOfPartitionEvent() {} + + // ------------------------------------------------------------------------ @Override public void read(DataInputView in) { @@ -38,6 +47,8 @@ public void write(DataOutputView out) { // Nothing to do here } + // ------------------------------------------------------------------------ + @Override public int hashCode() { return 1965146673; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java index 5d0199c7ad618..162afb713ce48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java @@ -22,20 +22,41 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.event.task.RuntimeEvent; -import java.io.IOException; - /** - * Marks the end of a superstep of one particular iteration head + * Marks the end of a superstep of one particular iteration superstep. */ public class EndOfSuperstepEvent extends RuntimeEvent { + /** The singleton instance of this event */ public static final EndOfSuperstepEvent INSTANCE = new EndOfSuperstepEvent(); + // ------------------------------------------------------------------------ + + // not instantiable + private EndOfSuperstepEvent() {} + + // ------------------------------------------------------------------------ + + @Override + public void write(DataOutputView out) {} + + @Override + public void read(DataInputView in) {} + + // ------------------------------------------------------------------------ + + @Override + public int hashCode() { + return 41; + } + @Override - public void write(DataOutputView out) throws IOException { + public boolean equals(Object obj) { + return obj != null && obj.getClass() == EndOfSuperstepEvent.class; } @Override - public void read(DataInputView in) throws IOException { + public String toString() { + return getClass().getSimpleName(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/EventList.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java similarity index 100% rename from flink-runtime/src/main/java/org/apache/flink/runtime/event/task/EventList.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java similarity index 100% rename from flink-runtime/src/main/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/StringTaskEvent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/StringTaskEvent.java similarity index 100% rename from flink-runtime/src/main/java/org/apache/flink/runtime/event/task/StringTaskEvent.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/event/task/StringTaskEvent.java From af88aa09ec94d4d11f38a7134d36793420d7d19d Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 30 Jul 2015 19:02:14 +0200 Subject: [PATCH 051/175] [FLINK-2438] [runtime] Improve channel event serialization performance. Because channel events may become very frequent now (frequent at-least-once checkpointing), their serialization perfomance starts to matter. --- .../event/{task => }/AbstractEvent.java | 2 +- .../event/{task => }/RuntimeEvent.java | 2 +- .../runtime/event/{task => }/TaskEvent.java | 2 +- .../io/network/TaskEventDispatcher.java | 2 +- .../io/network/api}/CheckpointBarrier.java | 35 ++--- .../io/network/api/EndOfPartitionEvent.java | 2 +- .../io/network/api/EndOfSuperstepEvent.java | 2 +- .../io/network/api/TaskEventHandler.java | 2 +- .../io/network/api/reader/AbstractReader.java | 4 +- .../io/network/api/reader/ReaderBase.java | 2 +- .../api/serialization/EventSerializer.java | 132 +++++++++++++----- .../io/network/api/writer/RecordWriter.java | 2 +- .../api/writer/ResultPartitionWriter.java | 4 +- .../buffer}/FreeingBufferRecycler.java | 12 +- .../io/network/netty/NettyMessage.java | 2 +- .../network/netty/PartitionRequestClient.java | 2 +- .../netty/PartitionRequestClientHandler.java | 5 +- .../partition/consumer/BufferOrEvent.java | 2 +- .../partition/consumer/InputChannel.java | 2 +- .../network/partition/consumer/InputGate.java | 2 +- .../partition/consumer/LocalInputChannel.java | 2 +- .../consumer/RemoteInputChannel.java | 2 +- .../partition/consumer/SingleInputGate.java | 4 +- .../partition/consumer/UnionInputGate.java | 2 +- .../consumer/UnknownInputChannel.java | 2 +- .../concurrent/SuperstepBarrier.java | 2 +- .../event/IterationEventWithAggregators.java | 2 +- .../iterative/event/TerminationEvent.java | 2 +- .../IterationSynchronizationSinkTask.java | 2 +- .../iterative/task/SyncEventHandler.java | 2 +- .../flink/runtime/event/task/EventList.java | 3 +- .../runtime/event/task/IntegerTaskEvent.java | 1 + .../runtime/event/task/StringTaskEvent.java | 1 + .../runtime/event/task/TaskEventTest.java | 3 +- .../api/reader/AbstractReaderTest.java | 9 +- .../network/api/reader/BufferReaderTest.java | 2 +- .../serialization/EventSerializerTest.java | 39 ++++-- .../partition/PipelinedSubpartitionTest.java | 2 +- .../partition/consumer/InputChannelTest.java | 2 +- .../consumer/SingleInputGateTest.java | 2 +- .../io/network/util/TestConsumerCallback.java | 2 +- .../util/TestSubpartitionConsumer.java | 2 +- .../io/network/util/TestTaskEvent.java | 2 +- .../concurrent/SuperstepBarrierTest.java | 2 +- .../util/event/TaskEventHandlerTest.java | 2 +- .../streaming/runtime/io/BarrierBuffer.java | 2 +- .../streaming/runtime/io/BarrierTracker.java | 2 +- .../streaming/runtime/io/BufferSpiller.java | 3 +- .../runtime/io/CheckpointBarrierHandler.java | 14 +- .../runtime/io/RecordWriterOutput.java | 4 +- .../runtime/io/StreamInputProcessor.java | 4 +- .../runtime/io/StreamTwoInputProcessor.java | 4 +- .../runtime/tasks/OutputHandler.java | 1 + .../streaming/runtime/tasks/StreamTask.java | 1 + .../consumer/StreamTestSingleInputGate.java | 3 +- .../io/BarrierBufferMassiveRandomTest.java | 4 +- .../runtime/io/BarrierBufferTest.java | 5 +- .../runtime/io/BarrierTrackerTest.java | 5 +- .../runtime/io/BufferSpillerTest.java | 1 + .../runtime/io/StreamRecordWriterTest.java | 1 + .../flink/streaming/runtime/io/TestEvent.java | 2 +- .../runtime/tasks/OneInputStreamTaskTest.java | 1 + .../tasks/OneInputStreamTaskTestHarness.java | 2 +- .../runtime/tasks/StreamMockEnvironment.java | 3 +- .../runtime/tasks/StreamTaskTestHarness.java | 2 +- .../runtime/tasks/TwoInputStreamTaskTest.java | 3 +- .../tasks/TwoInputStreamTaskTestHarness.java | 2 +- .../StreamingScalabilityAndLatency.java | 3 +- 68 files changed, 247 insertions(+), 141 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/event/{task => }/AbstractEvent.java (96%) rename flink-runtime/src/main/java/org/apache/flink/runtime/event/{task => }/RuntimeEvent.java (95%) rename flink-runtime/src/main/java/org/apache/flink/runtime/event/{task => }/TaskEvent.java (95%) rename {flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks => flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api}/CheckpointBarrier.java (69%) rename {flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io => flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer}/FreeingBufferRecycler.java (81%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/AbstractEvent.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/event/AbstractEvent.java index 244d67247afa9..a20aad9a9adcf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/AbstractEvent.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.event.task; +package org.apache.flink.runtime.event; import org.apache.flink.core.io.IOReadableWritable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/RuntimeEvent.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/event/RuntimeEvent.java index 8c44073e4b903..6d712ed62617e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/RuntimeEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/RuntimeEvent.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.event.task; +package org.apache.flink.runtime.event; /** * Subclasses of this event are recognized as events exchanged by the core runtime. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/TaskEvent.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/event/TaskEvent.java index 01ecce2733499..d37d0314759e7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/TaskEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/TaskEvent.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.event.task; +package org.apache.flink.runtime.event; /** * Subclasses of this event are recognized as custom events that are not part of the core diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java index 82793e2c33a1f..eddba8db46d42 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network; import com.google.common.collect.Maps; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/CheckpointBarrier.java similarity index 69% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointBarrier.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/CheckpointBarrier.java index d94b5b4b9c49d..59f56b07992d6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointBarrier.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/CheckpointBarrier.java @@ -16,32 +16,33 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks; +package org.apache.flink.runtime.io.network.api; import java.io.IOException; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.RuntimeEvent; /** - * Checkpoint barriers are used to synchronize checkpoints throughout the streaming topology. The + * Checkpoint barriers are used to align checkpoints throughout the streaming topology. The * barriers are emitted by the sources when instructed to do so by the JobManager. When - * operators receive a {@link CheckpointBarrier} on one of its inputs it must block processing - * of further elements on this input until all inputs received the checkpoint barrier - * corresponding to to that checkpoint. Once all inputs received the checkpoint barrier for - * a checkpoint the operator is to perform the checkpoint and then broadcast the barrier to - * downstream operators. - * - *

- * The checkpoint barrier IDs are advancing. Once an operator receives a {@link CheckpointBarrier} - * for a checkpoint with a higher id it is to discard all barriers that it received from previous - * checkpoints and unblock all other inputs. + * operators receive a CheckpointBarrier on one of its inputs, it knows that this is the point + * between the pre-checkpoint and post-checkpoint data. + * + *

Once an operator has received a checkpoint barrier from all its input channels, it + * knows that a certain checkpoint is complete. It can trigger the operator specific checkpoint + * behavior and broadcast the barrier to downstream operators.

+ * + *

Depending on the semantic guarantees, may hold off post-checkpoint data until the checkpoint + * is complete (exactly once)

+ * + *

The checkpoint barrier IDs are strictly monotonous increasing.

*/ -public class CheckpointBarrier extends TaskEvent { +public class CheckpointBarrier extends RuntimeEvent { - protected long id; - protected long timestamp; + private long id; + private long timestamp; public CheckpointBarrier() {} @@ -55,7 +56,7 @@ public long getId() { } public long getTimestamp() { - return id; + return timestamp; } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java index c2b6fa4d7dfed..293287ba844e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfPartitionEvent.java @@ -20,7 +20,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.RuntimeEvent; +import org.apache.flink.runtime.event.RuntimeEvent; /** * This event marks a subpartition as fully consumed. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java index 162afb713ce48..7f511876deebf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/EndOfSuperstepEvent.java @@ -20,7 +20,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.RuntimeEvent; +import org.apache.flink.runtime.event.RuntimeEvent; /** * Marks the end of a superstep of one particular iteration superstep. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java index ccd0febe06659..d2dc46b5520b6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java @@ -20,7 +20,7 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractReader.java index 90564a8446384..84189be1745c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractReader.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.io.network.api.reader; -import org.apache.flink.runtime.event.task.AbstractEvent; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; import org.apache.flink.runtime.io.network.api.TaskEventHandler; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java index 9f8ae20002e3c..a1d705f47a151 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java @@ -21,7 +21,7 @@ import java.io.IOException; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java index 76c88c17edb00..b23b83b098708 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java @@ -19,63 +19,122 @@ package org.apache.flink.runtime.io.network.api.serialization; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.util.DataInputDeserializer; import org.apache.flink.runtime.util.DataOutputSerializer; import org.apache.flink.util.InstantiationUtil; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; +/** + * Utility class to serialize and deserialize task events. + */ public class EventSerializer { + + private static final int END_OF_PARTITION_EVENT = 0; - public final static BufferRecycler RECYCLER = new BufferRecycler() { - @Override - public void recycle(MemorySegment memorySegment) { - memorySegment.free(); - } - }; - - public static ByteBuffer toSerializedEvent(AbstractEvent event) { - try { - final DataOutputSerializer serializer = new DataOutputSerializer(128); + private static final int CHECKPOINT_BARRIER_EVENT = 1; - serializer.writeUTF(event.getClass().getName()); - event.write(serializer); + private static final int END_OF_SUPERSTEP_EVENT = 2; - return serializer.wrapAsByteBuffer(); + private static final int OTHER_EVENT = 3; + + // ------------------------------------------------------------------------ + + public static ByteBuffer toSerializedEvent(AbstractEvent event) { + final Class eventClass = event.getClass(); + if (eventClass == EndOfPartitionEvent.class) { + return ByteBuffer.wrap(new byte[] { 0, 0, 0, END_OF_PARTITION_EVENT }); + } + else if (eventClass == CheckpointBarrier.class) { + CheckpointBarrier barrier = (CheckpointBarrier) event; + + ByteBuffer buf = ByteBuffer.allocate(20); + buf.putInt(0, CHECKPOINT_BARRIER_EVENT); + buf.putLong(4, barrier.getId()); + buf.putLong(12, barrier.getTimestamp()); + return buf; } - catch (IOException e) { - throw new RuntimeException("Error while serializing event.", e); + else if (eventClass == EndOfSuperstepEvent.class) { + return ByteBuffer.wrap(new byte[] { 0, 0, 0, END_OF_SUPERSTEP_EVENT }); + } + else { + try { + final DataOutputSerializer serializer = new DataOutputSerializer(128); + serializer.writeInt(OTHER_EVENT); + serializer.writeUTF(event.getClass().getName()); + event.write(serializer); + + return serializer.wrapAsByteBuffer(); + } + catch (IOException e) { + throw new RuntimeException("Error while serializing event.", e); + } } } public static AbstractEvent fromSerializedEvent(ByteBuffer buffer, ClassLoader classLoader) { + if (buffer.remaining() < 4) { + throw new RuntimeException("Incomplete event"); + } + + final ByteOrder bufferOrder = buffer.order(); + buffer.order(ByteOrder.BIG_ENDIAN); + try { - final DataInputDeserializer deserializer = new DataInputDeserializer(buffer); - - final String className = deserializer.readUTF(); - - final Class clazz; - try { - clazz = classLoader.loadClass(className).asSubclass(AbstractEvent.class); + int type = buffer.getInt(); + + if (type == END_OF_PARTITION_EVENT) { + return EndOfPartitionEvent.INSTANCE; } - catch (ClassNotFoundException e) { - throw new RuntimeException("Could not load event class '" + className + "'.", e); + else if (type == CHECKPOINT_BARRIER_EVENT) { + long id = buffer.getLong(); + long timestamp = buffer.getLong(); + return new CheckpointBarrier(id, timestamp); } - catch (ClassCastException e) { - throw new RuntimeException("The class '" + className + "' is not a valid subclass of '" + AbstractEvent.class.getName() + "'.", e); + else if (type == END_OF_SUPERSTEP_EVENT) { + return EndOfSuperstepEvent.INSTANCE; + } + else if (type == OTHER_EVENT) { + try { + final DataInputDeserializer deserializer = new DataInputDeserializer(buffer); + + final String className = deserializer.readUTF(); + + final Class clazz; + try { + clazz = classLoader.loadClass(className).asSubclass(AbstractEvent.class); + } + catch (ClassNotFoundException e) { + throw new RuntimeException("Could not load event class '" + className + "'.", e); + } + catch (ClassCastException e) { + throw new RuntimeException("The class '" + className + "' is not a valid subclass of '" + + AbstractEvent.class.getName() + "'.", e); + } + + final AbstractEvent event = InstantiationUtil.instantiate(clazz, AbstractEvent.class); + event.read(deserializer); + + return event; + } + catch (Exception e) { + throw new RuntimeException("Error while deserializing or instantiating event.", e); + } + } + else { + throw new RuntimeException("Corrupt byte stream for event"); } - - final AbstractEvent event = InstantiationUtil.instantiate(clazz, AbstractEvent.class); - event.read(deserializer); - - return event; } - catch (IOException e) { - throw new RuntimeException("Error while deserializing event.", e); + finally { + buffer.order(bufferOrder); } } @@ -86,7 +145,8 @@ public static AbstractEvent fromSerializedEvent(ByteBuffer buffer, ClassLoader c public static Buffer toBuffer(AbstractEvent event) { final ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(event); - final Buffer buffer = new Buffer(new MemorySegment(serializedEvent.array()), RECYCLER, false); + final Buffer buffer = new Buffer(new MemorySegment(serializedEvent.array()), + FreeingBufferRecycler.INSTANCE, false); buffer.setSize(serializedEvent.remaining()); return buffer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 885c316fcced4..2ae61ed60ed65 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -20,7 +20,7 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index 1192dbba499e5..79c21c6a5a601 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.io.network.api.writer; -import org.apache.flink.runtime.event.task.AbstractEvent; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; import org.apache.flink.runtime.io.network.api.TaskEventHandler; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/FreeingBufferRecycler.java similarity index 81% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/FreeingBufferRecycler.java index 27e37a5d215e1..fdce8837046bc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/FreeingBufferRecycler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/FreeingBufferRecycler.java @@ -16,13 +16,12 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.io; +package org.apache.flink.runtime.io.network.buffer; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.io.network.buffer.BufferRecycler; /** - * A simple buffer recycler that only frees the memory segments. + * A simple buffer recycler that frees the memory segments. */ public class FreeingBufferRecycler implements BufferRecycler { @@ -30,6 +29,13 @@ public class FreeingBufferRecycler implements BufferRecycler { // ------------------------------------------------------------------------ + // Not instantiable + private FreeingBufferRecycler() {} + + /** + * Frees the given memory segment. + * @param memorySegment The memory segment to be recycled. + */ @Override public void recycle(MemorySegment memorySegment) { memorySegment.free(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java index 154036939bedd..3a24181bfa561 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java @@ -32,7 +32,7 @@ import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java index 78f6398c4c227..f6120d4066730 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java @@ -21,7 +21,7 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java index 51b436bb0f8eb..3b7d921024dcc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java @@ -21,10 +21,11 @@ import com.google.common.collect.Maps; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; + import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException; import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException; import org.apache.flink.runtime.io.network.netty.exception.TransportException; @@ -300,7 +301,7 @@ else if (bufferProvider.isDestroyed()) { byte[] byteArray = new byte[bufferOrEvent.getSize()]; bufferOrEvent.getNettyBuffer().readBytes(byteArray); - Buffer buffer = new Buffer(new MemorySegment(byteArray), EventSerializer.RECYCLER, false); + Buffer buffer = new Buffer(new MemorySegment(byteArray), FreeingBufferRecycler.INSTANCE, false); inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java index d2f3035f44852..55e57677b182f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import static com.google.common.base.Preconditions.checkArgument; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 3998279320cbc..d282db5c16aea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java index c4f9dc4bb73ea..1f42cfa3a3885 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; import java.io.IOException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index e72f612bea70c..ff12153ec976f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index b70c3a868ac01..be2509f04d606 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.buffer.Buffer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 80a79d26b8b66..896fa9cb88d54 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -23,8 +23,8 @@ import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionLocation; -import org.apache.flink.runtime.event.task.AbstractEvent; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 730ead21e461e..559968719f901 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -20,7 +20,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.util.event.EventListener; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index e4b9e57ace1a1..cdf28be025c46 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.TaskEventDispatcher; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrier.java index c91be1a27583e..cc5d3c583cec5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrier.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrier.java @@ -21,7 +21,7 @@ import java.util.concurrent.CountDownLatch; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent; import org.apache.flink.runtime.iterative.event.TerminationEvent; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/IterationEventWithAggregators.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/IterationEventWithAggregators.java index bc815dcff855c..e25952334f656 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/IterationEventWithAggregators.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/IterationEventWithAggregators.java @@ -31,7 +31,7 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.InputViewDataInputStreamWrapper; import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.types.Value; import org.apache.flink.util.InstantiationUtil; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/TerminationEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/TerminationEvent.java index 9e74c34586f5e..28181e8a1b32f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/TerminationEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/event/TerminationEvent.java @@ -23,7 +23,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; /** * Signals that the iteration is completely executed, participating tasks must terminate now diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java index 5eccd7bcbd188..fed0a1716e6f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java @@ -24,7 +24,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.types.IntValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/SyncEventHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/SyncEventHandler.java index 8cce6ef666639..d7549d1f96ebe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/SyncEventHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/SyncEventHandler.java @@ -21,7 +21,7 @@ import java.util.Map; import org.apache.flink.api.common.aggregators.Aggregator; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.runtime.iterative.event.WorkerDoneEvent; import org.apache.flink.types.Value; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java index 1f97a15179556..c055a928288b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java @@ -19,10 +19,11 @@ package org.apache.flink.runtime.event.task; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.util.SerializableArrayList; /** - * Objects of this class can store and serialize/deserialize {@link org.apache.flink.runtime.event.task.AbstractEvent} + * Objects of this class can store and serialize/deserialize {@link org.apache.flink.runtime.event.AbstractEvent} * objects. * */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java index 648dacc619f63..cc67482ee2ac3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/IntegerTaskEvent.java @@ -23,6 +23,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.event.TaskEvent; /** * This class provides a simple implementation of an event that holds an integer value. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/StringTaskEvent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/StringTaskEvent.java index 87f2e91350d01..9095cc21cea86 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/StringTaskEvent.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/StringTaskEvent.java @@ -22,6 +22,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.util.StringUtils; /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java index 1ed8e39c2e588..b508923cd3637 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java @@ -25,12 +25,13 @@ import java.io.IOException; import java.util.Iterator; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.junit.Test; /** * This class contains serialization tests concerning task events derived from - * {@link org.apache.flink.runtime.event.task.AbstractEvent}. + * {@link org.apache.flink.runtime.event.AbstractEvent}. * */ public class TaskEventTest { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java index 14bf0220f267c..6853722330c7a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java @@ -18,11 +18,10 @@ package org.apache.flink.runtime.io.network.api.reader; -import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; @@ -70,7 +69,7 @@ public void testTaskEvent() throws Exception { public void testEndOfPartitionEvent() throws Exception { final AbstractReader reader = new MockReader(createInputGate(1)); - assertTrue(reader.handleEvent(new EndOfPartitionEvent())); + assertTrue(reader.handleEvent(EndOfPartitionEvent.INSTANCE)); } /** @@ -95,7 +94,7 @@ public void testExceptionsNonIterativeReader() throws Exception { } try { - reader.handleEvent(new EndOfSuperstepEvent()); + reader.handleEvent(EndOfSuperstepEvent.INSTANCE); fail("Did not throw expected exception when handling end of superstep event with non-iterative reader."); } @@ -122,7 +121,7 @@ public void testEndOfSuperstepEventLogic() throws IOException { // All good, expected exception. } - EndOfSuperstepEvent eos = new EndOfSuperstepEvent(); + EndOfSuperstepEvent eos = EndOfSuperstepEvent.INSTANCE; // One end of superstep event for each input channel. The superstep finishes with the last // received event. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java index e1f8fd83806dc..8519ac6eaa299 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.api.reader; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.TestSingleInputGate; import org.apache.flink.runtime.io.network.util.TestTaskEvent; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java index 5a20a4b9d80d7..ddfd758914cb6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java @@ -18,25 +18,44 @@ package org.apache.flink.runtime.io.network.api.serialization; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; import org.apache.flink.runtime.io.network.util.TestTaskEvent; + import org.junit.Test; import java.nio.ByteBuffer; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.*; public class EventSerializerTest { @Test public void testSerializeDeserializeEvent() { - - TestTaskEvent expected = new TestTaskEvent(Math.random(), 12361231273L); - - ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(expected); - - AbstractEvent actual = EventSerializer.fromSerializedEvent(serializedEvent, getClass().getClassLoader()); - - assertEquals(expected, actual); + try { + AbstractEvent[] events = { + EndOfPartitionEvent.INSTANCE, + EndOfSuperstepEvent.INSTANCE, + new CheckpointBarrier(1678L, 4623784L), + new TestTaskEvent(Math.random(), 12361231273L) + }; + + for (AbstractEvent evt : events) { + ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(evt); + assertTrue(serializedEvent.hasRemaining()); + + AbstractEvent deserialized = + EventSerializer.fromSerializedEvent(serializedEvent, getClass().getClassLoader()); + assertNotNull(deserialized); + assertEquals(evt, deserialized); + } + + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 74549df365166..8750a1a1e1c2b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java index e95c77459fe37..97175303e2616 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 24543994ac78c..82cc730300aed 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -22,7 +22,7 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionLocation; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.TaskEventDispatcher; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java index 52083c438a94e..0d66f13ccf8c4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.util; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import java.util.concurrent.atomic.AtomicInteger; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java index 2766e5310341e..18e0d4a9b5715 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.util; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java index 0b29032027b92..091d5d77aeead 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java @@ -20,7 +20,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import java.io.IOException; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrierTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrierTest.java index a6c0d72d12c53..2f26670f6f926 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrierTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/SuperstepBarrierTest.java @@ -24,7 +24,7 @@ import java.util.Random; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent; import org.apache.flink.runtime.iterative.event.TerminationEvent; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java index 5c6aeb719ae8d..cb76276c32a5e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.event.task.IntegerTaskEvent; import org.apache.flink.runtime.event.task.StringTaskEvent; import org.apache.flink.runtime.io.network.api.TaskEventHandler; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index 0441937eb5594..b7766eea995c7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java index a0b924fafe77c..119fb238cda0e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import java.io.IOException; import java.util.ArrayDeque; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java index 5f9a162b547d6..2bad1979a980b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java @@ -27,10 +27,11 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.util.StringUtils; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java index 02dd33d5da066..791fd40c395bc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import java.io.IOException; @@ -43,8 +43,18 @@ public interface CheckpointBarrierHandler { */ BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException; + /** + * Registers the given event handler to be notified on successful checkpoints. + * + * @param checkpointHandler The handler to register. + */ void registerCheckpointEventHandler(EventListener checkpointHandler); - + + /** + * Cleans up all internally held resources. + * + * @throws IOException Thrown, if the cleanup of I/O resources failed. + */ void cleanup() throws IOException; /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index de8c205dd677e..f0f18b1a2dc26 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -21,7 +21,7 @@ import com.google.common.base.Preconditions; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.api.operators.Output; @@ -111,7 +111,7 @@ public void clearBuffers() { recordWriter.clearBuffers(); } - public void broadcastEvent(TaskEvent barrier) throws IOException, InterruptedException { + public void broadcastEvent(AbstractEvent barrier) throws IOException, InterruptedException { recordWriter.broadcastEvent(barrier); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index f7d7fb0bff0ed..4ad5b45985304 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.reader.AbstractReader; import org.apache.flink.runtime.io.network.api.reader.ReaderBase; @@ -41,7 +41,7 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import org.slf4j.Logger; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index ae97974c9558b..e3d29116d4bb8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.reader.AbstractReader; import org.apache.flink.runtime.io.network.api.reader.ReaderBase; @@ -39,7 +39,7 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java index 84614bf8c2953..c8fa9e3bcd5fa 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 2098da83e1ce3..aabc95d0df956 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -32,6 +32,7 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.CheckpointNotificationOperator; import org.apache.flink.runtime.jobgraph.tasks.CheckpointedOperator; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index 4007da85fe2da..173e894d1ebfb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; @@ -31,7 +31,6 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java index 7350516204de7..678b14521cef1 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.Random; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -31,7 +31,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.junit.Test; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index b8b3a8c3ec378..872e22678bb52 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -19,15 +19,16 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.junit.AfterClass; import org.junit.BeforeClass; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java index 532078c5c8b3d..fb61633b38081 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java @@ -19,12 +19,13 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.junit.Test; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java index ae384e16c65c8..05ce4be84d08f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.junit.After; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java index fb3beeae7a1a6..f07e3a5960108 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.types.LongValue; import org.junit.Test; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java index 4a7775717e5d7..286477a1bc40b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java @@ -20,7 +20,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.util.StringUtils; import java.io.IOException; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 4399a104632ef..296324a7d6866 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamMap; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java index a8029e6a2a38c..4f07fdb234dd5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java @@ -19,7 +19,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import java.io.IOException; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index df0c9ee7f3bce..bbc64f1e9ffde 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -18,14 +18,13 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index 435831f017721..0f372cbdd90a8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.MemoryManager; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index 3c7204de09e30..f87d7ea9da267 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoMapFunction; @@ -28,7 +29,6 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.TestHarnessUtil; -import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -36,7 +36,6 @@ import org.powermock.modules.junit4.PowerMockRunner; import java.util.List; -import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java index f37eb6632bb13..2b201017cb91a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java @@ -45,7 +45,7 @@ * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task * thread to finish. Use {@link #processElement} * to send elements to the task. Use - * {@link #processEvent(org.apache.flink.runtime.event.task.AbstractEvent)} to send events to the task. + * {@link #processEvent(org.apache.flink.runtime.event.AbstractEvent)} to send events to the task. * Before waiting for the task to finish you must call {@link #endInput()} to signal to the task * that data entry is finished. * diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java index a34ec1576f710..8d41292fdf3f1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.StreamingMode; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; @@ -74,7 +75,7 @@ private static void runPartitioningProgram(int jobManagerPort, int parallelism) env.getConfig().enableObjectReuse(); env.setBufferTimeout(5L); -// env.enableCheckpointing(1000); + env.enableCheckpointing(1000, CheckpointingMode.AT_LEAST_ONCE); env .addSource(new TimeStampingSource()) From 40eef52e9f68c3c7727e9b9493959d5fd36d7f70 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 30 Jul 2015 21:09:57 +0200 Subject: [PATCH 052/175] [hotfix] [streaming] Fix race in stream tasks when canceling tasks early. --- .../flink/streaming/runtime/tasks/OneInputStreamTask.java | 2 -- .../flink/streaming/runtime/tasks/SourceStreamTask.java | 2 -- .../flink/streaming/runtime/tasks/StreamIterationHead.java | 1 - .../apache/flink/streaming/runtime/tasks/StreamTask.java | 7 +++++-- .../flink/streaming/runtime/tasks/TwoInputStreamTask.java | 2 -- 5 files changed, 5 insertions(+), 9 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index 605b8f5cfeb1a..6136f247cbcb9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -61,8 +61,6 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { - this.isRunning = true; - boolean operatorOpen = false; if (LOG.isDebugEnabled()) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index 1940c11f50c99..4b25577d26d96 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -45,8 +45,6 @@ public class SourceStreamTask extends StreamTask> { public void invoke() throws Exception { final SourceOutput> output = new SourceOutput>(outputHandler.getOutput(), checkpointLock); - this.isRunning = true; - boolean operatorOpen = false; if (LOG.isDebugEnabled()) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java index 1736e525b97d8..2911f44963b3c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java @@ -72,7 +72,6 @@ public void registerInputOutput() { @SuppressWarnings("unchecked") @Override public void invoke() throws Exception { - isRunning = true; if (LOG.isDebugEnabled()) { LOG.debug("Iteration source {} invoked", getName()); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index aabc95d0df956..88813d082990b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -66,7 +66,8 @@ public abstract class StreamTask> extends Abs protected boolean hasChainedOperators; - protected volatile boolean isRunning = false; + // needs to be initialized to true, so that early cancel() before invoke() behaves correctly + protected volatile boolean isRunning = true; protected List contexts; @@ -229,10 +230,12 @@ public void setInitialState(StateHandle stateHandle) throws Except @Override public void triggerCheckpoint(long checkpointId, long timestamp) throws Exception { + LOG.debug("Starting checkpoint {} on task {}", checkpointId, getName()); + synchronized (checkpointLock) { if (isRunning) { try { - LOG.debug("Starting checkpoint {} on task {}", checkpointId, getName()); + // We wrap the states of the chained operators in a list, marking non-stateful oeprators with null List, Map>> chainedStates = new ArrayList, Map>>(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index 99c053b124271..8cf5a405a885e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -85,8 +85,6 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { - this.isRunning = true; - boolean operatorOpen = false; if (LOG.isDebugEnabled()) { From 81f9b214a37aa81017eface6ba43f24f845a4ec8 Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Sat, 1 Aug 2015 21:04:39 +0530 Subject: [PATCH 053/175] [FLINK-2459] [cli] Cli API and doc fixes. 1. Remove CliFrontendLoggingTest. Test directly that the logging flag is interpreted correctly. 2. Doc fix for cli api 3. Info command shouldn't print logging option for help. This closes #971 --- docs/apis/cli.md | 5 + .../flink/client/cli/CliFrontendParser.java | 3 +- .../apache/flink/client/program/Client.java | 7 ++ .../flink/client/CliFrontendLoggingTest.java | 113 ------------------ .../flink/client/CliFrontendRunTest.java | 18 ++- 5 files changed, 28 insertions(+), 118 deletions(-) delete mode 100644 flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java diff --git a/docs/apis/cli.md b/docs/apis/cli.md index c0ae9b15b3fb0..5117bbb02e1da 100644 --- a/docs/apis/cli.md +++ b/docs/apis/cli.md @@ -128,6 +128,11 @@ Action "run" compiles and runs a program. program. Optional flag to override the default value specified in the configuration. + -q --sysoutLogging Specfying this flag will disable log messages + being reported on the console. All messages + however will still be logged by SLF4J loggers, + regardless of this setting. + Additional arguments if -m yarn-cluster is set: -yD Dynamic properties -yd,--yarndetached Start detached diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index ae8499b4d4769..0a9af8d98b67d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -149,7 +149,8 @@ private static Options getInfoOptions(Options options) { } private static Options getInfoOptionsWithoutDeprecatedOptions(Options options) { - options = getProgramSpecificOptionsWithoutDeprecatedOptions(options); + options.addOption(CLASS_OPTION); + options.addOption(PARALLELISM_OPTION); options = getJobManagerAddressOption(options); return options; } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 1de0703c266f9..d2022bbf78649 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -191,6 +191,13 @@ public void setPrintStatusDuringExecution(boolean print) { this.printStatusDuringExecution = print; } + /** + * @return whether the client will print progress updates during the execution to {@code System.out} + */ + public boolean getPrintStatusDuringExecution() { + return this.printStatusDuringExecution; + } + /** * @return -1 if unknown. The maximum number of available processing slots at the Flink cluster * connected to this client. diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java deleted file mode 100644 index 157e0705860cd..0000000000000 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendLoggingTest.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.client; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.StreamingMode; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; - -import static org.apache.flink.client.CliFrontendTestUtils.getTestJarPath; -import static org.apache.flink.client.CliFrontendTestUtils.getConfigDir; -import static org.junit.Assert.fail; - -public class CliFrontendLoggingTest { - - private static LocalFlinkMiniCluster cluster; - private static Configuration config; - private static String hostPort; - private ByteArrayOutputStream stream = new ByteArrayOutputStream(); - private CliFrontend cli; - private PrintStream output; - - @Before - public void setUp() throws Exception { - stream.reset(); - output = System.out; - System.setOut(new PrintStream(stream)); - - config = new Configuration(); - config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); - config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1); - config.setBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, false); - hostPort = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null) + ":" + - config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); - - try { - cluster = new LocalFlinkMiniCluster(config, false, StreamingMode.BATCH_ONLY); - } - catch (Exception e) { - e.printStackTrace(); - fail("Setup of test actor system failed."); - } - - cli = new CliFrontend(getConfigDir()); - } - - @After - public void shutDownActorSystem() { - System.setOut(output); - if(cluster != null){ - cluster.shutdown(); - } - } - - @Test - public void verifyLogging(){ - try { - int ret = cli.run(new String[]{"-m", hostPort, getTestJarPath()}); - System.out.flush(); - assert(ret == 0 && checkForLogs(stream.toString())); - } catch(Exception e){ - e.printStackTrace(); - fail(e.getMessage()); - } finally { - if(cluster != null){ - cluster.shutdown(); - } - } - } - - @Test - public void verifyNoLogging(){ - try { - int ret = cli.run(new String[]{"-q", "-m", hostPort, getTestJarPath()}); - System.out.flush(); - assert(ret == 0 && !checkForLogs(stream.toString())); - } catch(Exception e){ - e.printStackTrace(); - fail(e.getMessage()); - } finally { - if(cluster != null){ - cluster.shutdown(); - } - } - } - - private boolean checkForLogs(String output){ - return output.indexOf("RUNNING") >= 0; - } -} diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java index 034ee4e82a033..6798806618a80 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java @@ -50,17 +50,24 @@ public void testRun() { // test without parallelism { String[] parameters = {"-v", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(-1); + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(-1, true); assertEquals(0, testFrontend.run(parameters)); } // test configure parallelism { String[] parameters = {"-v", "-p", "42", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(42); + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(42, true); assertEquals(0, testFrontend.run(parameters)); } - + + // test configure sysout logging + { + String[] parameters = {"-p", "2", "-q", getTestJarPath()}; + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false); + assertEquals(0, testFrontend.run(parameters)); + } + // test configure parallelism with non integer value { String[] parameters = {"-v", "-p", "text", getTestJarPath()}; @@ -86,15 +93,18 @@ public void testRun() { public static final class RunTestingCliFrontend extends CliFrontend { private final int expectedParallelim; + private final boolean sysoutLogging; - public RunTestingCliFrontend(int expectedParallelim) throws Exception { + public RunTestingCliFrontend(int expectedParallelim, boolean logging) throws Exception { super(CliFrontendTestUtils.getConfigDir()); this.expectedParallelim = expectedParallelim; + this.sysoutLogging = logging; } @Override protected int executeProgram(PackagedProgram program, Client client, int parallelism, boolean wait) { assertEquals(this.expectedParallelim, parallelism); + assertEquals(client.getPrintStatusDuringExecution(), sysoutLogging); return 0; } } From b3dc3f050c125978f4fb14ccef230938e60a9198 Mon Sep 17 00:00:00 2001 From: Enrique Bautista Date: Wed, 29 Jul 2015 22:49:13 +0200 Subject: [PATCH 054/175] [FLINK-2463] [web dashboard] Shows job configuration in new dashboard This closes #953 --- .../runtime/webmonitor/WebRuntimeMonitor.java | 2 + .../webmonitor/handlers/JobConfigHandler.java | 81 +++++++++++++++++++ .../app/partials/jobs/job.config.jade | 50 ++++++++++++ .../web-dashboard/app/partials/jobs/job.jade | 3 + .../web-dashboard/app/scripts/index.coffee | 6 ++ .../app/scripts/modules/jobs/jobs.svc.coffee | 8 +- .../web-dashboard/web/js/index.js | 16 +++- .../web/partials/jobs/job.config.html | 57 +++++++++++++ .../web-dashboard/web/partials/jobs/job.html | 1 + 9 files changed, 219 insertions(+), 5 deletions(-) create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java create mode 100644 flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade create mode 100644 flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java index 0aa6b07f2c11e..a2095d4852dd3 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java @@ -37,6 +37,7 @@ import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler; import org.apache.flink.runtime.webmonitor.handlers.ExecutionPlanHandler; +import org.apache.flink.runtime.webmonitor.handlers.JobConfigHandler; import org.apache.flink.runtime.webmonitor.handlers.JobSummaryHandler; import org.apache.flink.runtime.webmonitor.handlers.JobVerticesOverviewHandler; import org.apache.flink.runtime.webmonitor.handlers.RequestConfigHandler; @@ -132,6 +133,7 @@ else if (flinkRoot != null) { .GET("/jobs/:jobid", handler(new JobSummaryHandler(currentGraphs))) .GET("/jobs/:jobid/vertices", handler(new JobVerticesOverviewHandler(currentGraphs))) .GET("/jobs/:jobid/plan", handler(new ExecutionPlanHandler(currentGraphs))) + .GET("/jobs/:jobid/config", handler(new JobConfigHandler(currentGraphs))) // .GET("/running/:jobid/:jobvertex", handler(new ExecutionPlanHandler(currentGraphs))) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java new file mode 100644 index 0000000000000..50cfd961e616b --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java @@ -0,0 +1,81 @@ +/* + * 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.runtime.webmonitor.handlers; + +import java.util.Map; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Request handler that returns the configuration of a job. + */ +public class JobConfigHandler extends AbstractExecutionGraphRequestHandler implements RequestHandler.JsonResponse { + + private static final Logger LOG = LoggerFactory.getLogger(JobConfigHandler.class); + + public JobConfigHandler(ExecutionGraphHolder executionGraphHolder) { + super(executionGraphHolder); + } + + @Override + public String handleRequest(ExecutionGraph graph, Map params) throws Exception { + + JSONObject obj = new JSONObject(); + + obj.put("jid", graph.getJobID().toString()); + obj.put("name", graph.getJobName()); + + JSONObject execConfig = new JSONObject(); + + ExecutionConfig ec = graph.getExecutionConfig(); + if (ec != null) { + execConfig.put("execution-mode", ec.getExecutionMode()); + execConfig.put("max-execution-retries", ec.getNumberOfExecutionRetries()); + execConfig.put("job-parallelism", ec.getParallelism()); + execConfig.put("object-reuse-mode", ec.isObjectReuseEnabled()); + + ExecutionConfig.GlobalJobParameters uc = ec.getGlobalJobParameters(); + if (uc != null) { + Map ucVals = uc.toMap(); + if (ucVals != null) { + JSONObject userConfig = new JSONObject(); + for (Map.Entry ucVal : ucVals.entrySet()) { + userConfig.put(ucVal.getKey(), ucVal.getValue()); + } + execConfig.put("user-config", userConfig); + } else { + LOG.debug("GlobalJobParameters.toMap() did not return anything"); + } + } else { + LOG.debug("No GlobalJobParameters were set in the execution config"); + } + } else { + LOG.warn("Unable to retrieve execution config from execution graph"); + } + + obj.put("execution-config", execConfig); + + return obj.toString(); + } +} diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade new file mode 100644 index 0000000000000..83b78807685e8 --- /dev/null +++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade @@ -0,0 +1,50 @@ +// + 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. + +table.table.table-properties(ng-if="job['execution-config']") + thead + tr + th(colspan="2") + | Execution configuration + + tbody + tr + td Execution mode + td {{ job['execution-config']['execution-mode'] }} + + tr + td Max. number of execution retries + td {{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }} + + tr + td Job parallelism + td {{ job['execution-config']['job-parallelism'] === -1 ? 'auto' : job['execution-config']['job-parallelism'] }} + + tr + td Object reuse mode + td {{ job['execution-config']['object-reuse-mode'] }} + +table.table.table-properties(ng-if="job['execution-config']['user-config']") + thead + tr + th(colspan="2") + | User configuration + + tbody + tr(ng-repeat="property in job['execution-config']['user-config']") + td {{property.name}} + td(table-property value="property.value") diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade index 2068529689d4b..2591aff295db4 100644 --- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade +++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade @@ -52,5 +52,8 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main-additional(ng-if="job") li(ui-sref-active='active') a(ui-sref=".exceptions") Exceptions + li(ui-sref-active='active') + a(ui-sref=".config") Configuration + #content-inner.has-navbar-main-additional div(ui-view="details") diff --git a/flink-runtime-web/web-dashboard/app/scripts/index.coffee b/flink-runtime-web/web-dashboard/app/scripts/index.coffee index 385b3d6aa6254..4ff05028c9cf2 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/index.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/index.coffee @@ -117,4 +117,10 @@ angular.module('flinkApp', ['ui.router', 'angularMoment']) details: templateUrl: "partials/jobs/job.exceptions.html" + .state "single-job.config", + url: "/config" + views: + details: + templateUrl: "partials/jobs/job.config.html" + $urlRouterProvider.otherwise "/overview" diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee index 11a08efa99f24..3c5c38549e6fe 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee @@ -112,8 +112,12 @@ angular.module('flinkApp') .success (oldVertices) -> data.oldV = oldVertices[0] - currentJob = data - deferreds.job.resolve(data) + $http.get flinkConfig.jobServer + "/jobs/" + jobid + "/config" + .success (jobConfig) -> + data = angular.extend(data, jobConfig) + + currentJob = data + deferreds.job.resolve(data) deferreds.job.promise diff --git a/flink-runtime-web/web-dashboard/web/js/index.js b/flink-runtime-web/web-dashboard/web/js/index.js index 14f45e3867ea2..f878b520236d8 100644 --- a/flink-runtime-web/web-dashboard/web/js/index.js +++ b/flink-runtime-web/web-dashboard/web/js/index.js @@ -92,6 +92,13 @@ angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", fu templateUrl: "partials/jobs/job.exceptions.html" } } + }).state("single-job.config", { + url: "/config", + views: { + details: { + templateUrl: "partials/jobs/job.config.html" + } + } }); return $urlRouterProvider.otherwise("/overview"); }]); @@ -717,8 +724,11 @@ angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log data = angular.extend(data, vertices); return $http.get(flinkConfig.jobServer + "/jobsInfo?get=job&job=" + jobid).success(function(oldVertices) { data.oldV = oldVertices[0]; - currentJob = data; - return deferreds.job.resolve(data); + return $http.get(flinkConfig.jobServer + "/jobs/" + jobid + "/config").success(function(jobConfig) { + data = angular.extend(data, jobConfig); + currentJob = data; + return deferreds.job.resolve(data); + }); }); }); }); @@ -800,4 +810,4 @@ angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", " return this; }]); -//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsIm1vZHVsZXMvam9icy9qb2JzLmN0cmwuY29mZmVlIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5qcyIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5kaXIuanMiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JzL2pvYnMuc3ZjLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvb3ZlcnZpZXcvb3ZlcnZpZXcuY3RybC5qcyIsIm1vZHVsZXMvb3ZlcnZpZXcvb3ZlcnZpZXcuc3ZjLmNvZmZlZSIsIm1vZHVsZXMvb3ZlcnZpZXcvb3ZlcnZpZXcuc3ZjLmpzIl0sIm5hbWVzIjpbXSwibWFwcGluZ3MiOiJBQWtCQSxRQUFRLE9BQU8sWUFBWSxDQUFDLGFBQWEsa0JBSXhDLG1CQUFJLFNBQUMsWUFBRDtFQUNILFdBQVcsaUJBQWlCO0VDckI1QixPRHNCQSxXQUFXLGNBQWMsV0FBQTtJQUN2QixXQUFXLGlCQUFpQixDQUFDLFdBQVc7SUNyQnhDLE9Ec0JBLFdBQVcsZUFBZTs7SUFJN0IsU0FBUyxlQUFlO0VBQ3ZCLFdBQVc7RUFDWCxXQUFXO0VBR1gsaUJBQWlCO0dBS2xCLGdEQUFJLFNBQUMsYUFBYSxhQUFhLFdBQTNCO0VBQ0gsWUFBWTtFQzdCWixPRCtCQSxVQUFVLFdBQUE7SUM5QlIsT0QrQkEsWUFBWTtLQUNaLFlBQVk7SUFLZixnREFBTyxTQUFDLGdCQUFnQixvQkFBakI7RUFDTixlQUFlLE1BQU0sWUFDbkI7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sY0FDTDtJQUFBLEtBQUs7SUFDTCxVQUFVO0lBQ1YsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sbUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sd0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sdUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSw4QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsUUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7OztFQ3RCbkIsT0R3QkEsbUJBQW1CLFVBQVU7O0FDdEIvQjtBQy9FQSxRQUFRLE9BQU8sWUFJZCxVQUFVLDJCQUFXLFNBQUMsYUFBRDtFQ3JCcEIsT0RzQkE7SUFBQSxZQUFZO0lBQ1osU0FBUztJQUNULE9BQ0U7TUFBQSxlQUFlO01BQ2YsUUFBUTs7SUFFVixVQUFVO0lBRVYsTUFBTSxTQUFDLE9BQU8sU0FBUyxPQUFqQjtNQ3JCRixPRHNCRixNQUFNLGdCQUFnQixXQUFBO1FDckJsQixPRHNCRixpQkFBaUIsWUFBWSxvQkFBb0IsTUFBTTs7OztJQUk1RCxVQUFVLG9DQUFvQixTQUFDLGFBQUQ7RUNyQjdCLE9Ec0JBO0lBQUEsU0FBUztJQUNULE9BQ0U7TUFBQSxlQUFlO01BQ2YsUUFBUTs7SUFFVixVQUFVO0lBRVYsTUFBTSxTQUFDLE9BQU8sU0FBUyxPQUFqQjtNQ3JCRixPRHNCRixNQUFNLGdCQUFnQixXQUFBO1FDckJsQixPRHNCRixzQ0FBc0MsWUFBWSxvQkFBb0IsTUFBTTs7OztJQUlqRixVQUFVLGlCQUFpQixXQUFBO0VDckIxQixPRHNCQTtJQUFBLFNBQVM7SUFDVCxPQUNFO01BQUEsT0FBTzs7SUFFVCxVQUFVOzs7QUNsQlo7QUNwQkEsUUFBUSxPQUFPLFlBRWQsT0FBTyxvREFBNEIsU0FBQyxxQkFBRDtFQUNsQyxJQUFBO0VBQUEsaUNBQWlDLFNBQUMsT0FBTyxRQUFRLGdCQUFoQjtJQUMvQixJQUFjLE9BQU8sVUFBUyxlQUFlLFVBQVMsTUFBdEQ7TUFBQSxPQUFPOztJQ2hCUCxPRGtCQSxPQUFPLFNBQVMsT0FBTyxRQUFRLE9BQU8sZ0JBQWdCO01BQUUsTUFBTTs7O0VBRWhFLCtCQUErQixZQUFZLG9CQUFvQjtFQ2YvRCxPRGlCQTs7QUNmRjtBQ0tBLFFBQVEsT0FBTyxZQUVkLFdBQVcsNkVBQXlCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDbkMsT0FBTyxjQUFjLFdBQUE7SUNuQm5CLE9Eb0JBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87SUFJUixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ3JDLE9BQU8sY0FBYyxXQUFBO0lDdEJuQixPRHVCQSxPQUFPLE9BQU8sWUFBWSxRQUFROztFQUVwQyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUN0QnJCLE9EdUJBLFlBQVksbUJBQW1CLE9BQU87O0VDckJ4QyxPRHVCQSxPQUFPO0lBSVIsV0FBVyx5RkFBdUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUFhLFlBQTVDO0VBQ2pDLE9BQU8sUUFBUSxhQUFhO0VBQzVCLFdBQVcsTUFBTTtFQUVqQixZQUFZLFFBQVEsYUFBYSxPQUFPLEtBQUssU0FBQyxNQUFEO0lDMUIzQyxPRDJCQSxXQUFXLE1BQU07O0VDekJuQixPRDJCQSxPQUFPLElBQUksWUFBWSxXQUFBO0lDMUJyQixPRDJCQSxXQUFXLE1BQU07O0lBSXBCLFdBQVcseUVBQXFCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUM1Qi9CLE9ENkJBLFlBQVksU0FBUyxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUM1QjVDLE9ENkJBLE9BQU8sT0FBTzs7SUFJakIsV0FBVyw2RUFBeUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUNuQyxPQUFPLFNBQVMsYUFBYTtFQUM3QixPQUFPLFlBQVksWUFBWTtFQzlCL0IsT0RnQ0EsWUFBWSxRQUFRLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtJQy9CdEMsT0RnQ0EsT0FBTyxPQUFPOztJQUlqQixXQUFXLG1GQUErQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VDakN6QyxPRGtDQSxZQUFZLFVBQVUsYUFBYSxPQUFPLGFBQWEsVUFBVSxLQUFLLFNBQUMsTUFBRDtJQ2pDcEUsT0RrQ0EsT0FBTyxTQUFTOzs7QUMvQnBCO0FDeEJBLFFBQVEsT0FBTyxZQUlkLFVBQVUscUJBQVUsU0FBQyxRQUFEO0VDckJuQixPRHNCQTtJQUFBLFVBQVU7SUFFVixPQUNFO01BQUEsTUFBTTs7SUFFUixNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLGFBQUEsWUFBQSxPQUFBO01BQUEsT0FBTyxHQUFHLFNBQVM7TUFDbkIsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUyxhQUFhO01BRWxELGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxNQUFBLE9BQUEsS0FBQSxNQUFBO1FBQUEsV0FBVztRQUVYLFFBQVEsUUFBUSxLQUFLLFlBQVksY0FBYyxTQUFDLFFBQVEsR0FBVDtVQUM3QyxJQUFBO1VBQUEsUUFBUSxLQUFLLGFBQWEsT0FBTztVQ3JCL0IsT0R1QkYsU0FBUyxLQUFLO1lBQ1osT0FBVSxPQUFPLHFCQUFtQixPQUFJLElBQUU7WUFDMUMsT0FBTztjQUNMO2dCQUNFLE9BQU87Z0JBQ1AsT0FBTztnQkFDUCxlQUFlLE1BQU0sZUFBZTtnQkFDcEMsYUFBYSxNQUFNLGVBQWU7aUJBRXBDO2dCQUNFLE9BQU87Z0JBQ1AsT0FBTztnQkFDUCxlQUFlLE1BQU0sZUFBZTtnQkFDcEMsYUFBYSxNQUFNLGFBQWE7aUJBRWxDO2dCQUNFLE9BQU87Z0JBQ1AsT0FBTztnQkFDUCxlQUFlLE1BQU0sYUFBYTtnQkFDbEMsYUFBYSxNQUFNLGNBQWM7Ozs7O1FBS3pDLFFBQVEsR0FBRyxXQUFXLFFBQVEsV0FBVztVQUN2QyxRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLGNBQWM7VUFDZCxVQUFVO1dBQ1QsWUFBWSxTQUFDLE9BQUQ7VUN6QlgsT0QwQkY7V0FDQSxPQUFPO1VBQUUsTUFBTTtVQUFLLE9BQU87VUFBRyxLQUFLO1VBQUcsUUFBUTs7UUFFaEQsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSyxPQUNMLEtBQUs7UUFFTixPQUFPLElBQUksT0FBTztRQUVsQixLQUFLLEdBQUcsUUFBUSxXQUFBO1VBQ2QsSUFBQTtVQUFBLEtBQUssR0FBRztVQUVSLEtBQUssVUFBVSxRQUFRLEtBQUssYUFBYSxlQUFlLEdBQUcsVUFBVSxLQUFLLGVBQWUsR0FBRyxRQUFRO1VDMUJsRyxPRDJCRixLQUFLLFVBQVUsUUFBUSxLQUFLLGFBQWEsZUFBZSxHQUFHLFVBQVUsS0FBSyxlQUFlLEdBQUcsUUFBUTs7UUFHdEcsT0FBTyxLQUFLLEdBQUcsR0FBRztRQzNCaEIsT0Q0QkYsSUFBSSxLQUFLLFVBQVUsS0FBSyxTQUFTOztNQUVuQyxZQUFZLE1BQU07OztJQU1yQixVQUFVLHVCQUFZLFNBQUMsUUFBRDtFQzlCckIsT0QrQkE7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLEtBQUs7O0lBRVAsTUFBTSxTQUFDLE9BQU8sTUFBTSxPQUFkO01BQ0osSUFBQSxhQUFBLFlBQUEsT0FBQTtNQUFBLE9BQU8sR0FBRyxTQUFTO01BQ25CLFFBQVEsS0FBSyxXQUFXO01BRXhCLGFBQWEsS0FBSztNQUNsQixRQUFRLFFBQVEsT0FBTyxLQUFLLFNBQVMsYUFBYTtNQUVsRCxjQUFjLFNBQUMsTUFBRDtRQUNaLElBQUEsTUFBQSxPQUFBLEtBQUEsTUFBQTtRQUFBLFdBQVc7UUFFWCxRQUFRLFFBQVEsS0FBSyxLQUFLLGVBQWUsU0FBQyxRQUFEO1VBQ3ZDLElBQUE7VUFBQSxRQUFRLEtBQUssS0FBSyxrQkFBa0IsT0FBTztVQzlCekMsT0RrQ0YsU0FBUyxLQUNQO1lBQUEsT0FBTztjQUNMO2dCQUFBLE9BQU8sT0FBTztnQkFDZCxPQUFPO2dCQUNQLGVBQWUsTUFBTTtnQkFDckIsYUFBYSxNQUFNO2dCQUNuQixNQUFNLE9BQU87Ozs7O1FBR25CLFFBQVEsR0FBRyxXQUFXLFFBQVEsTUFBTSxTQUFDLEdBQUcsR0FBRyxPQUFQO1VDOUJoQyxPRCtCRixPQUFPLEdBQUcsOEJBQThCO1lBQUUsT0FBTyxLQUFLO1lBQUssVUFBVSxFQUFFOztXQUV2RSxXQUFXO1VBQ1gsUUFBUSxHQUFHLEtBQUssT0FBTztVQUV2QixjQUFjO1VBQ2QsVUFBVTtXQUNULE9BQU87VUFBRSxNQUFNO1VBQUcsT0FBTztVQUFHLEtBQUs7VUFBRyxRQUFROztRQUUvQyxNQUFNLEdBQUcsT0FBTyxPQUNmLE1BQU0sVUFDTixLQUFLLE9BQ0wsS0FBSztRQUVOLE9BQU8sSUFBSSxPQUFPO1FBRWxCLEtBQUssR0FBRyxRQUFRLFdBQUE7VUFDZCxJQUFBO1VBQUEsS0FBSyxHQUFHO1VBRVIsS0FBSyxVQUFVLFFBQVEsS0FBSyxhQUFhLGVBQWUsR0FBRyxVQUFVLEtBQUssZUFBZSxHQUFHLFFBQVE7VUM5QmxHLE9EK0JGLEtBQUssVUFBVSxRQUFRLEtBQUssYUFBYSxlQUFlLEdBQUcsVUFBVSxLQUFLLGVBQWUsR0FBRyxRQUFROztRQUd0RyxPQUFPLEtBQUssR0FBRyxHQUFHO1FDL0JoQixPRGdDRixJQUFJLEtBQUssVUFBVSxLQUFLLFNBQVM7O01BRW5DLE1BQU0sT0FBTyxNQUFNLEtBQUssU0FBQyxNQUFEO1FBQ3RCLElBQXFCLE1BQXJCO1VDL0JJLE9EK0JKLFlBQVk7Ozs7O0lBTWpCLFVBQVUsd0JBQVcsU0FBQyxVQUFEO0VDL0JwQixPRGdDQTtJQUFBLFVBQVU7SUFRVixPQUNFO01BQUEsTUFBTTs7SUFFUixNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLFlBQUEsWUFBQSxpQkFBQSxpQkFBQSxZQUFBLFdBQUEsWUFBQSxVQUFBLFdBQUEsNkJBQUEsYUFBQSx3QkFBQSxPQUFBLGlCQUFBLE9BQUEsZ0JBQUEsZ0JBQUEsVUFBQSxlQUFBLGVBQUE7TUFBQSxXQUFXLEdBQUcsU0FBUztNQUN2QixZQUFZO01BQ1osUUFBUSxNQUFNO01BRWQsaUJBQWlCLEtBQUssV0FBVztNQUNqQyxRQUFRLEtBQUssV0FBVyxXQUFXO01BQ25DLGlCQUFpQixLQUFLLFdBQVc7TUFFakMsWUFBWSxHQUFHLE9BQU87TUFDdEIsYUFBYSxHQUFHLE9BQU87TUFDdkIsV0FBVyxHQUFHLE9BQU87TUFJckIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxLQUFLLFdBQVcsSUFBSSxNQUFNO01BRTFDLE1BQU0sU0FBUyxXQUFBO1FBQ2IsSUFBQSxXQUFBLElBQUE7UUFBQSxJQUFHLFNBQVMsVUFBVSxNQUF0QjtVQUdFLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxTQUFTLFVBQVUsQ0FBRSxJQUFJO1VDM0N2QixPRDhDRixXQUFXLEtBQUssYUFBYSxlQUFlLEtBQUssTUFBTSxLQUFLLGFBQWEsU0FBUyxVQUFVOzs7TUFFaEcsTUFBTSxVQUFVLFdBQUE7UUFDZCxJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxZQUFZLFNBQVM7VUFDckIsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELFNBQVMsVUFBVSxDQUFFLElBQUk7VUM3Q3ZCLE9EZ0RGLFdBQVcsS0FBSyxhQUFhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUdoRyxrQkFBa0IsU0FBQyxJQUFEO1FBQ2hCLElBQUE7UUFBQSxhQUFhO1FBQ2IsSUFBRyxDQUFBLEdBQUEsaUJBQUEsVUFBcUIsR0FBQSxrQkFBQSxPQUF4QjtVQUNFLGNBQWM7VUFDZCxJQUFtQyxHQUFBLGlCQUFBLE1BQW5DO1lBQUEsY0FBYyxHQUFHOztVQUNqQixJQUFnRCxHQUFHLGNBQWEsV0FBaEU7WUFBQSxjQUFjLE9BQU8sR0FBRyxZQUFZOztVQUNwQyxJQUFrRCxHQUFHLG1CQUFrQixXQUF2RTtZQUFBLGNBQWMsVUFBVSxHQUFHOztVQUMzQixjQUFjOztRQ3ZDZCxPRHdDRjs7TUFJRix5QkFBeUIsU0FBQyxNQUFEO1FDekNyQixPRDBDRCxTQUFRLHFCQUFxQixTQUFRLHlCQUF5QixTQUFRLGFBQWEsU0FBUSxpQkFBaUIsU0FBUSxpQkFBaUIsU0FBUTs7TUFFaEosY0FBYyxTQUFDLElBQUksTUFBTDtRQUNaLElBQUcsU0FBUSxVQUFYO1VDekNJLE9EMENGO2VBRUcsSUFBRyx1QkFBdUIsT0FBMUI7VUMxQ0QsT0QyQ0Y7ZUFERztVQUlILElBQUcsR0FBRyxTQUFRLGVBQWQ7WUMzQ0ksT0Q0Q0Y7aUJBQ0csSUFBRyxHQUFHLFNBQVEsYUFBZDtZQzNDRCxPRDRDRjtpQkFERztZQ3pDRCxPRDRDRjs7OztNQUdOLGtCQUFrQixTQUFDLElBQUksTUFBTSxNQUFNLE1BQWpCO1FBQ2hCLElBQUEsWUFBQTtRQUFBLGFBQWEscUJBQXFCLFFBQVEsTUFBTSxHQUFHLEtBQUsseUJBQXlCLFlBQVksSUFBSSxRQUFRO1FBR3pHLElBQUcsU0FBUSxVQUFYO1VBQ0UsY0FBYyxxQ0FBcUMsR0FBRyxPQUFPO2VBRC9EO1VBR0UsY0FBYywyQkFBMkIsR0FBRyxPQUFPOztRQUNyRCxJQUFHLEdBQUcsYUFBWSxJQUFsQjtVQUNFLGNBQWM7ZUFEaEI7VUFHRSxXQUFXLEdBQUc7VUFHZCxXQUFXLGNBQWM7VUFDekIsY0FBYywyQkFBMkIsV0FBVzs7UUFHdEQsSUFBRyxHQUFBLGlCQUFBLE1BQUg7VUFDRSxjQUFjLDRCQUE0QixHQUFHLElBQUksTUFBTTtlQUR6RDtVQUtFLElBQStDLHVCQUF1QixPQUF0RTtZQUFBLGNBQWMsU0FBUyxPQUFPOztVQUM5QixJQUFxRSxHQUFHLGdCQUFlLElBQXZGO1lBQUEsY0FBYyxzQkFBc0IsR0FBRyxjQUFjOztVQUNyRCxJQUEyRixHQUFHLG9CQUFtQixXQUFqSDtZQUFBLGNBQWMsMEJBQTBCLGNBQWMsR0FBRyxtQkFBbUI7OztRQUU5RSxjQUFjO1FDekNaLE9EMENGOztNQUdGLDhCQUE4QixTQUFDLElBQUksTUFBTSxNQUFYO1FBQzVCLElBQUEsWUFBQTtRQUFBLFFBQVEsU0FBUztRQUVqQixhQUFhLGlCQUFpQixRQUFRLGFBQWEsT0FBTyxhQUFhLE9BQU87UUMxQzVFLE9EMkNGOztNQUdGLGdCQUFnQixTQUFDLEdBQUQ7UUFFZCxJQUFBO1FBQUEsSUFBRyxFQUFFLE9BQU8sT0FBTSxLQUFsQjtVQUNFLElBQUksRUFBRSxRQUFRLEtBQUs7VUFDbkIsSUFBSSxFQUFFLFFBQVEsS0FBSzs7UUFDckIsTUFBTTtRQUNOLE9BQU0sRUFBRSxTQUFTLElBQWpCO1VBQ0UsTUFBTSxNQUFNLEVBQUUsVUFBVSxHQUFHLE1BQU07VUFDakMsSUFBSSxFQUFFLFVBQVUsSUFBSSxFQUFFOztRQUN4QixNQUFNLE1BQU07UUN6Q1YsT0QwQ0Y7O01BRUYsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLFVBQWtCLE1BQU0sTUFBdEM7UUN6Q1QsSUFBSSxZQUFZLE1BQU07VUR5Q0MsV0FBVzs7UUFFcEMsSUFBRyxHQUFHLE9BQU0sS0FBSyxrQkFBakI7VUN2Q0ksT0R3Q0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksbUJBQW1CLE1BQU07WUFDcEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLHVCQUFqQjtVQ3ZDRCxPRHdDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSx1QkFBdUIsTUFBTTtZQUN4RCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssU0FBakI7VUN2Q0QsT0R3Q0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksV0FBVyxNQUFNO1lBQzVDLFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3ZDRCxPRHdDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGNBQWpCO1VDdkNELE9Ed0NGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGVBQWUsTUFBTTtZQUNoRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssZ0JBQWpCO1VDdkNELE9Ed0NGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGlCQUFpQixNQUFNO1lBQ2xELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFKdEI7VUNqQ0QsT0R3Q0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksSUFBSSxNQUFNO1lBQ3JDLFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7OztNQUU3QixhQUFhLFNBQUMsR0FBRyxNQUFNLElBQUksZUFBZSxNQUE3QjtRQUNYLElBQUE7UUFBQSxJQUFPLGNBQWMsUUFBUSxLQUFLLFFBQU8sQ0FBQyxHQUExQztVQ3BDSSxPRHFDRixFQUFFLFFBQVEsS0FBSyxJQUFJLEdBQUcsSUFDcEI7WUFBQSxPQUFPLGdCQUFnQjtZQUN2QixXQUFXO1lBQ1gsV0FBVzs7ZUFKZjtVQU9FLGNBQWMsY0FBYyxNQUFNLEtBQUs7VUFDdkMsSUFBQSxFQUFPLENBQUMsZUFBZSxZQUFZLGlCQUFnQixPQUFuRDtZQUNFLFlBQVksZUFBZTtZQUMzQixFQUFFLFFBQVEsWUFBWSxJQUNwQjtjQUFBLE9BQU8sZ0JBQWdCLGFBQWE7Y0FDcEMsV0FBVztjQUNYLFNBQU8sWUFBWSxhQUFhOztZQ25DaEMsT0RxQ0YsRUFBRSxRQUFRLFlBQVksSUFBSSxHQUFHLElBQzNCO2NBQUEsT0FBTyxnQkFBZ0I7Y0FDdkIsV0FBVzs7Ozs7TUFFbkIsa0JBQWtCLFNBQUMsR0FBRyxNQUFKO1FBQ2hCLElBQUEsSUFBQSxlQUFBLFVBQUEsR0FBQSxHQUFBLEtBQUEsTUFBQSxNQUFBLE1BQUEsTUFBQSxHQUFBLEtBQUEsSUFBQTtRQUFBLGdCQUFnQjtRQUVoQixJQUFHLEtBQUEsU0FBQSxNQUFIO1VBRUUsWUFBWSxLQUFLO2VBRm5CO1VBTUUsWUFBWSxLQUFLO1VBQ2pCLFdBQVc7O1FBRWIsS0FBQSxJQUFBLEdBQUEsTUFBQSxVQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7VUNwQ0ksS0FBSyxVQUFVO1VEcUNqQixPQUFPO1VBQ1AsT0FBTztVQUVQLElBQUcsR0FBRyxlQUFOO1lBQ0UsS0FBUyxJQUFBLFFBQVEsU0FBUyxNQUFNO2NBQUUsWUFBWTtjQUFNLFVBQVU7ZUFBUSxTQUFTO2NBQzdFLFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUzs7WUFHWCxVQUFVLEdBQUcsTUFBTTtZQUVuQixnQkFBZ0IsSUFBSTtZQUVwQixJQUFRLElBQUEsUUFBUTtZQUNoQixTQUFTLE9BQU8sS0FBSyxLQUFLLEdBQUc7WUFDN0IsT0FBTyxHQUFHLFFBQVE7WUFDbEIsT0FBTyxHQUFHLFFBQVE7WUFFbEIsUUFBUSxRQUFRLGdCQUFnQjs7VUFFbEMsV0FBVyxHQUFHLE1BQU0sSUFBSSxVQUFVLE1BQU07VUFFeEMsY0FBYyxLQUFLLEdBQUc7VUFHdEIsSUFBRyxHQUFBLGdCQUFBLE1BQUg7WUFDRSxNQUFBLEdBQUE7WUFBQSxLQUFBLElBQUEsR0FBQSxPQUFBLElBQUEsUUFBQSxJQUFBLE1BQUEsS0FBQTtjQ3ZDSSxPQUFPLElBQUk7Y0R3Q2IsV0FBVyxHQUFHLE1BQU0sSUFBSSxlQUFlOzs7O1FDbkMzQyxPRHFDRjs7TUFHRixnQkFBZ0IsU0FBQyxNQUFNLFFBQVA7UUFDZCxJQUFBLElBQUEsR0FBQTtRQUFBLEtBQUEsS0FBQSxLQUFBLE9BQUE7VUFDRSxLQUFLLEtBQUssTUFBTTtVQUNoQixJQUFjLEdBQUcsT0FBTSxRQUF2QjtZQUFBLE9BQU87O1VBR1AsSUFBRyxHQUFBLGlCQUFBLE1BQUg7WUFDRSxLQUFBLEtBQUEsR0FBQSxlQUFBO2NBQ0UsSUFBK0IsR0FBRyxjQUFjLEdBQUcsT0FBTSxRQUF6RDtnQkFBQSxPQUFPLEdBQUcsY0FBYzs7Ozs7O01BRWhDLFlBQVksU0FBQyxNQUFEO1FBQ1YsSUFBQSxHQUFBLEdBQUEsVUFBQSxVQUFBLElBQUEsZUFBQTtRQUFBLElBQVEsSUFBQSxRQUFRLFNBQVMsTUFBTTtVQUFFLFlBQVk7VUFBTSxVQUFVO1dBQVEsU0FBUztVQUM1RSxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7O1FBR1gsZ0JBQWdCLEdBQUc7UUFFbkIsV0FBZSxJQUFBLFFBQVE7UUFDdkIsV0FBVyxLQUFLLFVBQVU7UUFFMUIsS0FBQSxLQUFBLFdBQUE7VUM5QkksS0FBSyxVQUFVO1VEK0JqQixVQUFVLE9BQU8sYUFBYSxJQUFJLE1BQU0sS0FBSyxVQUFVOztRQUV6RCxXQUFXO1FBRVgsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsVUFBVSxFQUFFLFFBQVEsUUFBUSxZQUFZO1FBQ3BHLGdCQUFnQixLQUFLLE1BQU0sQ0FBQyxRQUFRLFFBQVEsZ0JBQWdCLFdBQVcsRUFBRSxRQUFRLFNBQVMsWUFBWTtRQUV0RyxTQUFTLE1BQU0sVUFBVSxVQUFVLENBQUMsZUFBZTtRQUVuRCxXQUFXLEtBQUssYUFBYSxlQUFlLGdCQUFnQixPQUFPLGdCQUFnQixhQUFhLFNBQVMsVUFBVTtRQUVuSCxTQUFTLEdBQUcsUUFBUSxXQUFBO1VBQ2xCLElBQUE7VUFBQSxLQUFLLEdBQUc7VUNoQ04sT0RpQ0YsV0FBVyxLQUFLLGFBQWEsZUFBZSxHQUFHLFlBQVksYUFBYSxHQUFHLFFBQVE7O1FDL0JuRixPRGlDRixTQUFTOztNQUVYLE1BQU0sT0FBTyxNQUFNLE1BQU0sU0FBQyxTQUFEO1FBQ3ZCLElBQXNCLFNBQXRCO1VDaENJLE9EZ0NKLFVBQVU7Ozs7OztBQzFCaEI7QUM1WkEsUUFBUSxPQUFPLFlBRWQsUUFBUSw4RUFBZSxTQUFDLE9BQU8sYUFBYSxNQUFNLFVBQVUsSUFBSSxVQUF6QztFQUN0QixJQUFBLFlBQUEsYUFBQSxXQUFBLGNBQUEsTUFBQTtFQUFBLGFBQWE7RUFDYixjQUFjO0VBQ2QsWUFBWTtFQUNaLE9BQU87SUFDTCxTQUFTO0lBQ1QsVUFBVTtJQUNWLFdBQVc7SUFDWCxRQUFROztFQUdWLGVBQWU7RUFFZixrQkFBa0IsV0FBQTtJQ3BCaEIsT0RxQkEsUUFBUSxRQUFRLGNBQWMsU0FBQyxVQUFEO01DcEI1QixPRHFCQTs7O0VBRUosS0FBQyxtQkFBbUIsU0FBQyxVQUFEO0lDbkJsQixPRG9CQSxhQUFhLEtBQUs7O0VBRXBCLEtBQUMscUJBQXFCLFNBQUMsVUFBRDtJQUNwQixJQUFBO0lBQUEsUUFBUSxhQUFhLFFBQVE7SUNsQjdCLE9EbUJBLGFBQWEsT0FBTyxPQUFPOztFQUU3QixLQUFDLFlBQVksV0FBQTtJQ2xCWCxPRG1CQSxDQUVFLGFBQ0EsYUFDQSxXQUNBLFlBQ0EsVUFDQSxhQUNBOztFQUdKLEtBQUMsc0JBQXNCLFNBQUMsT0FBRDtJQUNyQixRQUFPLE1BQU07TUFBYixLQUNPO1FDM0JILE9EMkJtQjtNQUR2QixLQUVPO1FDMUJILE9EMEJpQjtNQUZyQixLQUdPO1FDekJILE9EeUJvQjtNQUh4QixLQUlPO1FDeEJILE9Ed0JvQjtNQUp4QixLQUtPO1FDdkJILE9EdUJrQjtNQUx0QixLQU1PO1FDdEJILE9Ec0JvQjtNQU54QixLQU9PO1FDckJILE9EcUJrQjtNQVB0QixLQVFPO1FDcEJILE9Eb0JnQjtNQVJwQjtRQ1ZJLE9EbUJHOzs7RUFFVCxLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksWUFBWSxZQUFZLFNBQ2pDLFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUVQLFFBQVEsUUFBUSxNQUFNLFNBQUMsTUFBTSxTQUFQO1FBRXBCLFFBQU87VUFBUCxLQUNPO1lBQW9CLEtBQUssVUFBVTtZQUFuQztVQURQLEtBRU87WUFBcUIsS0FBSyxXQUFXO1lBQXJDO1VBRlAsS0FHTztZQUFzQixLQUFLLFlBQVk7WUFBdkM7VUFIUCxLQUlPO1lBQW1CLEtBQUssU0FBUzs7UUNaeEMsT0RjQSxRQUFRLFFBQVEsTUFBTSxTQUFDLE9BQU8sT0FBUjtVQ2JwQixPRGNBLE1BQU0sSUFBSSxZQUFZLFlBQVksV0FBVyxPQUM1QyxRQUFRLFNBQUMsU0FBRDtZQ2RQLE9EZUEsS0FBSyxTQUFTOzs7O01BRXBCLFNBQVMsUUFBUTtNQ1pqQixPRGFBOztJQ1hGLE9EYUEsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDWlQsT0RhQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDWlosT0RhQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lBRW5CLE1BQU0sSUFBSSxZQUFZLFlBQVksV0FBVyxPQUM1QyxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUFDUCxLQUFLLE9BQU8sS0FBSztNQ2RqQixPRGdCQSxNQUFNLElBQUksWUFBWSxZQUFZLFdBQVcsUUFBUSxhQUNwRCxRQUFRLFNBQUMsVUFBRDtRQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07UUNoQjVCLE9Ea0JBLE1BQU0sSUFBSSxZQUFZLFlBQVksMkJBQTJCLE9BQzVELFFBQVEsU0FBQyxhQUFEO1VBQ1AsS0FBSyxPQUFPLFlBQVk7VUFFeEIsYUFBYTtVQ25CYixPRG9CQSxVQUFVLElBQUksUUFBUTs7OztJQ2hCNUIsT0RrQkEsVUFBVSxJQUFJOztFQUVoQixLQUFDLFdBQVcsU0FBQyxPQUFEO0lBQ1YsY0FBYztJQUNkLFVBQVUsT0FBTyxHQUFHO0lBRXBCLE1BQU0sSUFBSSxZQUFZLFlBQVksV0FBVyxRQUFRLFNBQ3BELFFBQVEsU0FBQyxNQUFEO01BQ1AsY0FBYztNQ25CZCxPRHFCQSxVQUFVLEtBQUssUUFBUTs7SUNuQnpCLE9EcUJBLFVBQVUsS0FBSzs7RUFFakIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsU0FBUyxTQUFTO01BRWxCLEtBQUEsSUFBQSxHQUFBLE1BQUEsS0FBQSxRQUFBLElBQUEsS0FBQSxLQUFBO1FDbkJFLE9BQU8sS0FBSztRRG9CWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ1hULE9EYUE7O0lBRUYsV0FBVyxHQUFHO0lBVWQsR0FBRyxJQUFJLENBQUMsVUFBVSxLQUFLLFNBQVMsVUFBVSxJQUFJLFVBQVUsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ3JCM0QsT0RxQjJELFNBQUMsTUFBRDtRQUMzRCxJQUFBO1FBQUEsWUFBWSxTQUFTLFFBQVEsWUFBWTtRQ25CdkMsT0RzQkYsTUFBQyxVQUFVLFdBQVcsS0FBSyxXQUFXLEtBQUssY0FBYyxHQUFHLGVBQWUsS0FBSyxTQUFDLFFBQUQ7VUFDOUUsVUFBVSxTQUFTO1VDckJqQixPRHNCRixTQUFTLFFBQVE7OztPQU53QztJQ1o3RCxPRG9CQSxTQUFTOztFQUdYLEtBQUMsWUFBWSxTQUFDLE9BQU8sVUFBUjtJQUNYLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksWUFBWSxZQUFZLG1DQUFtQyxRQUFRLGtCQUFrQixVQUM5RixRQUFRLFNBQUMsTUFBRDtNQ3JCUCxPRHNCQSxTQUFTLFFBQVE7O0lDcEJuQixPRHNCQSxTQUFTOztFQ3BCWCxPRHNCQTs7QUNwQkY7QUN0SUEsUUFBUSxPQUFPLFlBRWQsV0FBVyxtRUFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUExQjtFQUNoQyxPQUFPLGNBQWMsV0FBQTtJQUNuQixPQUFPLGNBQWMsWUFBWSxRQUFRO0lDbkJ6QyxPRG9CQSxPQUFPLGVBQWUsWUFBWSxRQUFROztFQUU1QyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFlBQVksbUJBQW1CLE9BQU87O0VDbEJ4QyxPRG9CQSxPQUFPOztBQ2xCVDtBQ09BLFFBQVEsT0FBTyxZQUVkLFFBQVEsb0RBQW1CLFNBQUMsT0FBTyxhQUFhLE1BQXJCO0VBQzFCLElBQUE7RUFBQSxlQUFlO0VBRWYsS0FBQyxtQkFBbUIsV0FBQTtJQUNsQixNQUFNLElBQUksWUFBWSxZQUFZLG1CQUNqQyxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUNwQlAsT0RxQkEsS0FBSztPQUVOLE1BQU0sU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtJQ3JCUCxPRHdCQTs7RUN0QkYsT0R3QkE7O0FDdEJGIiwiZmlsZSI6ImluZGV4LmpzIiwic291cmNlc0NvbnRlbnQiOlsiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSlcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4ucnVuICgkcm9vdFNjb3BlKSAtPlxuICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gZmFsc2VcbiAgJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IC0+XG4gICAgJHJvb3RTY29wZS5zaWRlYmFyVmlzaWJsZSA9ICEkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlXG4gICAgJHJvb3RTY29wZS5zaWRlYmFyQ2xhc3MgPSAnZm9yY2Utc2hvdydcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29uc3RhbnQgJ2ZsaW5rQ29uZmlnJywge1xuICBqb2JTZXJ2ZXI6ICdodHRwOi8vbG9jYWxob3N0OjgwODEnXG4gIG5ld1NlcnZlcjogJ2h0dHA6Ly9sb2NhbGhvc3Q6ODA4MSdcbiMgIGpvYlNlcnZlcjogJ2h0dHA6Ly9sb2NhbGhvc3Q6MzAwMC9uZXctc2VydmVyJ1xuIyAgbmV3U2VydmVyOiAnaHR0cDovL2xvY2FsaG9zdDozMDAwL25ldy1zZXJ2ZXInXG4gIHJlZnJlc2hJbnRlcnZhbDogMTAwMDBcbn1cblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4ucnVuIChKb2JzU2VydmljZSwgZmxpbmtDb25maWcsICRpbnRlcnZhbCkgLT5cbiAgSm9ic1NlcnZpY2UubGlzdEpvYnMoKVxuXG4gICRpbnRlcnZhbCAtPlxuICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcbiAgLCBmbGlua0NvbmZpZy5yZWZyZXNoSW50ZXJ2YWxcblxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5jb25maWcgKCRzdGF0ZVByb3ZpZGVyLCAkdXJsUm91dGVyUHJvdmlkZXIpIC0+XG4gICRzdGF0ZVByb3ZpZGVyLnN0YXRlIFwib3ZlcnZpZXdcIixcbiAgICB1cmw6IFwiL292ZXJ2aWV3XCJcbiAgICB2aWV3czpcbiAgICAgIG1haW46XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnT3ZlcnZpZXdDb250cm9sbGVyJ1xuXG4gIC5zdGF0ZSBcInJ1bm5pbmctam9ic1wiLFxuICAgIHVybDogXCIvcnVubmluZy1qb2JzXCJcbiAgICB2aWV3czpcbiAgICAgIG1haW46XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvcnVubmluZy1qb2JzLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnUnVubmluZ0pvYnNDb250cm9sbGVyJ1xuICBcbiAgLnN0YXRlIFwiY29tcGxldGVkLWpvYnNcIixcbiAgICB1cmw6IFwiL2NvbXBsZXRlZC1qb2JzXCJcbiAgICB2aWV3czpcbiAgICAgIG1haW46XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvY29tcGxldGVkLWpvYnMuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iXCIsXG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIlxuICAgIGFic3RyYWN0OiB0cnVlXG4gICAgdmlld3M6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ1NpbmdsZUpvYkNvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuXCIsXG4gICAgdXJsOiBcIlwiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkNvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLm5vZGVcIixcbiAgICB1cmw6IFwiL3tub2RlaWQ6aW50fVwiXG4gICAgdmlld3M6XG4gICAgICBub2RlOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLm5vZGUuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuTm9kZUNvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi50aW1lbGluZVwiLFxuICAgIHVybDogXCIvdGltZWxpbmVcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUuaHRtbFwiXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi50aW1lbGluZS52ZXJ0ZXhcIixcbiAgICB1cmw6IFwiL3t2ZXJ0ZXhJZH1cIlxuICAgIHZpZXdzOlxuICAgICAgdmVydGV4OlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi50aW1lbGluZS52ZXJ0ZXguaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5zdGF0aXN0aWNzXCIsXG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCJcbiAgICB2aWV3czpcbiAgICAgIGRldGFpbHM6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnN0YXRpc3RpY3MuaHRtbFwiXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5leGNlcHRpb25zXCIsXG4gICAgdXJsOiBcIi9leGNlcHRpb25zXCJcbiAgICB2aWV3czpcbiAgICAgIGRldGFpbHM6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXG5cbiAgJHVybFJvdXRlclByb3ZpZGVyLm90aGVyd2lzZSBcIi9vdmVydmlld1wiXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnLCBbJ3VpLnJvdXRlcicsICdhbmd1bGFyTW9tZW50J10pLnJ1bihmdW5jdGlvbigkcm9vdFNjb3BlKSB7XG4gICRyb290U2NvcGUuc2lkZWJhclZpc2libGUgPSBmYWxzZTtcbiAgcmV0dXJuICRyb290U2NvcGUuc2hvd1NpZGViYXIgPSBmdW5jdGlvbigpIHtcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGU7XG4gICAgcmV0dXJuICRyb290U2NvcGUuc2lkZWJhckNsYXNzID0gJ2ZvcmNlLXNob3cnO1xuICB9O1xufSkuY29uc3RhbnQoJ2ZsaW5rQ29uZmlnJywge1xuICBqb2JTZXJ2ZXI6ICdodHRwOi8vbG9jYWxob3N0OjgwODEnLFxuICBuZXdTZXJ2ZXI6ICdodHRwOi8vbG9jYWxob3N0OjgwODEnLFxuICByZWZyZXNoSW50ZXJ2YWw6IDEwMDAwXG59KS5ydW4oZnVuY3Rpb24oSm9ic1NlcnZpY2UsIGZsaW5rQ29uZmlnLCAkaW50ZXJ2YWwpIHtcbiAgSm9ic1NlcnZpY2UubGlzdEpvYnMoKTtcbiAgcmV0dXJuICRpbnRlcnZhbChmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gSm9ic1NlcnZpY2UubGlzdEpvYnMoKTtcbiAgfSwgZmxpbmtDb25maWcucmVmcmVzaEludGVydmFsKTtcbn0pLmNvbmZpZyhmdW5jdGlvbigkc3RhdGVQcm92aWRlciwgJHVybFJvdXRlclByb3ZpZGVyKSB7XG4gICRzdGF0ZVByb3ZpZGVyLnN0YXRlKFwib3ZlcnZpZXdcIiwge1xuICAgIHVybDogXCIvb3ZlcnZpZXdcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9vdmVydmlldy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInJ1bm5pbmctam9ic1wiLCB7XG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL3J1bm5pbmctam9icy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdSdW5uaW5nSm9ic0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcImNvbXBsZXRlZC1qb2JzXCIsIHtcbiAgICB1cmw6IFwiL2NvbXBsZXRlZC1qb2JzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIG1haW46IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9jb21wbGV0ZWQtam9icy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYlwiLCB7XG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIixcbiAgICBhYnN0cmFjdDogdHJ1ZSxcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW5cIiwge1xuICAgIHVybDogXCJcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW4ubm9kZVwiLCB7XG4gICAgdXJsOiBcIi97bm9kZWlkOmludH1cIixcbiAgICB2aWV3czoge1xuICAgICAgbm9kZToge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLm5vZGUuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbk5vZGVDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsIHtcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IudGltZWxpbmUudmVydGV4XCIsIHtcbiAgICB1cmw6IFwiL3t2ZXJ0ZXhJZH1cIixcbiAgICB2aWV3czoge1xuICAgICAgdmVydGV4OiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLnZlcnRleC5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLCB7XG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2Iuc3RhdGlzdGljcy5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5leGNlcHRpb25zXCIsIHtcbiAgICB1cmw6IFwiL2V4Y2VwdGlvbnNcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5leGNlcHRpb25zLmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSk7XG4gIHJldHVybiAkdXJsUm91dGVyUHJvdmlkZXIub3RoZXJ3aXNlKFwiL292ZXJ2aWV3XCIpO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ2JzTGFiZWwnLCAoSm9ic1NlcnZpY2UpIC0+XG4gIHRyYW5zY2x1ZGU6IHRydWVcbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTogXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcbiAgICBzdGF0dXM6IFwiQFwiXG5cbiAgdGVtcGxhdGU6IFwiPHNwYW4gdGl0bGU9J3t7c3RhdHVzfX0nIG5nLWNsYXNzPSdnZXRMYWJlbENsYXNzKCknPjxuZy10cmFuc2NsdWRlPjwvbmctdHJhbnNjbHVkZT48L3NwYW4+XCJcbiAgXG4gIGxpbms6IChzY29wZSwgZWxlbWVudCwgYXR0cnMpIC0+XG4gICAgc2NvcGUuZ2V0TGFiZWxDbGFzcyA9IC0+XG4gICAgICAnbGFiZWwgbGFiZWwtJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmRpcmVjdGl2ZSAnaW5kaWNhdG9yUHJpbWFyeScsIChKb2JzU2VydmljZSkgLT5cbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTogXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcbiAgICBzdGF0dXM6ICdAJ1xuXG4gIHRlbXBsYXRlOiBcIjxpIHRpdGxlPSd7e3N0YXR1c319JyBuZy1jbGFzcz0nZ2V0TGFiZWxDbGFzcygpJyAvPlwiXG4gIFxuICBsaW5rOiAoc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSAtPlxuICAgIHNjb3BlLmdldExhYmVsQ2xhc3MgPSAtPlxuICAgICAgJ2ZhIGZhLWNpcmNsZSBpbmRpY2F0b3IgaW5kaWNhdG9yLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ3RhYmxlUHJvcGVydHknLCAtPlxuICByZXBsYWNlOiB0cnVlXG4gIHNjb3BlOlxuICAgIHZhbHVlOiAnPSdcblxuICB0ZW1wbGF0ZTogXCI8dGQgdGl0bGU9XFxcInt7dmFsdWUgfHwgJ05vbmUnfX1cXFwiPnt7dmFsdWUgfHwgJ05vbmUnfX08L3RkPlwiXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5kaXJlY3RpdmUoJ2JzTGFiZWwnLCBmdW5jdGlvbihKb2JzU2VydmljZSkge1xuICByZXR1cm4ge1xuICAgIHRyYW5zY2x1ZGU6IHRydWUsXG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgZ2V0TGFiZWxDbGFzczogXCImXCIsXG4gICAgICBzdGF0dXM6IFwiQFwiXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8c3BhbiB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKSc+PG5nLXRyYW5zY2x1ZGU+PC9uZy10cmFuc2NsdWRlPjwvc3Bhbj5cIixcbiAgICBsaW5rOiBmdW5jdGlvbihzY29wZSwgZWxlbWVudCwgYXR0cnMpIHtcbiAgICAgIHJldHVybiBzY29wZS5nZXRMYWJlbENsYXNzID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHJldHVybiAnbGFiZWwgbGFiZWwtJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKTtcbiAgICAgIH07XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCdpbmRpY2F0b3JQcmltYXJ5JywgZnVuY3Rpb24oSm9ic1NlcnZpY2UpIHtcbiAgcmV0dXJuIHtcbiAgICByZXBsYWNlOiB0cnVlLFxuICAgIHNjb3BlOiB7XG4gICAgICBnZXRMYWJlbENsYXNzOiBcIiZcIixcbiAgICAgIHN0YXR1czogJ0AnXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8aSB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKScgLz5cIixcbiAgICBsaW5rOiBmdW5jdGlvbihzY29wZSwgZWxlbWVudCwgYXR0cnMpIHtcbiAgICAgIHJldHVybiBzY29wZS5nZXRMYWJlbENsYXNzID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHJldHVybiAnZmEgZmEtY2lyY2xlIGluZGljYXRvciBpbmRpY2F0b3ItJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKTtcbiAgICAgIH07XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCd0YWJsZVByb3BlcnR5JywgZnVuY3Rpb24oKSB7XG4gIHJldHVybiB7XG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgdmFsdWU6ICc9J1xuICAgIH0sXG4gICAgdGVtcGxhdGU6IFwiPHRkIHRpdGxlPVxcXCJ7e3ZhbHVlIHx8ICdOb25lJ319XFxcIj57e3ZhbHVlIHx8ICdOb25lJ319PC90ZD5cIlxuICB9O1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLmZpbHRlciBcImFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZFwiLCAoYW5ndWxhck1vbWVudENvbmZpZykgLT5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyID0gKHZhbHVlLCBmb3JtYXQsIGR1cmF0aW9uRm9ybWF0KSAtPlxuICAgIHJldHVybiBcIlwiICBpZiB0eXBlb2YgdmFsdWUgaXMgXCJ1bmRlZmluZWRcIiBvciB2YWx1ZSBpcyBudWxsXG5cbiAgICBtb21lbnQuZHVyYXRpb24odmFsdWUsIGZvcm1hdCkuZm9ybWF0KGR1cmF0aW9uRm9ybWF0LCB7IHRyaW06IGZhbHNlIH0pXG5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyLiRzdGF0ZWZ1bCA9IGFuZ3VsYXJNb21lbnRDb25maWcuc3RhdGVmdWxGaWx0ZXJzXG5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5maWx0ZXIoXCJhbUR1cmF0aW9uRm9ybWF0RXh0ZW5kZWRcIiwgZnVuY3Rpb24oYW5ndWxhck1vbWVudENvbmZpZykge1xuICB2YXIgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyO1xuICBhbUR1cmF0aW9uRm9ybWF0RXh0ZW5kZWRGaWx0ZXIgPSBmdW5jdGlvbih2YWx1ZSwgZm9ybWF0LCBkdXJhdGlvbkZvcm1hdCkge1xuICAgIGlmICh0eXBlb2YgdmFsdWUgPT09IFwidW5kZWZpbmVkXCIgfHwgdmFsdWUgPT09IG51bGwpIHtcbiAgICAgIHJldHVybiBcIlwiO1xuICAgIH1cbiAgICByZXR1cm4gbW9tZW50LmR1cmF0aW9uKHZhbHVlLCBmb3JtYXQpLmZvcm1hdChkdXJhdGlvbkZvcm1hdCwge1xuICAgICAgdHJpbTogZmFsc2VcbiAgICB9KTtcbiAgfTtcbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyLiRzdGF0ZWZ1bCA9IGFuZ3VsYXJNb21lbnRDb25maWcuc3RhdGVmdWxGaWx0ZXJzO1xuICByZXR1cm4gYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLmNvbnRyb2xsZXIgJ1J1bm5pbmdKb2JzQ29udHJvbGxlcicsICgkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkgLT5cbiAgJHNjb3BlLmpvYk9ic2VydmVyID0gLT5cbiAgICAkc2NvcGUuam9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ3J1bm5pbmcnKVxuXG4gIEpvYnNTZXJ2aWNlLnJlZ2lzdGVyT2JzZXJ2ZXIoJHNjb3BlLmpvYk9ic2VydmVyKVxuICAkc2NvcGUuJG9uICckZGVzdHJveScsIC0+XG4gICAgSm9ic1NlcnZpY2UudW5SZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcblxuICAkc2NvcGUuam9iT2JzZXJ2ZXIoKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5jb250cm9sbGVyICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcicsICgkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkgLT5cbiAgJHNjb3BlLmpvYk9ic2VydmVyID0gLT5cbiAgICAkc2NvcGUuam9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ2ZpbmlzaGVkJylcblxuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcbiAgJHNjb3BlLiRvbiAnJGRlc3Ryb3knLCAtPlxuICAgIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpXG5cbiAgJHNjb3BlLmpvYk9ic2VydmVyKClcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29udHJvbGxlciAnU2luZ2xlSm9iQ29udHJvbGxlcicsICgkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSwgJHJvb3RTY29wZSkgLT5cbiAgJHNjb3BlLmpvYmlkID0gJHN0YXRlUGFyYW1zLmpvYmlkXG4gICRyb290U2NvcGUuam9iID0gbnVsbFxuXG4gIEpvYnNTZXJ2aWNlLmxvYWRKb2IoJHN0YXRlUGFyYW1zLmpvYmlkKS50aGVuIChkYXRhKSAtPlxuICAgICRyb290U2NvcGUuam9iID0gZGF0YVxuXG4gICRzY29wZS4kb24gJyRkZXN0cm95JywgLT5cbiAgICAkcm9vdFNjb3BlLmpvYiA9IG51bGxcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29udHJvbGxlciAnSm9iUGxhbkNvbnRyb2xsZXInLCAoJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIC0+XG4gIEpvYnNTZXJ2aWNlLmxvYWRQbGFuKCRzdGF0ZVBhcmFtcy5qb2JpZCkudGhlbiAoZGF0YSkgLT5cbiAgICAkc2NvcGUucGxhbiA9IGRhdGFcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29udHJvbGxlciAnSm9iUGxhbk5vZGVDb250cm9sbGVyJywgKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSAtPlxuICAkc2NvcGUubm9kZWlkID0gJHN0YXRlUGFyYW1zLm5vZGVpZFxuICAkc2NvcGUuc3RhdGVMaXN0ID0gSm9ic1NlcnZpY2Uuc3RhdGVMaXN0KClcblxuICBKb2JzU2VydmljZS5nZXROb2RlKCRzY29wZS5ub2RlaWQpLnRoZW4gKGRhdGEpIC0+XG4gICAgJHNjb3BlLm5vZGUgPSBkYXRhXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbnRyb2xsZXIgJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcicsICgkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkgLT5cbiAgSm9ic1NlcnZpY2UuZ2V0VmVydGV4KCRzdGF0ZVBhcmFtcy5qb2JpZCwgJHN0YXRlUGFyYW1zLnZlcnRleElkKS50aGVuIChkYXRhKSAtPlxuICAgICRzY29wZS52ZXJ0ZXggPSBkYXRhXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5jb250cm9sbGVyKCdSdW5uaW5nSm9ic0NvbnRyb2xsZXInLCBmdW5jdGlvbigkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkge1xuICAkc2NvcGUuam9iT2JzZXJ2ZXIgPSBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gJHNjb3BlLmpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdydW5uaW5nJyk7XG4gIH07XG4gIEpvYnNTZXJ2aWNlLnJlZ2lzdGVyT2JzZXJ2ZXIoJHNjb3BlLmpvYk9ic2VydmVyKTtcbiAgJHNjb3BlLiRvbignJGRlc3Ryb3knLCBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gSm9ic1NlcnZpY2UudW5SZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gIH0pO1xuICByZXR1cm4gJHNjb3BlLmpvYk9ic2VydmVyKCk7XG59KS5jb250cm9sbGVyKCdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSB7XG4gICRzY29wZS5qb2JPYnNlcnZlciA9IGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiAkc2NvcGUuam9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ2ZpbmlzaGVkJyk7XG4gIH07XG4gIEpvYnNTZXJ2aWNlLnJlZ2lzdGVyT2JzZXJ2ZXIoJHNjb3BlLmpvYk9ic2VydmVyKTtcbiAgJHNjb3BlLiRvbignJGRlc3Ryb3knLCBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gSm9ic1NlcnZpY2UudW5SZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gIH0pO1xuICByZXR1cm4gJHNjb3BlLmpvYk9ic2VydmVyKCk7XG59KS5jb250cm9sbGVyKCdTaW5nbGVKb2JDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UsICRyb290U2NvcGUpIHtcbiAgJHNjb3BlLmpvYmlkID0gJHN0YXRlUGFyYW1zLmpvYmlkO1xuICAkcm9vdFNjb3BlLmpvYiA9IG51bGw7XG4gIEpvYnNTZXJ2aWNlLmxvYWRKb2IoJHN0YXRlUGFyYW1zLmpvYmlkKS50aGVuKGZ1bmN0aW9uKGRhdGEpIHtcbiAgICByZXR1cm4gJHJvb3RTY29wZS5qb2IgPSBkYXRhO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuICRyb290U2NvcGUuam9iID0gbnVsbDtcbiAgfSk7XG59KS5jb250cm9sbGVyKCdKb2JQbGFuQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSB7XG4gIHJldHVybiBKb2JzU2VydmljZS5sb2FkUGxhbigkc3RhdGVQYXJhbXMuam9iaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgIHJldHVybiAkc2NvcGUucGxhbiA9IGRhdGE7XG4gIH0pO1xufSkuY29udHJvbGxlcignSm9iUGxhbk5vZGVDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIHtcbiAgJHNjb3BlLm5vZGVpZCA9ICRzdGF0ZVBhcmFtcy5ub2RlaWQ7XG4gICRzY29wZS5zdGF0ZUxpc3QgPSBKb2JzU2VydmljZS5zdGF0ZUxpc3QoKTtcbiAgcmV0dXJuIEpvYnNTZXJ2aWNlLmdldE5vZGUoJHNjb3BlLm5vZGVpZCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgcmV0dXJuICRzY29wZS5ub2RlID0gZGF0YTtcbiAgfSk7XG59KS5jb250cm9sbGVyKCdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInLCBmdW5jdGlvbigkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkge1xuICByZXR1cm4gSm9ic1NlcnZpY2UuZ2V0VmVydGV4KCRzdGF0ZVBhcmFtcy5qb2JpZCwgJHN0YXRlUGFyYW1zLnZlcnRleElkKS50aGVuKGZ1bmN0aW9uKGRhdGEpIHtcbiAgICByZXR1cm4gJHNjb3BlLnZlcnRleCA9IGRhdGE7XG4gIH0pO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ3ZlcnRleCcsICgkc3RhdGUpIC0+XG4gIHRlbXBsYXRlOiBcIjxzdmcgY2xhc3M9J3RpbWVsaW5lIHNlY29uZGFyeScgd2lkdGg9JzAnIGhlaWdodD0nMCc+PC9zdmc+XCJcblxuICBzY29wZTpcbiAgICBkYXRhOiBcIj1cIlxuXG4gIGxpbms6IChzY29wZSwgZWxlbSwgYXR0cnMpIC0+XG4gICAgem9vbSA9IGQzLmJlaGF2aW9yLnpvb20oKVxuICAgIHN2Z0VsID0gZWxlbS5jaGlsZHJlbigpWzBdXG5cbiAgICBjb250YWluZXJXID0gZWxlbS53aWR0aCgpXG4gICAgYW5ndWxhci5lbGVtZW50KHN2Z0VsKS5hdHRyKCd3aWR0aCcsIGNvbnRhaW5lclcgLSAxNilcblxuICAgIGFuYWx5emVUaW1lID0gKGRhdGEpIC0+XG4gICAgICB0ZXN0RGF0YSA9IFtdXG5cbiAgICAgIGFuZ3VsYXIuZm9yRWFjaCBkYXRhLmdyb3VwdmVydGV4Lmdyb3VwbWVtYmVycywgKHZlcnRleCwgaSkgLT5cbiAgICAgICAgdlRpbWUgPSBkYXRhLnZlcnRpY2V0aW1lc1t2ZXJ0ZXgudmVydGV4aWRdXG5cbiAgICAgICAgdGVzdERhdGEucHVzaCB7XG4gICAgICAgICAgbGFiZWw6IFwiI3t2ZXJ0ZXgudmVydGV4aW5zdGFuY2VuYW1lfSAoI3tpfSlcIlxuICAgICAgICAgIHRpbWVzOiBbXG4gICAgICAgICAgICB7XG4gICAgICAgICAgICAgIGxhYmVsOiBcIlNjaGVkdWxlZFwiXG4gICAgICAgICAgICAgIGNvbG9yOiBcIiM2NjZcIlxuICAgICAgICAgICAgICBzdGFydGluZ190aW1lOiB2VGltZVtcIlNDSEVEVUxFRFwiXSAqIDEwMFxuICAgICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJERVBMT1lJTkdcIl0gKiAxMDBcbiAgICAgICAgICAgIH1cbiAgICAgICAgICAgIHtcbiAgICAgICAgICAgICAgbGFiZWw6IFwiRGVwbG95aW5nXCJcbiAgICAgICAgICAgICAgY29sb3I6IFwiI2FhYVwiXG4gICAgICAgICAgICAgIHN0YXJ0aW5nX3RpbWU6IHZUaW1lW1wiREVQTE9ZSU5HXCJdICogMTAwXG4gICAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIlJVTk5JTkdcIl0gKiAxMDBcbiAgICAgICAgICAgIH1cbiAgICAgICAgICAgIHtcbiAgICAgICAgICAgICAgbGFiZWw6IFwiUnVubmluZ1wiXG4gICAgICAgICAgICAgIGNvbG9yOiBcIiNkZGRcIlxuICAgICAgICAgICAgICBzdGFydGluZ190aW1lOiB2VGltZVtcIlJVTk5JTkdcIl0gKiAxMDBcbiAgICAgICAgICAgICAgZW5kaW5nX3RpbWU6IHZUaW1lW1wiRklOSVNIRURcIl0gKiAxMDBcbiAgICAgICAgICAgIH1cbiAgICAgICAgICBdXG4gICAgICAgIH1cblxuICAgICAgY2hhcnQgPSBkMy50aW1lbGluZSgpLnN0YWNrKCkudGlja0Zvcm1hdCh7XG4gICAgICAgIGZvcm1hdDogZDMudGltZS5mb3JtYXQoXCIlU1wiKSxcbiAgICAgICAgIyB0aWNrVGltZTogZDMudGltZS5taWxsaXNlY29uZHMsXG4gICAgICAgIHRpY2tJbnRlcnZhbDogMSxcbiAgICAgICAgdGlja1NpemU6IDFcbiAgICAgIH0pLmxhYmVsRm9ybWF0KChsYWJlbCkgLT5cbiAgICAgICAgbGFiZWxcbiAgICAgICkubWFyZ2luKHsgbGVmdDogMTAwLCByaWdodDogMCwgdG9wOiAwLCBib3R0b206IDAgfSlcblxuICAgICAgc3ZnID0gZDMuc2VsZWN0KHN2Z0VsKVxuICAgICAgLmRhdHVtKHRlc3REYXRhKVxuICAgICAgLmNhbGwoY2hhcnQpXG4gICAgICAuY2FsbCh6b29tKVxuXG4gICAgICBzdmdHID0gc3ZnLnNlbGVjdChcImdcIilcblxuICAgICAgem9vbS5vbihcInpvb21cIiwgLT5cbiAgICAgICAgZXYgPSBkMy5ldmVudFxuXG4gICAgICAgIHN2Z0cuc2VsZWN0QWxsKCdyZWN0JykuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIGV2LnRyYW5zbGF0ZVswXSArIFwiLDApIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIiwxKVwiKVxuICAgICAgICBzdmdHLnNlbGVjdEFsbCgndGV4dCcpLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGVbMF0gKyBcIiwwKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIsMSlcIilcbiAgICAgIClcblxuICAgICAgYmJveCA9IHN2Z0dbMF1bMF0uZ2V0QkJveCgpXG4gICAgICBzdmcuYXR0cignaGVpZ2h0JywgYmJveC5oZWlnaHQgKyAzMClcblxuICAgIGFuYWx5emVUaW1lKHNjb3BlLmRhdGEpXG5cbiAgICByZXR1cm5cblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ3RpbWVsaW5lJywgKCRzdGF0ZSkgLT5cbiAgdGVtcGxhdGU6IFwiPHN2ZyBjbGFzcz0ndGltZWxpbmUnIHdpZHRoPScwJyBoZWlnaHQ9JzAnPjwvc3ZnPlwiXG5cbiAgc2NvcGU6XG4gICAgam9iOiBcIj1cIlxuXG4gIGxpbms6IChzY29wZSwgZWxlbSwgYXR0cnMpIC0+XG4gICAgem9vbSA9IGQzLmJlaGF2aW9yLnpvb20oKVxuICAgIHN2Z0VsID0gZWxlbS5jaGlsZHJlbigpWzBdXG5cbiAgICBjb250YWluZXJXID0gZWxlbS53aWR0aCgpXG4gICAgYW5ndWxhci5lbGVtZW50KHN2Z0VsKS5hdHRyKCd3aWR0aCcsIGNvbnRhaW5lclcgLSAxNilcblxuICAgIGFuYWx5emVUaW1lID0gKGRhdGEpIC0+XG4gICAgICB0ZXN0RGF0YSA9IFtdXG5cbiAgICAgIGFuZ3VsYXIuZm9yRWFjaCBkYXRhLm9sZFYuZ3JvdXB2ZXJ0aWNlcywgKHZlcnRleCkgLT5cbiAgICAgICAgdlRpbWUgPSBkYXRhLm9sZFYuZ3JvdXB2ZXJ0aWNldGltZXNbdmVydGV4Lmdyb3VwdmVydGV4aWRdXG5cbiAgICAgICAgIyBjb25zb2xlLmxvZyB2VGltZSwgdmVydGV4Lmdyb3VwdmVydGV4aWRcblxuICAgICAgICB0ZXN0RGF0YS5wdXNoIFxuICAgICAgICAgIHRpbWVzOiBbXG4gICAgICAgICAgICBsYWJlbDogdmVydGV4Lmdyb3VwdmVydGV4bmFtZVxuICAgICAgICAgICAgY29sb3I6IFwiIzNmYjZkOFwiXG4gICAgICAgICAgICBzdGFydGluZ190aW1lOiB2VGltZVtcIlNUQVJURURcIl1cbiAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIkVOREVEXCJdXG4gICAgICAgICAgICBsaW5rOiB2ZXJ0ZXguZ3JvdXB2ZXJ0ZXhpZFxuICAgICAgICAgIF1cblxuICAgICAgY2hhcnQgPSBkMy50aW1lbGluZSgpLnN0YWNrKCkuY2xpY2soKGQsIGksIGRhdHVtKSAtPlxuICAgICAgICAkc3RhdGUuZ28gXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLCB7IGpvYmlkOiBkYXRhLmppZCwgdmVydGV4SWQ6IGQubGluayB9XG5cbiAgICAgICkudGlja0Zvcm1hdCh7XG4gICAgICAgIGZvcm1hdDogZDMudGltZS5mb3JtYXQoXCIlU1wiKVxuICAgICAgICAjIHRpY2tUaW1lOiBkMy50aW1lLm1pbGxpc2Vjb25kc1xuICAgICAgICB0aWNrSW50ZXJ2YWw6IDFcbiAgICAgICAgdGlja1NpemU6IDFcbiAgICAgIH0pLm1hcmdpbih7IGxlZnQ6IDAsIHJpZ2h0OiAwLCB0b3A6IDAsIGJvdHRvbTogMCB9KVxuXG4gICAgICBzdmcgPSBkMy5zZWxlY3Qoc3ZnRWwpXG4gICAgICAuZGF0dW0odGVzdERhdGEpXG4gICAgICAuY2FsbChjaGFydClcbiAgICAgIC5jYWxsKHpvb20pXG5cbiAgICAgIHN2Z0cgPSBzdmcuc2VsZWN0KFwiZ1wiKVxuXG4gICAgICB6b29tLm9uKFwiem9vbVwiLCAtPlxuICAgICAgICBldiA9IGQzLmV2ZW50XG5cbiAgICAgICAgc3ZnRy5zZWxlY3RBbGwoJ3JlY3QnKS5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlWzBdICsgXCIsMCkgc2NhbGUoXCIgKyBldi5zY2FsZSArIFwiLDEpXCIpXG4gICAgICAgIHN2Z0cuc2VsZWN0QWxsKCd0ZXh0JykuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIGV2LnRyYW5zbGF0ZVswXSArIFwiLDApIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIiwxKVwiKVxuICAgICAgKVxuXG4gICAgICBiYm94ID0gc3ZnR1swXVswXS5nZXRCQm94KClcbiAgICAgIHN2Zy5hdHRyKCdoZWlnaHQnLCBiYm94LmhlaWdodCArIDMwKVxuXG4gICAgc2NvcGUuJHdhdGNoIGF0dHJzLmpvYiwgKGRhdGEpIC0+XG4gICAgICBhbmFseXplVGltZShkYXRhKSBpZiBkYXRhXG5cbiAgICByZXR1cm5cblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ2pvYlBsYW4nLCAoJHRpbWVvdXQpIC0+XG4gIHRlbXBsYXRlOiBcIlxuICAgIDxzdmcgY2xhc3M9J2dyYXBoJyB3aWR0aD0nNTAwJyBoZWlnaHQ9JzQwMCc+PGcgLz48L3N2Zz5cbiAgICA8c3ZnIGNsYXNzPSd0bXAnIHdpZHRoPScxJyBoZWlnaHQ9JzEnPjxnIC8+PC9zdmc+XG4gICAgPGRpdiBjbGFzcz0nYnRuLWdyb3VwIHpvb20tYnV0dG9ucyc+XG4gICAgICA8YSBjbGFzcz0nYnRuIGJ0bi1kZWZhdWx0IHpvb20taW4nIG5nLWNsaWNrPSd6b29tSW4oKSc+PGkgY2xhc3M9J2ZhIGZhLXBsdXMnIC8+PC9hPlxuICAgICAgPGEgY2xhc3M9J2J0biBidG4tZGVmYXVsdCB6b29tLW91dCcgbmctY2xpY2s9J3pvb21PdXQoKSc+PGkgY2xhc3M9J2ZhIGZhLW1pbnVzJyAvPjwvYT5cbiAgICA8L2Rpdj5cIlxuXG4gIHNjb3BlOlxuICAgIHBsYW46ICc9J1xuXG4gIGxpbms6IChzY29wZSwgZWxlbSwgYXR0cnMpIC0+XG4gICAgbWFpblpvb20gPSBkMy5iZWhhdmlvci56b29tKClcbiAgICBzdWJncmFwaHMgPSBbXVxuICAgIGpvYmlkID0gYXR0cnMuam9iaWRcblxuICAgIG1haW5TdmdFbGVtZW50ID0gZWxlbS5jaGlsZHJlbigpWzBdXG4gICAgbWFpbkcgPSBlbGVtLmNoaWxkcmVuKCkuY2hpbGRyZW4oKVswXVxuICAgIG1haW5UbXBFbGVtZW50ID0gZWxlbS5jaGlsZHJlbigpWzFdXG5cbiAgICBkM21haW5TdmcgPSBkMy5zZWxlY3QobWFpblN2Z0VsZW1lbnQpXG4gICAgZDNtYWluU3ZnRyA9IGQzLnNlbGVjdChtYWluRylcbiAgICBkM3RtcFN2ZyA9IGQzLnNlbGVjdChtYWluVG1wRWxlbWVudClcblxuICAgICMgYW5ndWxhci5lbGVtZW50KG1haW5HKS5lbXB0eSgpXG5cbiAgICBjb250YWluZXJXID0gZWxlbS53aWR0aCgpXG4gICAgYW5ndWxhci5lbGVtZW50KGVsZW0uY2hpbGRyZW4oKVswXSkud2lkdGgoY29udGFpbmVyVylcblxuICAgIHNjb3BlLnpvb21JbiA9IC0+XG4gICAgICBpZiBtYWluWm9vbS5zY2FsZSgpIDwgMi45OVxuICAgICAgICBcbiAgICAgICAgIyBDYWxjdWxhdGUgYW5kIHN0b3JlIG5ldyB2YWx1ZXMgaW4gem9vbSBvYmplY3RcbiAgICAgICAgdHJhbnNsYXRlID0gbWFpblpvb20udHJhbnNsYXRlKClcbiAgICAgICAgdjEgPSB0cmFuc2xhdGVbMF0gKiAobWFpblpvb20uc2NhbGUoKSArIDAuMSAvIChtYWluWm9vbS5zY2FsZSgpKSlcbiAgICAgICAgdjIgPSB0cmFuc2xhdGVbMV0gKiAobWFpblpvb20uc2NhbGUoKSArIDAuMSAvIChtYWluWm9vbS5zY2FsZSgpKSlcbiAgICAgICAgbWFpblpvb20uc2NhbGUgbWFpblpvb20uc2NhbGUoKSArIDAuMVxuICAgICAgICBtYWluWm9vbS50cmFuc2xhdGUgWyB2MSwgdjIgXVxuICAgICAgICBcbiAgICAgICAgIyBUcmFuc2Zvcm0gc3ZnXG4gICAgICAgIGQzbWFpblN2Z0cuYXR0ciBcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIHYxICsgXCIsXCIgKyB2MiArIFwiKSBzY2FsZShcIiArIG1haW5ab29tLnNjYWxlKCkgKyBcIilcIlxuXG4gICAgc2NvcGUuem9vbU91dCA9IC0+XG4gICAgICBpZiBtYWluWm9vbS5zY2FsZSgpID4gMC4zMVxuICAgICAgICBcbiAgICAgICAgIyBDYWxjdWxhdGUgYW5kIHN0b3JlIG5ldyB2YWx1ZXMgaW4gbWFpblpvb20gb2JqZWN0XG4gICAgICAgIG1haW5ab29tLnNjYWxlIG1haW5ab29tLnNjYWxlKCkgLSAwLjFcbiAgICAgICAgdHJhbnNsYXRlID0gbWFpblpvb20udHJhbnNsYXRlKClcbiAgICAgICAgdjEgPSB0cmFuc2xhdGVbMF0gKiAobWFpblpvb20uc2NhbGUoKSAtIDAuMSAvIChtYWluWm9vbS5zY2FsZSgpKSlcbiAgICAgICAgdjIgPSB0cmFuc2xhdGVbMV0gKiAobWFpblpvb20uc2NhbGUoKSAtIDAuMSAvIChtYWluWm9vbS5zY2FsZSgpKSlcbiAgICAgICAgbWFpblpvb20udHJhbnNsYXRlIFsgdjEsIHYyIF1cbiAgICAgICAgXG4gICAgICAgICMgVHJhbnNmb3JtIHN2Z1xuICAgICAgICBkM21haW5TdmdHLmF0dHIgXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyB2MSArIFwiLFwiICsgdjIgKyBcIikgc2NhbGUoXCIgKyBtYWluWm9vbS5zY2FsZSgpICsgXCIpXCJcblxuICAgICNjcmVhdGUgYSBsYWJlbCBvZiBhbiBlZGdlXG4gICAgY3JlYXRlTGFiZWxFZGdlID0gKGVsKSAtPlxuICAgICAgbGFiZWxWYWx1ZSA9IFwiXCJcbiAgICAgIGlmIGVsLnNoaXBfc3RyYXRlZ3k/IG9yIGVsLmxvY2FsX3N0cmF0ZWd5P1xuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGRpdiBjbGFzcz0nZWRnZS1sYWJlbCc+XCJcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBlbC5zaGlwX3N0cmF0ZWd5ICBpZiBlbC5zaGlwX3N0cmF0ZWd5P1xuICAgICAgICBsYWJlbFZhbHVlICs9IFwiIChcIiArIGVsLnRlbXBfbW9kZSArIFwiKVwiICB1bmxlc3MgZWwudGVtcF9tb2RlIGlzIGB1bmRlZmluZWRgXG4gICAgICAgIGxhYmVsVmFsdWUgKz0gXCIsPGJyPlwiICsgZWwubG9jYWxfc3RyYXRlZ3kgIHVubGVzcyBlbC5sb2NhbF9zdHJhdGVneSBpcyBgdW5kZWZpbmVkYFxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPC9kaXY+XCJcbiAgICAgIGxhYmVsVmFsdWVcblxuXG4gICAgIyB0cnVlLCBpZiB0aGUgbm9kZSBpcyBhIHNwZWNpYWwgbm9kZSBmcm9tIGFuIGl0ZXJhdGlvblxuICAgIGlzU3BlY2lhbEl0ZXJhdGlvbk5vZGUgPSAoaW5mbykgLT5cbiAgICAgIChpbmZvIGlzIFwicGFydGlhbFNvbHV0aW9uXCIgb3IgaW5mbyBpcyBcIm5leHRQYXJ0aWFsU29sdXRpb25cIiBvciBpbmZvIGlzIFwid29ya3NldFwiIG9yIGluZm8gaXMgXCJuZXh0V29ya3NldFwiIG9yIGluZm8gaXMgXCJzb2x1dGlvblNldFwiIG9yIGluZm8gaXMgXCJzb2x1dGlvbkRlbHRhXCIpXG5cbiAgICBnZXROb2RlVHlwZSA9IChlbCwgaW5mbykgLT5cbiAgICAgIGlmIGluZm8gaXMgXCJtaXJyb3JcIlxuICAgICAgICAnbm9kZS1taXJyb3InXG5cbiAgICAgIGVsc2UgaWYgaXNTcGVjaWFsSXRlcmF0aW9uTm9kZShpbmZvKVxuICAgICAgICAnbm9kZS1pdGVyYXRpb24nXG5cbiAgICAgIGVsc2VcbiAgICAgICAgaWYgZWwucGFjdCBpcyBcIkRhdGEgU291cmNlXCJcbiAgICAgICAgICAnbm9kZS1zb3VyY2UnXG4gICAgICAgIGVsc2UgaWYgZWwucGFjdCBpcyBcIkRhdGEgU2lua1wiXG4gICAgICAgICAgJ25vZGUtc2luaydcbiAgICAgICAgZWxzZVxuICAgICAgICAgICdub2RlLW5vcm1hbCdcbiAgICAgIFxuICAgICMgY3JlYXRlcyB0aGUgbGFiZWwgb2YgYSBub2RlLCBpbiBpbmZvIGlzIHN0b3JlZCwgd2hldGhlciBpdCBpcyBhIHNwZWNpYWwgbm9kZSAobGlrZSBhIG1pcnJvciBpbiBhbiBpdGVyYXRpb24pXG4gICAgY3JlYXRlTGFiZWxOb2RlID0gKGVsLCBpbmZvLCBtYXhXLCBtYXhIKSAtPlxuICAgICAgbGFiZWxWYWx1ZSA9IFwiPGEgaHJlZj0nIy9qb2JzL1wiICsgam9iaWQgKyBcIi9cIiArIGVsLmlkICsgXCInIGNsYXNzPSdub2RlLWxhYmVsIFwiICsgZ2V0Tm9kZVR5cGUoZWwsIGluZm8pICsgXCInPlwiXG5cbiAgICAgICMgTm9kZW5hbWVcbiAgICAgIGlmIGluZm8gaXMgXCJtaXJyb3JcIlxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGgzIGNsYXNzPSdub2RlLW5hbWUnPk1pcnJvciBvZiBcIiArIGVsLnBhY3QgKyBcIjwvaDM+XCJcbiAgICAgIGVsc2VcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoMyBjbGFzcz0nbm9kZS1uYW1lJz5cIiArIGVsLnBhY3QgKyBcIjwvaDM+XCJcbiAgICAgIGlmIGVsLmNvbnRlbnRzIGlzIFwiXCJcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIlwiXG4gICAgICBlbHNlXG4gICAgICAgIHN0ZXBOYW1lID0gZWwuY29udGVudHNcbiAgICAgICAgXG4gICAgICAgICMgY2xlYW4gc3RlcE5hbWVcbiAgICAgICAgc3RlcE5hbWUgPSBzaG9ydGVuU3RyaW5nKHN0ZXBOYW1lKVxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg0IGNsYXNzPSdzdGVwLW5hbWUnPlwiICsgc3RlcE5hbWUgKyBcIjwvaDQ+XCJcbiAgICAgIFxuICAgICAgIyBJZiB0aGlzIG5vZGUgaXMgYW4gXCJpdGVyYXRpb25cIiB3ZSBuZWVkIGEgZGlmZmVyZW50IHBhbmVsLWJvZHlcbiAgICAgIGlmIGVsLnN0ZXBfZnVuY3Rpb24/XG4gICAgICAgIGxhYmVsVmFsdWUgKz0gZXh0ZW5kTGFiZWxOb2RlRm9ySXRlcmF0aW9uKGVsLmlkLCBtYXhXLCBtYXhIKVxuICAgICAgZWxzZVxuICAgICAgICBcbiAgICAgICAgIyBPdGhlcndpc2UgYWRkIGluZm9zICAgIFxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg1PlwiICsgaW5mbyArIFwiIE5vZGU8L2g1PlwiICBpZiBpc1NwZWNpYWxJdGVyYXRpb25Ob2RlKGluZm8pXG4gICAgICAgIGxhYmVsVmFsdWUgKz0gXCI8aDU+UGFyYWxsZWxpc206IFwiICsgZWwucGFyYWxsZWxpc20gKyBcIjwvaDU+XCIgIHVubGVzcyBlbC5wYXJhbGxlbGlzbSBpcyBcIlwiXG4gICAgICAgIGxhYmVsVmFsdWUgKz0gXCI8aDU+RHJpdmVyIFN0cmF0ZWd5OiBcIiArIHNob3J0ZW5TdHJpbmcoZWwuZHJpdmVyX3N0cmF0ZWd5KSArIFwiPC9oNVwiICB1bmxlc3MgZWwuZHJpdmVyX3N0cmF0ZWd5IGlzIGB1bmRlZmluZWRgXG4gICAgICBcbiAgICAgIGxhYmVsVmFsdWUgKz0gXCI8L2E+XCJcbiAgICAgIGxhYmVsVmFsdWVcblxuICAgICMgRXh0ZW5kcyB0aGUgbGFiZWwgb2YgYSBub2RlIHdpdGggYW4gYWRkaXRpb25hbCBzdmcgRWxlbWVudCB0byBwcmVzZW50IHRoZSBpdGVyYXRpb24uXG4gICAgZXh0ZW5kTGFiZWxOb2RlRm9ySXRlcmF0aW9uID0gKGlkLCBtYXhXLCBtYXhIKSAtPlxuICAgICAgc3ZnSUQgPSBcInN2Zy1cIiArIGlkXG5cbiAgICAgIGxhYmVsVmFsdWUgPSBcIjxzdmcgY2xhc3M9J1wiICsgc3ZnSUQgKyBcIicgd2lkdGg9XCIgKyBtYXhXICsgXCIgaGVpZ2h0PVwiICsgbWF4SCArIFwiPjxnIC8+PC9zdmc+XCJcbiAgICAgIGxhYmVsVmFsdWVcblxuICAgICMgU3BsaXQgYSBzdHJpbmcgaW50byBtdWx0aXBsZSBsaW5lcyBzbyB0aGF0IGVhY2ggbGluZSBoYXMgbGVzcyB0aGFuIDMwIGxldHRlcnMuXG4gICAgc2hvcnRlblN0cmluZyA9IChzKSAtPlxuICAgICAgIyBtYWtlIHN1cmUgdGhhdCBuYW1lIGRvZXMgbm90IGNvbnRhaW4gYSA8IChiZWNhdXNlIG9mIGh0bWwpXG4gICAgICBpZiBzLmNoYXJBdCgwKSBpcyBcIjxcIlxuICAgICAgICBzID0gcy5yZXBsYWNlKFwiPFwiLCBcIiZsdDtcIilcbiAgICAgICAgcyA9IHMucmVwbGFjZShcIj5cIiwgXCImZ3Q7XCIpXG4gICAgICBzYnIgPSBcIlwiXG4gICAgICB3aGlsZSBzLmxlbmd0aCA+IDMwXG4gICAgICAgIHNiciA9IHNiciArIHMuc3Vic3RyaW5nKDAsIDMwKSArIFwiPGJyPlwiXG4gICAgICAgIHMgPSBzLnN1YnN0cmluZygzMCwgcy5sZW5ndGgpXG4gICAgICBzYnIgPSBzYnIgKyBzXG4gICAgICBzYnJcblxuICAgIGNyZWF0ZU5vZGUgPSAoZywgZGF0YSwgZWwsIGlzUGFyZW50ID0gZmFsc2UsIG1heFcsIG1heEgpIC0+XG4gICAgICAjIGNyZWF0ZSBub2RlLCBzZW5kIGFkZGl0aW9uYWwgaW5mb3JtYXRpb25zIGFib3V0IHRoZSBub2RlIGlmIGl0IGlzIGEgc3BlY2lhbCBvbmVcbiAgICAgIGlmIGVsLmlkIGlzIGRhdGEucGFydGlhbF9zb2x1dGlvblxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJwYXJ0aWFsU29sdXRpb25cIiwgbWF4VywgbWF4SClcbiAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJ1xuICAgICAgICAgIGNsYXNzOiBnZXROb2RlVHlwZShlbCwgXCJwYXJ0aWFsU29sdXRpb25cIilcblxuICAgICAgZWxzZSBpZiBlbC5pZCBpcyBkYXRhLm5leHRfcGFydGlhbF9zb2x1dGlvblxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJuZXh0UGFydGlhbFNvbHV0aW9uXCIsIG1heFcsIG1heEgpXG4gICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICBjbGFzczogZ2V0Tm9kZVR5cGUoZWwsIFwibmV4dFBhcnRpYWxTb2x1dGlvblwiKVxuXG4gICAgICBlbHNlIGlmIGVsLmlkIGlzIGRhdGEud29ya3NldFxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJ3b3Jrc2V0XCIsIG1heFcsIG1heEgpXG4gICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICBjbGFzczogZ2V0Tm9kZVR5cGUoZWwsIFwid29ya3NldFwiKVxuXG4gICAgICBlbHNlIGlmIGVsLmlkIGlzIGRhdGEubmV4dF93b3Jrc2V0XG4gICAgICAgIGcuc2V0Tm9kZSBlbC5pZCxcbiAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxOb2RlKGVsLCBcIm5leHRXb3Jrc2V0XCIsIG1heFcsIG1heEgpXG4gICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICBjbGFzczogZ2V0Tm9kZVR5cGUoZWwsIFwibmV4dFdvcmtzZXRcIilcblxuICAgICAgZWxzZSBpZiBlbC5pZCBpcyBkYXRhLnNvbHV0aW9uX3NldFxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJzb2x1dGlvblNldFwiLCBtYXhXLCBtYXhIKVxuICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgY2xhc3M6IGdldE5vZGVUeXBlKGVsLCBcInNvbHV0aW9uU2V0XCIpXG5cbiAgICAgIGVsc2UgaWYgZWwuaWQgaXMgZGF0YS5zb2x1dGlvbl9kZWx0YVxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJzb2x1dGlvbkRlbHRhXCIsIG1heFcsIG1heEgpXG4gICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICBjbGFzczogZ2V0Tm9kZVR5cGUoZWwsIFwic29sdXRpb25EZWx0YVwiKVxuXG4gICAgICBlbHNlXG4gICAgICAgIGcuc2V0Tm9kZSBlbC5pZCxcbiAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxOb2RlKGVsLCBcIlwiLCBtYXhXLCBtYXhIKVxuICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgY2xhc3M6IGdldE5vZGVUeXBlKGVsLCBcIlwiKVxuXG4gICAgY3JlYXRlRWRnZSA9IChnLCBkYXRhLCBlbCwgZXhpc3RpbmdOb2RlcywgcHJlZCkgLT5cbiAgICAgIHVubGVzcyBleGlzdGluZ05vZGVzLmluZGV4T2YocHJlZC5pZCkgaXMgLTFcbiAgICAgICAgZy5zZXRFZGdlIHByZWQuaWQsIGVsLmlkLFxuICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbEVkZ2UocHJlZClcbiAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJ1xuICAgICAgICAgIGFycm93aGVhZDogJ25vcm1hbCdcblxuICAgICAgZWxzZVxuICAgICAgICBtaXNzaW5nTm9kZSA9IHNlYXJjaEZvck5vZGUoZGF0YSwgcHJlZC5pZClcbiAgICAgICAgdW5sZXNzICFtaXNzaW5nTm9kZSBvciBtaXNzaW5nTm9kZS5hbHJlYWR5QWRkZWQgaXMgdHJ1ZVxuICAgICAgICAgIG1pc3NpbmdOb2RlLmFscmVhZHlBZGRlZCA9IHRydWVcbiAgICAgICAgICBnLnNldE5vZGUgbWlzc2luZ05vZGUuaWQsXG4gICAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxOb2RlKG1pc3NpbmdOb2RlLCBcIm1pcnJvclwiKVxuICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICAgIGNsYXNzOiBnZXROb2RlVHlwZShtaXNzaW5nTm9kZSwgJ21pcnJvcicpXG5cbiAgICAgICAgICBnLnNldEVkZ2UgbWlzc2luZ05vZGUuaWQsIGVsLmlkLFxuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsRWRnZShtaXNzaW5nTm9kZSlcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG5cbiAgICBsb2FkSnNvblRvRGFncmUgPSAoZywgZGF0YSkgLT5cbiAgICAgIGV4aXN0aW5nTm9kZXMgPSBbXVxuXG4gICAgICBpZiBkYXRhLm5vZGVzP1xuICAgICAgICAjIFRoaXMgaXMgdGhlIG5vcm1hbCBqc29uIGRhdGFcbiAgICAgICAgdG9JdGVyYXRlID0gZGF0YS5ub2Rlc1xuXG4gICAgICBlbHNlXG4gICAgICAgICMgVGhpcyBpcyBhbiBpdGVyYXRpb24sIHdlIG5vdyBzdG9yZSBzcGVjaWFsIGl0ZXJhdGlvbiBub2RlcyBpZiBwb3NzaWJsZVxuICAgICAgICB0b0l0ZXJhdGUgPSBkYXRhLnN0ZXBfZnVuY3Rpb25cbiAgICAgICAgaXNQYXJlbnQgPSB0cnVlXG5cbiAgICAgIGZvciBlbCBpbiB0b0l0ZXJhdGVcbiAgICAgICAgbWF4VyA9IDBcbiAgICAgICAgbWF4SCA9IDBcblxuICAgICAgICBpZiBlbC5zdGVwX2Z1bmN0aW9uXG4gICAgICAgICAgc2cgPSBuZXcgZGFncmVEMy5ncmFwaGxpYi5HcmFwaCh7IG11bHRpZ3JhcGg6IHRydWUsIGNvbXBvdW5kOiB0cnVlIH0pLnNldEdyYXBoKHtcbiAgICAgICAgICAgIG5vZGVzZXA6IDIwXG4gICAgICAgICAgICBlZGdlc2VwOiAwXG4gICAgICAgICAgICByYW5rc2VwOiAyMFxuICAgICAgICAgICAgcmFua2RpcjogXCJMUlwiXG4gICAgICAgICAgICBtYXJnaW54OiAxMFxuICAgICAgICAgICAgbWFyZ2lueTogMTBcbiAgICAgICAgICAgIH0pXG5cbiAgICAgICAgICBzdWJncmFwaHNbZWwuaWRdID0gc2dcblxuICAgICAgICAgIGxvYWRKc29uVG9EYWdyZShzZywgZWwpXG5cbiAgICAgICAgICByID0gbmV3IGRhZ3JlRDMucmVuZGVyKClcbiAgICAgICAgICBkM3RtcFN2Zy5zZWxlY3QoJ2cnKS5jYWxsKHIsIHNnKVxuICAgICAgICAgIG1heFcgPSBzZy5ncmFwaCgpLndpZHRoXG4gICAgICAgICAgbWF4SCA9IHNnLmdyYXBoKCkuaGVpZ2h0XG5cbiAgICAgICAgICBhbmd1bGFyLmVsZW1lbnQobWFpblRtcEVsZW1lbnQpLmVtcHR5KClcblxuICAgICAgICBjcmVhdGVOb2RlKGcsIGRhdGEsIGVsLCBpc1BhcmVudCwgbWF4VywgbWF4SClcblxuICAgICAgICBleGlzdGluZ05vZGVzLnB1c2ggZWwuaWRcbiAgICAgICAgXG4gICAgICAgICMgY3JlYXRlIGVkZ2VzIGZyb20gcHJlZGVjZXNzb3JzIHRvIGN1cnJlbnQgbm9kZVxuICAgICAgICBpZiBlbC5wcmVkZWNlc3NvcnM/XG4gICAgICAgICAgZm9yIHByZWQgaW4gZWwucHJlZGVjZXNzb3JzXG4gICAgICAgICAgICBjcmVhdGVFZGdlKGcsIGRhdGEsIGVsLCBleGlzdGluZ05vZGVzLCBwcmVkKVxuXG4gICAgICBnXG5cbiAgICAjIHNlYXJjaGVzIGluIHRoZSBnbG9iYWwgSlNPTkRhdGEgZm9yIHRoZSBub2RlIHdpdGggdGhlIGdpdmVuIGlkXG4gICAgc2VhcmNoRm9yTm9kZSA9IChkYXRhLCBub2RlSUQpIC0+XG4gICAgICBmb3IgaSBvZiBkYXRhLm5vZGVzXG4gICAgICAgIGVsID0gZGF0YS5ub2Rlc1tpXVxuICAgICAgICByZXR1cm4gZWwgIGlmIGVsLmlkIGlzIG5vZGVJRFxuICAgICAgICBcbiAgICAgICAgIyBsb29rIGZvciBub2RlcyB0aGF0IGFyZSBpbiBpdGVyYXRpb25zXG4gICAgICAgIGlmIGVsLnN0ZXBfZnVuY3Rpb24/XG4gICAgICAgICAgZm9yIGogb2YgZWwuc3RlcF9mdW5jdGlvblxuICAgICAgICAgICAgcmV0dXJuIGVsLnN0ZXBfZnVuY3Rpb25bal0gIGlmIGVsLnN0ZXBfZnVuY3Rpb25bal0uaWQgaXMgbm9kZUlEXG5cbiAgICBkcmF3R3JhcGggPSAoZGF0YSkgLT5cbiAgICAgIGcgPSBuZXcgZGFncmVEMy5ncmFwaGxpYi5HcmFwaCh7IG11bHRpZ3JhcGg6IHRydWUsIGNvbXBvdW5kOiB0cnVlIH0pLnNldEdyYXBoKHtcbiAgICAgICAgbm9kZXNlcDogNzBcbiAgICAgICAgZWRnZXNlcDogMFxuICAgICAgICByYW5rc2VwOiA1MFxuICAgICAgICByYW5rZGlyOiBcIkxSXCJcbiAgICAgICAgbWFyZ2lueDogNDBcbiAgICAgICAgbWFyZ2lueTogNDBcbiAgICAgICAgfSlcblxuICAgICAgbG9hZEpzb25Ub0RhZ3JlKGcsIGRhdGEpXG5cbiAgICAgIHJlbmRlcmVyID0gbmV3IGRhZ3JlRDMucmVuZGVyKClcbiAgICAgIGQzbWFpblN2Z0cuY2FsbChyZW5kZXJlciwgZylcblxuICAgICAgZm9yIGksIHNnIG9mIHN1YmdyYXBoc1xuICAgICAgICBkM21haW5Tdmcuc2VsZWN0KCdzdmcuc3ZnLScgKyBpICsgJyBnJykuY2FsbChyZW5kZXJlciwgc2cpXG5cbiAgICAgIG5ld1NjYWxlID0gMC41XG5cbiAgICAgIHhDZW50ZXJPZmZzZXQgPSBNYXRoLmZsb29yKChhbmd1bGFyLmVsZW1lbnQobWFpblN2Z0VsZW1lbnQpLndpZHRoKCkgLSBnLmdyYXBoKCkud2lkdGggKiBuZXdTY2FsZSkgLyAyKVxuICAgICAgeUNlbnRlck9mZnNldCA9IE1hdGguZmxvb3IoKGFuZ3VsYXIuZWxlbWVudChtYWluU3ZnRWxlbWVudCkuaGVpZ2h0KCkgLSBnLmdyYXBoKCkuaGVpZ2h0ICogbmV3U2NhbGUpIC8gMilcblxuICAgICAgbWFpblpvb20uc2NhbGUobmV3U2NhbGUpLnRyYW5zbGF0ZShbeENlbnRlck9mZnNldCwgeUNlbnRlck9mZnNldF0pXG5cbiAgICAgIGQzbWFpblN2Z0cuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIHhDZW50ZXJPZmZzZXQgKyBcIiwgXCIgKyB5Q2VudGVyT2Zmc2V0ICsgXCIpIHNjYWxlKFwiICsgbWFpblpvb20uc2NhbGUoKSArIFwiKVwiKVxuXG4gICAgICBtYWluWm9vbS5vbihcInpvb21cIiwgLT5cbiAgICAgICAgZXYgPSBkMy5ldmVudFxuICAgICAgICBkM21haW5TdmdHLmF0dHIgXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGUgKyBcIikgc2NhbGUoXCIgKyBldi5zY2FsZSArIFwiKVwiXG4gICAgICApXG4gICAgICBtYWluWm9vbShkM21haW5TdmcpXG5cbiAgICBzY29wZS4kd2F0Y2ggYXR0cnMucGxhbiwgKG5ld1BsYW4pIC0+XG4gICAgICBkcmF3R3JhcGgobmV3UGxhbikgaWYgbmV3UGxhblxuXG4gICAgcmV0dXJuXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5kaXJlY3RpdmUoJ3ZlcnRleCcsIGZ1bmN0aW9uKCRzdGF0ZSkge1xuICByZXR1cm4ge1xuICAgIHRlbXBsYXRlOiBcIjxzdmcgY2xhc3M9J3RpbWVsaW5lIHNlY29uZGFyeScgd2lkdGg9JzAnIGhlaWdodD0nMCc+PC9zdmc+XCIsXG4gICAgc2NvcGU6IHtcbiAgICAgIGRhdGE6IFwiPVwiXG4gICAgfSxcbiAgICBsaW5rOiBmdW5jdGlvbihzY29wZSwgZWxlbSwgYXR0cnMpIHtcbiAgICAgIHZhciBhbmFseXplVGltZSwgY29udGFpbmVyVywgc3ZnRWwsIHpvb207XG4gICAgICB6b29tID0gZDMuYmVoYXZpb3Iuem9vbSgpO1xuICAgICAgc3ZnRWwgPSBlbGVtLmNoaWxkcmVuKClbMF07XG4gICAgICBjb250YWluZXJXID0gZWxlbS53aWR0aCgpO1xuICAgICAgYW5ndWxhci5lbGVtZW50KHN2Z0VsKS5hdHRyKCd3aWR0aCcsIGNvbnRhaW5lclcgLSAxNik7XG4gICAgICBhbmFseXplVGltZSA9IGZ1bmN0aW9uKGRhdGEpIHtcbiAgICAgICAgdmFyIGJib3gsIGNoYXJ0LCBzdmcsIHN2Z0csIHRlc3REYXRhO1xuICAgICAgICB0ZXN0RGF0YSA9IFtdO1xuICAgICAgICBhbmd1bGFyLmZvckVhY2goZGF0YS5ncm91cHZlcnRleC5ncm91cG1lbWJlcnMsIGZ1bmN0aW9uKHZlcnRleCwgaSkge1xuICAgICAgICAgIHZhciB2VGltZTtcbiAgICAgICAgICB2VGltZSA9IGRhdGEudmVydGljZXRpbWVzW3ZlcnRleC52ZXJ0ZXhpZF07XG4gICAgICAgICAgcmV0dXJuIHRlc3REYXRhLnB1c2goe1xuICAgICAgICAgICAgbGFiZWw6IHZlcnRleC52ZXJ0ZXhpbnN0YW5jZW5hbWUgKyBcIiAoXCIgKyBpICsgXCIpXCIsXG4gICAgICAgICAgICB0aW1lczogW1xuICAgICAgICAgICAgICB7XG4gICAgICAgICAgICAgICAgbGFiZWw6IFwiU2NoZWR1bGVkXCIsXG4gICAgICAgICAgICAgICAgY29sb3I6IFwiIzY2NlwiLFxuICAgICAgICAgICAgICAgIHN0YXJ0aW5nX3RpbWU6IHZUaW1lW1wiU0NIRURVTEVEXCJdICogMTAwLFxuICAgICAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIkRFUExPWUlOR1wiXSAqIDEwMFxuICAgICAgICAgICAgICB9LCB7XG4gICAgICAgICAgICAgICAgbGFiZWw6IFwiRGVwbG95aW5nXCIsXG4gICAgICAgICAgICAgICAgY29sb3I6IFwiI2FhYVwiLFxuICAgICAgICAgICAgICAgIHN0YXJ0aW5nX3RpbWU6IHZUaW1lW1wiREVQTE9ZSU5HXCJdICogMTAwLFxuICAgICAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIlJVTk5JTkdcIl0gKiAxMDBcbiAgICAgICAgICAgICAgfSwge1xuICAgICAgICAgICAgICAgIGxhYmVsOiBcIlJ1bm5pbmdcIixcbiAgICAgICAgICAgICAgICBjb2xvcjogXCIjZGRkXCIsXG4gICAgICAgICAgICAgICAgc3RhcnRpbmdfdGltZTogdlRpbWVbXCJSVU5OSU5HXCJdICogMTAwLFxuICAgICAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIkZJTklTSEVEXCJdICogMTAwXG4gICAgICAgICAgICAgIH1cbiAgICAgICAgICAgIF1cbiAgICAgICAgICB9KTtcbiAgICAgICAgfSk7XG4gICAgICAgIGNoYXJ0ID0gZDMudGltZWxpbmUoKS5zdGFjaygpLnRpY2tGb3JtYXQoe1xuICAgICAgICAgIGZvcm1hdDogZDMudGltZS5mb3JtYXQoXCIlU1wiKSxcbiAgICAgICAgICB0aWNrSW50ZXJ2YWw6IDEsXG4gICAgICAgICAgdGlja1NpemU6IDFcbiAgICAgICAgfSkubGFiZWxGb3JtYXQoZnVuY3Rpb24obGFiZWwpIHtcbiAgICAgICAgICByZXR1cm4gbGFiZWw7XG4gICAgICAgIH0pLm1hcmdpbih7XG4gICAgICAgICAgbGVmdDogMTAwLFxuICAgICAgICAgIHJpZ2h0OiAwLFxuICAgICAgICAgIHRvcDogMCxcbiAgICAgICAgICBib3R0b206IDBcbiAgICAgICAgfSk7XG4gICAgICAgIHN2ZyA9IGQzLnNlbGVjdChzdmdFbCkuZGF0dW0odGVzdERhdGEpLmNhbGwoY2hhcnQpLmNhbGwoem9vbSk7XG4gICAgICAgIHN2Z0cgPSBzdmcuc2VsZWN0KFwiZ1wiKTtcbiAgICAgICAgem9vbS5vbihcInpvb21cIiwgZnVuY3Rpb24oKSB7XG4gICAgICAgICAgdmFyIGV2O1xuICAgICAgICAgIGV2ID0gZDMuZXZlbnQ7XG4gICAgICAgICAgc3ZnRy5zZWxlY3RBbGwoJ3JlY3QnKS5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlWzBdICsgXCIsMCkgc2NhbGUoXCIgKyBldi5zY2FsZSArIFwiLDEpXCIpO1xuICAgICAgICAgIHJldHVybiBzdmdHLnNlbGVjdEFsbCgndGV4dCcpLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGVbMF0gKyBcIiwwKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIsMSlcIik7XG4gICAgICAgIH0pO1xuICAgICAgICBiYm94ID0gc3ZnR1swXVswXS5nZXRCQm94KCk7XG4gICAgICAgIHJldHVybiBzdmcuYXR0cignaGVpZ2h0JywgYmJveC5oZWlnaHQgKyAzMCk7XG4gICAgICB9O1xuICAgICAgYW5hbHl6ZVRpbWUoc2NvcGUuZGF0YSk7XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCd0aW1lbGluZScsIGZ1bmN0aW9uKCRzdGF0ZSkge1xuICByZXR1cm4ge1xuICAgIHRlbXBsYXRlOiBcIjxzdmcgY2xhc3M9J3RpbWVsaW5lJyB3aWR0aD0nMCcgaGVpZ2h0PScwJz48L3N2Zz5cIixcbiAgICBzY29wZToge1xuICAgICAgam9iOiBcIj1cIlxuICAgIH0sXG4gICAgbGluazogZnVuY3Rpb24oc2NvcGUsIGVsZW0sIGF0dHJzKSB7XG4gICAgICB2YXIgYW5hbHl6ZVRpbWUsIGNvbnRhaW5lclcsIHN2Z0VsLCB6b29tO1xuICAgICAgem9vbSA9IGQzLmJlaGF2aW9yLnpvb20oKTtcbiAgICAgIHN2Z0VsID0gZWxlbS5jaGlsZHJlbigpWzBdO1xuICAgICAgY29udGFpbmVyVyA9IGVsZW0ud2lkdGgoKTtcbiAgICAgIGFuZ3VsYXIuZWxlbWVudChzdmdFbCkuYXR0cignd2lkdGgnLCBjb250YWluZXJXIC0gMTYpO1xuICAgICAgYW5hbHl6ZVRpbWUgPSBmdW5jdGlvbihkYXRhKSB7XG4gICAgICAgIHZhciBiYm94LCBjaGFydCwgc3ZnLCBzdmdHLCB0ZXN0RGF0YTtcbiAgICAgICAgdGVzdERhdGEgPSBbXTtcbiAgICAgICAgYW5ndWxhci5mb3JFYWNoKGRhdGEub2xkVi5ncm91cHZlcnRpY2VzLCBmdW5jdGlvbih2ZXJ0ZXgpIHtcbiAgICAgICAgICB2YXIgdlRpbWU7XG4gICAgICAgICAgdlRpbWUgPSBkYXRhLm9sZFYuZ3JvdXB2ZXJ0aWNldGltZXNbdmVydGV4Lmdyb3VwdmVydGV4aWRdO1xuICAgICAgICAgIHJldHVybiB0ZXN0RGF0YS5wdXNoKHtcbiAgICAgICAgICAgIHRpbWVzOiBbXG4gICAgICAgICAgICAgIHtcbiAgICAgICAgICAgICAgICBsYWJlbDogdmVydGV4Lmdyb3VwdmVydGV4bmFtZSxcbiAgICAgICAgICAgICAgICBjb2xvcjogXCIjM2ZiNmQ4XCIsXG4gICAgICAgICAgICAgICAgc3RhcnRpbmdfdGltZTogdlRpbWVbXCJTVEFSVEVEXCJdLFxuICAgICAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIkVOREVEXCJdLFxuICAgICAgICAgICAgICAgIGxpbms6IHZlcnRleC5ncm91cHZlcnRleGlkXG4gICAgICAgICAgICAgIH1cbiAgICAgICAgICAgIF1cbiAgICAgICAgICB9KTtcbiAgICAgICAgfSk7XG4gICAgICAgIGNoYXJ0ID0gZDMudGltZWxpbmUoKS5zdGFjaygpLmNsaWNrKGZ1bmN0aW9uKGQsIGksIGRhdHVtKSB7XG4gICAgICAgICAgcmV0dXJuICRzdGF0ZS5nbyhcInNpbmdsZS1qb2IudGltZWxpbmUudmVydGV4XCIsIHtcbiAgICAgICAgICAgIGpvYmlkOiBkYXRhLmppZCxcbiAgICAgICAgICAgIHZlcnRleElkOiBkLmxpbmtcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSkudGlja0Zvcm1hdCh7XG4gICAgICAgICAgZm9ybWF0OiBkMy50aW1lLmZvcm1hdChcIiVTXCIpLFxuICAgICAgICAgIHRpY2tJbnRlcnZhbDogMSxcbiAgICAgICAgICB0aWNrU2l6ZTogMVxuICAgICAgICB9KS5tYXJnaW4oe1xuICAgICAgICAgIGxlZnQ6IDAsXG4gICAgICAgICAgcmlnaHQ6IDAsXG4gICAgICAgICAgdG9wOiAwLFxuICAgICAgICAgIGJvdHRvbTogMFxuICAgICAgICB9KTtcbiAgICAgICAgc3ZnID0gZDMuc2VsZWN0KHN2Z0VsKS5kYXR1bSh0ZXN0RGF0YSkuY2FsbChjaGFydCkuY2FsbCh6b29tKTtcbiAgICAgICAgc3ZnRyA9IHN2Zy5zZWxlY3QoXCJnXCIpO1xuICAgICAgICB6b29tLm9uKFwiem9vbVwiLCBmdW5jdGlvbigpIHtcbiAgICAgICAgICB2YXIgZXY7XG4gICAgICAgICAgZXYgPSBkMy5ldmVudDtcbiAgICAgICAgICBzdmdHLnNlbGVjdEFsbCgncmVjdCcpLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGVbMF0gKyBcIiwwKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIsMSlcIik7XG4gICAgICAgICAgcmV0dXJuIHN2Z0cuc2VsZWN0QWxsKCd0ZXh0JykuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIGV2LnRyYW5zbGF0ZVswXSArIFwiLDApIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIiwxKVwiKTtcbiAgICAgICAgfSk7XG4gICAgICAgIGJib3ggPSBzdmdHWzBdWzBdLmdldEJCb3goKTtcbiAgICAgICAgcmV0dXJuIHN2Zy5hdHRyKCdoZWlnaHQnLCBiYm94LmhlaWdodCArIDMwKTtcbiAgICAgIH07XG4gICAgICBzY29wZS4kd2F0Y2goYXR0cnMuam9iLCBmdW5jdGlvbihkYXRhKSB7XG4gICAgICAgIGlmIChkYXRhKSB7XG4gICAgICAgICAgcmV0dXJuIGFuYWx5emVUaW1lKGRhdGEpO1xuICAgICAgICB9XG4gICAgICB9KTtcbiAgICB9XG4gIH07XG59KS5kaXJlY3RpdmUoJ2pvYlBsYW4nLCBmdW5jdGlvbigkdGltZW91dCkge1xuICByZXR1cm4ge1xuICAgIHRlbXBsYXRlOiBcIjxzdmcgY2xhc3M9J2dyYXBoJyB3aWR0aD0nNTAwJyBoZWlnaHQ9JzQwMCc+PGcgLz48L3N2Zz4gPHN2ZyBjbGFzcz0ndG1wJyB3aWR0aD0nMScgaGVpZ2h0PScxJz48ZyAvPjwvc3ZnPiA8ZGl2IGNsYXNzPSdidG4tZ3JvdXAgem9vbS1idXR0b25zJz4gPGEgY2xhc3M9J2J0biBidG4tZGVmYXVsdCB6b29tLWluJyBuZy1jbGljaz0nem9vbUluKCknPjxpIGNsYXNzPSdmYSBmYS1wbHVzJyAvPjwvYT4gPGEgY2xhc3M9J2J0biBidG4tZGVmYXVsdCB6b29tLW91dCcgbmctY2xpY2s9J3pvb21PdXQoKSc+PGkgY2xhc3M9J2ZhIGZhLW1pbnVzJyAvPjwvYT4gPC9kaXY+XCIsXG4gICAgc2NvcGU6IHtcbiAgICAgIHBsYW46ICc9J1xuICAgIH0sXG4gICAgbGluazogZnVuY3Rpb24oc2NvcGUsIGVsZW0sIGF0dHJzKSB7XG4gICAgICB2YXIgY29udGFpbmVyVywgY3JlYXRlRWRnZSwgY3JlYXRlTGFiZWxFZGdlLCBjcmVhdGVMYWJlbE5vZGUsIGNyZWF0ZU5vZGUsIGQzbWFpblN2ZywgZDNtYWluU3ZnRywgZDN0bXBTdmcsIGRyYXdHcmFwaCwgZXh0ZW5kTGFiZWxOb2RlRm9ySXRlcmF0aW9uLCBnZXROb2RlVHlwZSwgaXNTcGVjaWFsSXRlcmF0aW9uTm9kZSwgam9iaWQsIGxvYWRKc29uVG9EYWdyZSwgbWFpbkcsIG1haW5TdmdFbGVtZW50LCBtYWluVG1wRWxlbWVudCwgbWFpblpvb20sIHNlYXJjaEZvck5vZGUsIHNob3J0ZW5TdHJpbmcsIHN1YmdyYXBocztcbiAgICAgIG1haW5ab29tID0gZDMuYmVoYXZpb3Iuem9vbSgpO1xuICAgICAgc3ViZ3JhcGhzID0gW107XG4gICAgICBqb2JpZCA9IGF0dHJzLmpvYmlkO1xuICAgICAgbWFpblN2Z0VsZW1lbnQgPSBlbGVtLmNoaWxkcmVuKClbMF07XG4gICAgICBtYWluRyA9IGVsZW0uY2hpbGRyZW4oKS5jaGlsZHJlbigpWzBdO1xuICAgICAgbWFpblRtcEVsZW1lbnQgPSBlbGVtLmNoaWxkcmVuKClbMV07XG4gICAgICBkM21haW5TdmcgPSBkMy5zZWxlY3QobWFpblN2Z0VsZW1lbnQpO1xuICAgICAgZDNtYWluU3ZnRyA9IGQzLnNlbGVjdChtYWluRyk7XG4gICAgICBkM3RtcFN2ZyA9IGQzLnNlbGVjdChtYWluVG1wRWxlbWVudCk7XG4gICAgICBjb250YWluZXJXID0gZWxlbS53aWR0aCgpO1xuICAgICAgYW5ndWxhci5lbGVtZW50KGVsZW0uY2hpbGRyZW4oKVswXSkud2lkdGgoY29udGFpbmVyVyk7XG4gICAgICBzY29wZS56b29tSW4gPSBmdW5jdGlvbigpIHtcbiAgICAgICAgdmFyIHRyYW5zbGF0ZSwgdjEsIHYyO1xuICAgICAgICBpZiAobWFpblpvb20uc2NhbGUoKSA8IDIuOTkpIHtcbiAgICAgICAgICB0cmFuc2xhdGUgPSBtYWluWm9vbS50cmFuc2xhdGUoKTtcbiAgICAgICAgICB2MSA9IHRyYW5zbGF0ZVswXSAqIChtYWluWm9vbS5zY2FsZSgpICsgMC4xIC8gKG1haW5ab29tLnNjYWxlKCkpKTtcbiAgICAgICAgICB2MiA9IHRyYW5zbGF0ZVsxXSAqIChtYWluWm9vbS5zY2FsZSgpICsgMC4xIC8gKG1haW5ab29tLnNjYWxlKCkpKTtcbiAgICAgICAgICBtYWluWm9vbS5zY2FsZShtYWluWm9vbS5zY2FsZSgpICsgMC4xKTtcbiAgICAgICAgICBtYWluWm9vbS50cmFuc2xhdGUoW3YxLCB2Ml0pO1xuICAgICAgICAgIHJldHVybiBkM21haW5TdmdHLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyB2MSArIFwiLFwiICsgdjIgKyBcIikgc2NhbGUoXCIgKyBtYWluWm9vbS5zY2FsZSgpICsgXCIpXCIpO1xuICAgICAgICB9XG4gICAgICB9O1xuICAgICAgc2NvcGUuem9vbU91dCA9IGZ1bmN0aW9uKCkge1xuICAgICAgICB2YXIgdHJhbnNsYXRlLCB2MSwgdjI7XG4gICAgICAgIGlmIChtYWluWm9vbS5zY2FsZSgpID4gMC4zMSkge1xuICAgICAgICAgIG1haW5ab29tLnNjYWxlKG1haW5ab29tLnNjYWxlKCkgLSAwLjEpO1xuICAgICAgICAgIHRyYW5zbGF0ZSA9IG1haW5ab29tLnRyYW5zbGF0ZSgpO1xuICAgICAgICAgIHYxID0gdHJhbnNsYXRlWzBdICogKG1haW5ab29tLnNjYWxlKCkgLSAwLjEgLyAobWFpblpvb20uc2NhbGUoKSkpO1xuICAgICAgICAgIHYyID0gdHJhbnNsYXRlWzFdICogKG1haW5ab29tLnNjYWxlKCkgLSAwLjEgLyAobWFpblpvb20uc2NhbGUoKSkpO1xuICAgICAgICAgIG1haW5ab29tLnRyYW5zbGF0ZShbdjEsIHYyXSk7XG4gICAgICAgICAgcmV0dXJuIGQzbWFpblN2Z0cuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIHYxICsgXCIsXCIgKyB2MiArIFwiKSBzY2FsZShcIiArIG1haW5ab29tLnNjYWxlKCkgKyBcIilcIik7XG4gICAgICAgIH1cbiAgICAgIH07XG4gICAgICBjcmVhdGVMYWJlbEVkZ2UgPSBmdW5jdGlvbihlbCkge1xuICAgICAgICB2YXIgbGFiZWxWYWx1ZTtcbiAgICAgICAgbGFiZWxWYWx1ZSA9IFwiXCI7XG4gICAgICAgIGlmICgoZWwuc2hpcF9zdHJhdGVneSAhPSBudWxsKSB8fCAoZWwubG9jYWxfc3RyYXRlZ3kgIT0gbnVsbCkpIHtcbiAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGRpdiBjbGFzcz0nZWRnZS1sYWJlbCc+XCI7XG4gICAgICAgICAgaWYgKGVsLnNoaXBfc3RyYXRlZ3kgIT0gbnVsbCkge1xuICAgICAgICAgICAgbGFiZWxWYWx1ZSArPSBlbC5zaGlwX3N0cmF0ZWd5O1xuICAgICAgICAgIH1cbiAgICAgICAgICBpZiAoZWwudGVtcF9tb2RlICE9PSB1bmRlZmluZWQpIHtcbiAgICAgICAgICAgIGxhYmVsVmFsdWUgKz0gXCIgKFwiICsgZWwudGVtcF9tb2RlICsgXCIpXCI7XG4gICAgICAgICAgfVxuICAgICAgICAgIGlmIChlbC5sb2NhbF9zdHJhdGVneSAhPT0gdW5kZWZpbmVkKSB7XG4gICAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiLDxicj5cIiArIGVsLmxvY2FsX3N0cmF0ZWd5O1xuICAgICAgICAgIH1cbiAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiPC9kaXY+XCI7XG4gICAgICAgIH1cbiAgICAgICAgcmV0dXJuIGxhYmVsVmFsdWU7XG4gICAgICB9O1xuICAgICAgaXNTcGVjaWFsSXRlcmF0aW9uTm9kZSA9IGZ1bmN0aW9uKGluZm8pIHtcbiAgICAgICAgcmV0dXJuIGluZm8gPT09IFwicGFydGlhbFNvbHV0aW9uXCIgfHwgaW5mbyA9PT0gXCJuZXh0UGFydGlhbFNvbHV0aW9uXCIgfHwgaW5mbyA9PT0gXCJ3b3Jrc2V0XCIgfHwgaW5mbyA9PT0gXCJuZXh0V29ya3NldFwiIHx8IGluZm8gPT09IFwic29sdXRpb25TZXRcIiB8fCBpbmZvID09PSBcInNvbHV0aW9uRGVsdGFcIjtcbiAgICAgIH07XG4gICAgICBnZXROb2RlVHlwZSA9IGZ1bmN0aW9uKGVsLCBpbmZvKSB7XG4gICAgICAgIGlmIChpbmZvID09PSBcIm1pcnJvclwiKSB7XG4gICAgICAgICAgcmV0dXJuICdub2RlLW1pcnJvcic7XG4gICAgICAgIH0gZWxzZSBpZiAoaXNTcGVjaWFsSXRlcmF0aW9uTm9kZShpbmZvKSkge1xuICAgICAgICAgIHJldHVybiAnbm9kZS1pdGVyYXRpb24nO1xuICAgICAgICB9IGVsc2Uge1xuICAgICAgICAgIGlmIChlbC5wYWN0ID09PSBcIkRhdGEgU291cmNlXCIpIHtcbiAgICAgICAgICAgIHJldHVybiAnbm9kZS1zb3VyY2UnO1xuICAgICAgICAgIH0gZWxzZSBpZiAoZWwucGFjdCA9PT0gXCJEYXRhIFNpbmtcIikge1xuICAgICAgICAgICAgcmV0dXJuICdub2RlLXNpbmsnO1xuICAgICAgICAgIH0gZWxzZSB7XG4gICAgICAgICAgICByZXR1cm4gJ25vZGUtbm9ybWFsJztcbiAgICAgICAgICB9XG4gICAgICAgIH1cbiAgICAgIH07XG4gICAgICBjcmVhdGVMYWJlbE5vZGUgPSBmdW5jdGlvbihlbCwgaW5mbywgbWF4VywgbWF4SCkge1xuICAgICAgICB2YXIgbGFiZWxWYWx1ZSwgc3RlcE5hbWU7XG4gICAgICAgIGxhYmVsVmFsdWUgPSBcIjxhIGhyZWY9JyMvam9icy9cIiArIGpvYmlkICsgXCIvXCIgKyBlbC5pZCArIFwiJyBjbGFzcz0nbm9kZS1sYWJlbCBcIiArIGdldE5vZGVUeXBlKGVsLCBpbmZvKSArIFwiJz5cIjtcbiAgICAgICAgaWYgKGluZm8gPT09IFwibWlycm9yXCIpIHtcbiAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGgzIGNsYXNzPSdub2RlLW5hbWUnPk1pcnJvciBvZiBcIiArIGVsLnBhY3QgKyBcIjwvaDM+XCI7XG4gICAgICAgIH0gZWxzZSB7XG4gICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoMyBjbGFzcz0nbm9kZS1uYW1lJz5cIiArIGVsLnBhY3QgKyBcIjwvaDM+XCI7XG4gICAgICAgIH1cbiAgICAgICAgaWYgKGVsLmNvbnRlbnRzID09PSBcIlwiKSB7XG4gICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIlwiO1xuICAgICAgICB9IGVsc2Uge1xuICAgICAgICAgIHN0ZXBOYW1lID0gZWwuY29udGVudHM7XG4gICAgICAgICAgc3RlcE5hbWUgPSBzaG9ydGVuU3RyaW5nKHN0ZXBOYW1lKTtcbiAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg0IGNsYXNzPSdzdGVwLW5hbWUnPlwiICsgc3RlcE5hbWUgKyBcIjwvaDQ+XCI7XG4gICAgICAgIH1cbiAgICAgICAgaWYgKGVsLnN0ZXBfZnVuY3Rpb24gIT0gbnVsbCkge1xuICAgICAgICAgIGxhYmVsVmFsdWUgKz0gZXh0ZW5kTGFiZWxOb2RlRm9ySXRlcmF0aW9uKGVsLmlkLCBtYXhXLCBtYXhIKTtcbiAgICAgICAgfSBlbHNlIHtcbiAgICAgICAgICBpZiAoaXNTcGVjaWFsSXRlcmF0aW9uTm9kZShpbmZvKSkge1xuICAgICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoNT5cIiArIGluZm8gKyBcIiBOb2RlPC9oNT5cIjtcbiAgICAgICAgICB9XG4gICAgICAgICAgaWYgKGVsLnBhcmFsbGVsaXNtICE9PSBcIlwiKSB7XG4gICAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg1PlBhcmFsbGVsaXNtOiBcIiArIGVsLnBhcmFsbGVsaXNtICsgXCI8L2g1PlwiO1xuICAgICAgICAgIH1cbiAgICAgICAgICBpZiAoZWwuZHJpdmVyX3N0cmF0ZWd5ICE9PSB1bmRlZmluZWQpIHtcbiAgICAgICAgICAgIGxhYmVsVmFsdWUgKz0gXCI8aDU+RHJpdmVyIFN0cmF0ZWd5OiBcIiArIHNob3J0ZW5TdHJpbmcoZWwuZHJpdmVyX3N0cmF0ZWd5KSArIFwiPC9oNVwiO1xuICAgICAgICAgIH1cbiAgICAgICAgfVxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPC9hPlwiO1xuICAgICAgICByZXR1cm4gbGFiZWxWYWx1ZTtcbiAgICAgIH07XG4gICAgICBleHRlbmRMYWJlbE5vZGVGb3JJdGVyYXRpb24gPSBmdW5jdGlvbihpZCwgbWF4VywgbWF4SCkge1xuICAgICAgICB2YXIgbGFiZWxWYWx1ZSwgc3ZnSUQ7XG4gICAgICAgIHN2Z0lEID0gXCJzdmctXCIgKyBpZDtcbiAgICAgICAgbGFiZWxWYWx1ZSA9IFwiPHN2ZyBjbGFzcz0nXCIgKyBzdmdJRCArIFwiJyB3aWR0aD1cIiArIG1heFcgKyBcIiBoZWlnaHQ9XCIgKyBtYXhIICsgXCI+PGcgLz48L3N2Zz5cIjtcbiAgICAgICAgcmV0dXJuIGxhYmVsVmFsdWU7XG4gICAgICB9O1xuICAgICAgc2hvcnRlblN0cmluZyA9IGZ1bmN0aW9uKHMpIHtcbiAgICAgICAgdmFyIHNicjtcbiAgICAgICAgaWYgKHMuY2hhckF0KDApID09PSBcIjxcIikge1xuICAgICAgICAgIHMgPSBzLnJlcGxhY2UoXCI8XCIsIFwiJmx0O1wiKTtcbiAgICAgICAgICBzID0gcy5yZXBsYWNlKFwiPlwiLCBcIiZndDtcIik7XG4gICAgICAgIH1cbiAgICAgICAgc2JyID0gXCJcIjtcbiAgICAgICAgd2hpbGUgKHMubGVuZ3RoID4gMzApIHtcbiAgICAgICAgICBzYnIgPSBzYnIgKyBzLnN1YnN0cmluZygwLCAzMCkgKyBcIjxicj5cIjtcbiAgICAgICAgICBzID0gcy5zdWJzdHJpbmcoMzAsIHMubGVuZ3RoKTtcbiAgICAgICAgfVxuICAgICAgICBzYnIgPSBzYnIgKyBzO1xuICAgICAgICByZXR1cm4gc2JyO1xuICAgICAgfTtcbiAgICAgIGNyZWF0ZU5vZGUgPSBmdW5jdGlvbihnLCBkYXRhLCBlbCwgaXNQYXJlbnQsIG1heFcsIG1heEgpIHtcbiAgICAgICAgaWYgKGlzUGFyZW50ID09IG51bGwpIHtcbiAgICAgICAgICBpc1BhcmVudCA9IGZhbHNlO1xuICAgICAgICB9XG4gICAgICAgIGlmIChlbC5pZCA9PT0gZGF0YS5wYXJ0aWFsX3NvbHV0aW9uKSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJwYXJ0aWFsU29sdXRpb25cIiwgbWF4VywgbWF4SCksXG4gICAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJyxcbiAgICAgICAgICAgIFwiY2xhc3NcIjogZ2V0Tm9kZVR5cGUoZWwsIFwicGFydGlhbFNvbHV0aW9uXCIpXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0gZWxzZSBpZiAoZWwuaWQgPT09IGRhdGEubmV4dF9wYXJ0aWFsX3NvbHV0aW9uKSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJuZXh0UGFydGlhbFNvbHV0aW9uXCIsIG1heFcsIG1heEgpLFxuICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCcsXG4gICAgICAgICAgICBcImNsYXNzXCI6IGdldE5vZGVUeXBlKGVsLCBcIm5leHRQYXJ0aWFsU29sdXRpb25cIilcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSBlbHNlIGlmIChlbC5pZCA9PT0gZGF0YS53b3Jrc2V0KSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJ3b3Jrc2V0XCIsIG1heFcsIG1heEgpLFxuICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCcsXG4gICAgICAgICAgICBcImNsYXNzXCI6IGdldE5vZGVUeXBlKGVsLCBcIndvcmtzZXRcIilcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSBlbHNlIGlmIChlbC5pZCA9PT0gZGF0YS5uZXh0X3dvcmtzZXQpIHtcbiAgICAgICAgICByZXR1cm4gZy5zZXROb2RlKGVsLmlkLCB7XG4gICAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxOb2RlKGVsLCBcIm5leHRXb3Jrc2V0XCIsIG1heFcsIG1heEgpLFxuICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCcsXG4gICAgICAgICAgICBcImNsYXNzXCI6IGdldE5vZGVUeXBlKGVsLCBcIm5leHRXb3Jrc2V0XCIpXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0gZWxzZSBpZiAoZWwuaWQgPT09IGRhdGEuc29sdXRpb25fc2V0KSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJzb2x1dGlvblNldFwiLCBtYXhXLCBtYXhIKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgXCJjbGFzc1wiOiBnZXROb2RlVHlwZShlbCwgXCJzb2x1dGlvblNldFwiKVxuICAgICAgICAgIH0pO1xuICAgICAgICB9IGVsc2UgaWYgKGVsLmlkID09PSBkYXRhLnNvbHV0aW9uX2RlbHRhKSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJzb2x1dGlvbkRlbHRhXCIsIG1heFcsIG1heEgpLFxuICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCcsXG4gICAgICAgICAgICBcImNsYXNzXCI6IGdldE5vZGVUeXBlKGVsLCBcInNvbHV0aW9uRGVsdGFcIilcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSBlbHNlIHtcbiAgICAgICAgICByZXR1cm4gZy5zZXROb2RlKGVsLmlkLCB7XG4gICAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxOb2RlKGVsLCBcIlwiLCBtYXhXLCBtYXhIKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgXCJjbGFzc1wiOiBnZXROb2RlVHlwZShlbCwgXCJcIilcbiAgICAgICAgICB9KTtcbiAgICAgICAgfVxuICAgICAgfTtcbiAgICAgIGNyZWF0ZUVkZ2UgPSBmdW5jdGlvbihnLCBkYXRhLCBlbCwgZXhpc3RpbmdOb2RlcywgcHJlZCkge1xuICAgICAgICB2YXIgbWlzc2luZ05vZGU7XG4gICAgICAgIGlmIChleGlzdGluZ05vZGVzLmluZGV4T2YocHJlZC5pZCkgIT09IC0xKSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0RWRnZShwcmVkLmlkLCBlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsRWRnZShwcmVkKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgYXJyb3doZWFkOiAnbm9ybWFsJ1xuICAgICAgICAgIH0pO1xuICAgICAgICB9IGVsc2Uge1xuICAgICAgICAgIG1pc3NpbmdOb2RlID0gc2VhcmNoRm9yTm9kZShkYXRhLCBwcmVkLmlkKTtcbiAgICAgICAgICBpZiAoISghbWlzc2luZ05vZGUgfHwgbWlzc2luZ05vZGUuYWxyZWFkeUFkZGVkID09PSB0cnVlKSkge1xuICAgICAgICAgICAgbWlzc2luZ05vZGUuYWxyZWFkeUFkZGVkID0gdHJ1ZTtcbiAgICAgICAgICAgIGcuc2V0Tm9kZShtaXNzaW5nTm9kZS5pZCwge1xuICAgICAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxOb2RlKG1pc3NpbmdOb2RlLCBcIm1pcnJvclwiKSxcbiAgICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCcsXG4gICAgICAgICAgICAgIFwiY2xhc3NcIjogZ2V0Tm9kZVR5cGUobWlzc2luZ05vZGUsICdtaXJyb3InKVxuICAgICAgICAgICAgfSk7XG4gICAgICAgICAgICByZXR1cm4gZy5zZXRFZGdlKG1pc3NpbmdOb2RlLmlkLCBlbC5pZCwge1xuICAgICAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxFZGdlKG1pc3NpbmdOb2RlKSxcbiAgICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICAgIH0pO1xuICAgICAgICAgIH1cbiAgICAgICAgfVxuICAgICAgfTtcbiAgICAgIGxvYWRKc29uVG9EYWdyZSA9IGZ1bmN0aW9uKGcsIGRhdGEpIHtcbiAgICAgICAgdmFyIGVsLCBleGlzdGluZ05vZGVzLCBpc1BhcmVudCwgaywgbCwgbGVuLCBsZW4xLCBtYXhILCBtYXhXLCBwcmVkLCByLCByZWYsIHNnLCB0b0l0ZXJhdGU7XG4gICAgICAgIGV4aXN0aW5nTm9kZXMgPSBbXTtcbiAgICAgICAgaWYgKGRhdGEubm9kZXMgIT0gbnVsbCkge1xuICAgICAgICAgIHRvSXRlcmF0ZSA9IGRhdGEubm9kZXM7XG4gICAgICAgIH0gZWxzZSB7XG4gICAgICAgICAgdG9JdGVyYXRlID0gZGF0YS5zdGVwX2Z1bmN0aW9uO1xuICAgICAgICAgIGlzUGFyZW50ID0gdHJ1ZTtcbiAgICAgICAgfVxuICAgICAgICBmb3IgKGsgPSAwLCBsZW4gPSB0b0l0ZXJhdGUubGVuZ3RoOyBrIDwgbGVuOyBrKyspIHtcbiAgICAgICAgICBlbCA9IHRvSXRlcmF0ZVtrXTtcbiAgICAgICAgICBtYXhXID0gMDtcbiAgICAgICAgICBtYXhIID0gMDtcbiAgICAgICAgICBpZiAoZWwuc3RlcF9mdW5jdGlvbikge1xuICAgICAgICAgICAgc2cgPSBuZXcgZGFncmVEMy5ncmFwaGxpYi5HcmFwaCh7XG4gICAgICAgICAgICAgIG11bHRpZ3JhcGg6IHRydWUsXG4gICAgICAgICAgICAgIGNvbXBvdW5kOiB0cnVlXG4gICAgICAgICAgICB9KS5zZXRHcmFwaCh7XG4gICAgICAgICAgICAgIG5vZGVzZXA6IDIwLFxuICAgICAgICAgICAgICBlZGdlc2VwOiAwLFxuICAgICAgICAgICAgICByYW5rc2VwOiAyMCxcbiAgICAgICAgICAgICAgcmFua2RpcjogXCJMUlwiLFxuICAgICAgICAgICAgICBtYXJnaW54OiAxMCxcbiAgICAgICAgICAgICAgbWFyZ2lueTogMTBcbiAgICAgICAgICAgIH0pO1xuICAgICAgICAgICAgc3ViZ3JhcGhzW2VsLmlkXSA9IHNnO1xuICAgICAgICAgICAgbG9hZEpzb25Ub0RhZ3JlKHNnLCBlbCk7XG4gICAgICAgICAgICByID0gbmV3IGRhZ3JlRDMucmVuZGVyKCk7XG4gICAgICAgICAgICBkM3RtcFN2Zy5zZWxlY3QoJ2cnKS5jYWxsKHIsIHNnKTtcbiAgICAgICAgICAgIG1heFcgPSBzZy5ncmFwaCgpLndpZHRoO1xuICAgICAgICAgICAgbWF4SCA9IHNnLmdyYXBoKCkuaGVpZ2h0O1xuICAgICAgICAgICAgYW5ndWxhci5lbGVtZW50KG1haW5UbXBFbGVtZW50KS5lbXB0eSgpO1xuICAgICAgICAgIH1cbiAgICAgICAgICBjcmVhdGVOb2RlKGcsIGRhdGEsIGVsLCBpc1BhcmVudCwgbWF4VywgbWF4SCk7XG4gICAgICAgICAgZXhpc3RpbmdOb2Rlcy5wdXNoKGVsLmlkKTtcbiAgICAgICAgICBpZiAoZWwucHJlZGVjZXNzb3JzICE9IG51bGwpIHtcbiAgICAgICAgICAgIHJlZiA9IGVsLnByZWRlY2Vzc29ycztcbiAgICAgICAgICAgIGZvciAobCA9IDAsIGxlbjEgPSByZWYubGVuZ3RoOyBsIDwgbGVuMTsgbCsrKSB7XG4gICAgICAgICAgICAgIHByZWQgPSByZWZbbF07XG4gICAgICAgICAgICAgIGNyZWF0ZUVkZ2UoZywgZGF0YSwgZWwsIGV4aXN0aW5nTm9kZXMsIHByZWQpO1xuICAgICAgICAgICAgfVxuICAgICAgICAgIH1cbiAgICAgICAgfVxuICAgICAgICByZXR1cm4gZztcbiAgICAgIH07XG4gICAgICBzZWFyY2hGb3JOb2RlID0gZnVuY3Rpb24oZGF0YSwgbm9kZUlEKSB7XG4gICAgICAgIHZhciBlbCwgaSwgajtcbiAgICAgICAgZm9yIChpIGluIGRhdGEubm9kZXMpIHtcbiAgICAgICAgICBlbCA9IGRhdGEubm9kZXNbaV07XG4gICAgICAgICAgaWYgKGVsLmlkID09PSBub2RlSUQpIHtcbiAgICAgICAgICAgIHJldHVybiBlbDtcbiAgICAgICAgICB9XG4gICAgICAgICAgaWYgKGVsLnN0ZXBfZnVuY3Rpb24gIT0gbnVsbCkge1xuICAgICAgICAgICAgZm9yIChqIGluIGVsLnN0ZXBfZnVuY3Rpb24pIHtcbiAgICAgICAgICAgICAgaWYgKGVsLnN0ZXBfZnVuY3Rpb25bal0uaWQgPT09IG5vZGVJRCkge1xuICAgICAgICAgICAgICAgIHJldHVybiBlbC5zdGVwX2Z1bmN0aW9uW2pdO1xuICAgICAgICAgICAgICB9XG4gICAgICAgICAgICB9XG4gICAgICAgICAgfVxuICAgICAgICB9XG4gICAgICB9O1xuICAgICAgZHJhd0dyYXBoID0gZnVuY3Rpb24oZGF0YSkge1xuICAgICAgICB2YXIgZywgaSwgbmV3U2NhbGUsIHJlbmRlcmVyLCBzZywgeENlbnRlck9mZnNldCwgeUNlbnRlck9mZnNldDtcbiAgICAgICAgZyA9IG5ldyBkYWdyZUQzLmdyYXBobGliLkdyYXBoKHtcbiAgICAgICAgICBtdWx0aWdyYXBoOiB0cnVlLFxuICAgICAgICAgIGNvbXBvdW5kOiB0cnVlXG4gICAgICAgIH0pLnNldEdyYXBoKHtcbiAgICAgICAgICBub2Rlc2VwOiA3MCxcbiAgICAgICAgICBlZGdlc2VwOiAwLFxuICAgICAgICAgIHJhbmtzZXA6IDUwLFxuICAgICAgICAgIHJhbmtkaXI6IFwiTFJcIixcbiAgICAgICAgICBtYXJnaW54OiA0MCxcbiAgICAgICAgICBtYXJnaW55OiA0MFxuICAgICAgICB9KTtcbiAgICAgICAgbG9hZEpzb25Ub0RhZ3JlKGcsIGRhdGEpO1xuICAgICAgICByZW5kZXJlciA9IG5ldyBkYWdyZUQzLnJlbmRlcigpO1xuICAgICAgICBkM21haW5TdmdHLmNhbGwocmVuZGVyZXIsIGcpO1xuICAgICAgICBmb3IgKGkgaW4gc3ViZ3JhcGhzKSB7XG4gICAgICAgICAgc2cgPSBzdWJncmFwaHNbaV07XG4gICAgICAgICAgZDNtYWluU3ZnLnNlbGVjdCgnc3ZnLnN2Zy0nICsgaSArICcgZycpLmNhbGwocmVuZGVyZXIsIHNnKTtcbiAgICAgICAgfVxuICAgICAgICBuZXdTY2FsZSA9IDAuNTtcbiAgICAgICAgeENlbnRlck9mZnNldCA9IE1hdGguZmxvb3IoKGFuZ3VsYXIuZWxlbWVudChtYWluU3ZnRWxlbWVudCkud2lkdGgoKSAtIGcuZ3JhcGgoKS53aWR0aCAqIG5ld1NjYWxlKSAvIDIpO1xuICAgICAgICB5Q2VudGVyT2Zmc2V0ID0gTWF0aC5mbG9vcigoYW5ndWxhci5lbGVtZW50KG1haW5TdmdFbGVtZW50KS5oZWlnaHQoKSAtIGcuZ3JhcGgoKS5oZWlnaHQgKiBuZXdTY2FsZSkgLyAyKTtcbiAgICAgICAgbWFpblpvb20uc2NhbGUobmV3U2NhbGUpLnRyYW5zbGF0ZShbeENlbnRlck9mZnNldCwgeUNlbnRlck9mZnNldF0pO1xuICAgICAgICBkM21haW5TdmdHLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyB4Q2VudGVyT2Zmc2V0ICsgXCIsIFwiICsgeUNlbnRlck9mZnNldCArIFwiKSBzY2FsZShcIiArIG1haW5ab29tLnNjYWxlKCkgKyBcIilcIik7XG4gICAgICAgIG1haW5ab29tLm9uKFwiem9vbVwiLCBmdW5jdGlvbigpIHtcbiAgICAgICAgICB2YXIgZXY7XG4gICAgICAgICAgZXYgPSBkMy5ldmVudDtcbiAgICAgICAgICByZXR1cm4gZDNtYWluU3ZnRy5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlICsgXCIpIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIilcIik7XG4gICAgICAgIH0pO1xuICAgICAgICByZXR1cm4gbWFpblpvb20oZDNtYWluU3ZnKTtcbiAgICAgIH07XG4gICAgICBzY29wZS4kd2F0Y2goYXR0cnMucGxhbiwgZnVuY3Rpb24obmV3UGxhbikge1xuICAgICAgICBpZiAobmV3UGxhbikge1xuICAgICAgICAgIHJldHVybiBkcmF3R3JhcGgobmV3UGxhbik7XG4gICAgICAgIH1cbiAgICAgIH0pO1xuICAgIH1cbiAgfTtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbi5zZXJ2aWNlICdKb2JzU2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRsb2csIGFtTW9tZW50LCAkcSwgJHRpbWVvdXQpIC0+XG4gIGN1cnJlbnRKb2IgPSBudWxsXG4gIGN1cnJlbnRQbGFuID0gbnVsbFxuICBkZWZlcnJlZHMgPSB7fVxuICBqb2JzID0ge1xuICAgIHJ1bm5pbmc6IFtdXG4gICAgZmluaXNoZWQ6IFtdXG4gICAgY2FuY2VsbGVkOiBbXVxuICAgIGZhaWxlZDogW11cbiAgfVxuXG4gIGpvYk9ic2VydmVycyA9IFtdXG5cbiAgbm90aWZ5T2JzZXJ2ZXJzID0gLT5cbiAgICBhbmd1bGFyLmZvckVhY2ggam9iT2JzZXJ2ZXJzLCAoY2FsbGJhY2spIC0+XG4gICAgICBjYWxsYmFjaygpXG5cbiAgQHJlZ2lzdGVyT2JzZXJ2ZXIgPSAoY2FsbGJhY2spIC0+XG4gICAgam9iT2JzZXJ2ZXJzLnB1c2goY2FsbGJhY2spXG5cbiAgQHVuUmVnaXN0ZXJPYnNlcnZlciA9IChjYWxsYmFjaykgLT5cbiAgICBpbmRleCA9IGpvYk9ic2VydmVycy5pbmRleE9mKGNhbGxiYWNrKVxuICAgIGpvYk9ic2VydmVycy5zcGxpY2UoaW5kZXgsIDEpXG5cbiAgQHN0YXRlTGlzdCA9IC0+XG4gICAgWyBcbiAgICAgICMgJ0NSRUFURUQnXG4gICAgICAnU0NIRURVTEVEJ1xuICAgICAgJ0RFUExPWUlORydcbiAgICAgICdSVU5OSU5HJ1xuICAgICAgJ0ZJTklTSEVEJ1xuICAgICAgJ0ZBSUxFRCdcbiAgICAgICdDQU5DRUxJTkcnXG4gICAgICAnQ0FOQ0VMRUQnXG4gICAgXVxuXG4gIEB0cmFuc2xhdGVMYWJlbFN0YXRlID0gKHN0YXRlKSAtPlxuICAgIHN3aXRjaCBzdGF0ZS50b0xvd2VyQ2FzZSgpXG4gICAgICB3aGVuICdmaW5pc2hlZCcgdGhlbiAnc3VjY2VzcydcbiAgICAgIHdoZW4gJ2ZhaWxlZCcgdGhlbiAnZGFuZ2VyJ1xuICAgICAgd2hlbiAnc2NoZWR1bGVkJyB0aGVuICdkZWZhdWx0J1xuICAgICAgd2hlbiAnZGVwbG95aW5nJyB0aGVuICdpbmZvJ1xuICAgICAgd2hlbiAncnVubmluZycgdGhlbiAncHJpbWFyeSdcbiAgICAgIHdoZW4gJ2NhbmNlbGluZycgdGhlbiAnd2FybmluZydcbiAgICAgIHdoZW4gJ3BlbmRpbmcnIHRoZW4gJ2luZm8nXG4gICAgICB3aGVuICd0b3RhbCcgdGhlbiAnYmxhY2snXG4gICAgICBlbHNlICdkZWZhdWx0J1xuXG4gIEBsaXN0Sm9icyA9IC0+XG4gICAgZGVmZXJyZWQgPSAkcS5kZWZlcigpXG5cbiAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9ic1wiXG4gICAgLnN1Y2Nlc3MgKGRhdGEsIHN0YXR1cywgaGVhZGVycywgY29uZmlnKSAtPlxuXG4gICAgICBhbmd1bGFyLmZvckVhY2ggZGF0YSwgKGxpc3QsIGxpc3RLZXkpIC0+XG5cbiAgICAgICAgc3dpdGNoIGxpc3RLZXlcbiAgICAgICAgICB3aGVuICdqb2JzLXJ1bm5pbmcnIHRoZW4gam9icy5ydW5uaW5nID0gbGlzdFxuICAgICAgICAgIHdoZW4gJ2pvYnMtZmluaXNoZWQnIHRoZW4gam9icy5maW5pc2hlZCA9IGxpc3RcbiAgICAgICAgICB3aGVuICdqb2JzLWNhbmNlbGxlZCcgdGhlbiBqb2JzLmNhbmNlbGxlZCA9IGxpc3RcbiAgICAgICAgICB3aGVuICdqb2JzLWZhaWxlZCcgdGhlbiBqb2JzLmZhaWxlZCA9IGxpc3RcblxuICAgICAgICBhbmd1bGFyLmZvckVhY2ggbGlzdCwgKGpvYmlkLCBpbmRleCkgLT5cbiAgICAgICAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkXG4gICAgICAgICAgLnN1Y2Nlc3MgKGRldGFpbHMpIC0+XG4gICAgICAgICAgICBsaXN0W2luZGV4XSA9IGRldGFpbHNcblxuICAgICAgZGVmZXJyZWQucmVzb2x2ZShqb2JzKVxuICAgICAgbm90aWZ5T2JzZXJ2ZXJzKClcblxuICAgIGRlZmVycmVkLnByb21pc2VcblxuICBAZ2V0Sm9icyA9ICh0eXBlKSAtPlxuICAgIGpvYnNbdHlwZV1cblxuICBAZ2V0QWxsSm9icyA9IC0+XG4gICAgam9ic1xuXG4gIEBsb2FkSm9iID0gKGpvYmlkKSAtPlxuICAgIGN1cnJlbnRKb2IgPSBudWxsXG4gICAgZGVmZXJyZWRzLmpvYiA9ICRxLmRlZmVyKClcblxuICAgICRodHRwLmdldCBmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzL1wiICsgam9iaWRcbiAgICAuc3VjY2VzcyAoZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIC0+XG4gICAgICBkYXRhLnRpbWUgPSBEYXRlLm5vdygpXG5cbiAgICAgICRodHRwLmdldCBmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzL1wiICsgam9iaWQgKyBcIi92ZXJ0aWNlc1wiXG4gICAgICAuc3VjY2VzcyAodmVydGljZXMpIC0+XG4gICAgICAgIGRhdGEgPSBhbmd1bGFyLmV4dGVuZChkYXRhLCB2ZXJ0aWNlcylcblxuICAgICAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcuam9iU2VydmVyICsgXCIvam9ic0luZm8/Z2V0PWpvYiZqb2I9XCIgKyBqb2JpZFxuICAgICAgICAuc3VjY2VzcyAob2xkVmVydGljZXMpIC0+XG4gICAgICAgICAgZGF0YS5vbGRWID0gb2xkVmVydGljZXNbMF1cblxuICAgICAgICAgIGN1cnJlbnRKb2IgPSBkYXRhXG4gICAgICAgICAgZGVmZXJyZWRzLmpvYi5yZXNvbHZlKGRhdGEpXG5cbiAgICBkZWZlcnJlZHMuam9iLnByb21pc2VcblxuICBAbG9hZFBsYW4gPSAoam9iaWQpIC0+XG4gICAgY3VycmVudFBsYW4gPSBudWxsXG4gICAgZGVmZXJyZWRzLnBsYW4gPSAkcS5kZWZlcigpXG5cbiAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkICsgXCIvcGxhblwiXG4gICAgLnN1Y2Nlc3MgKGRhdGEpIC0+XG4gICAgICBjdXJyZW50UGxhbiA9IGRhdGFcblxuICAgICAgZGVmZXJyZWRzLnBsYW4ucmVzb2x2ZShkYXRhKVxuXG4gICAgZGVmZXJyZWRzLnBsYW4ucHJvbWlzZVxuXG4gIEBnZXROb2RlID0gKG5vZGVpZCkgLT5cbiAgICBzZWVrTm9kZSA9IChub2RlaWQsIGRhdGEpIC0+XG4gICAgICBub2RlaWQgPSBwYXJzZUludChub2RlaWQpXG5cbiAgICAgIGZvciBub2RlIGluIGRhdGFcbiAgICAgICAgcmV0dXJuIG5vZGUgaWYgbm9kZS5pZCBpcyBub2RlaWRcbiAgICAgICAgc3ViID0gc2Vla05vZGUobm9kZWlkLCBub2RlLnN0ZXBfZnVuY3Rpb24pIGlmIG5vZGUuc3RlcF9mdW5jdGlvblxuICAgICAgICByZXR1cm4gc3ViIGlmIHN1YlxuXG4gICAgICBudWxsXG5cbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKClcblxuICAgICMgaWYgY3VycmVudFBsYW5cbiAgICAjICAgZGVmZXJyZWQucmVzb2x2ZShzZWVrTm9kZShub2RlaWQsIGN1cnJlbnRQbGFuLm5vZGVzKSlcbiAgICAjIGVsc2VcbiAgICAjICAgIyBkZWZlcnJlZHMucGxhbi5wcm9taXNlLnRoZW4gKGRhdGEpIC0+XG4gICAgIyAgICRxLmFsbChbZGVmZXJyZWRzLnBsYW4ucHJvbWlzZSwgZGVmZXJyZWRzLmpvYi5wcm9taXNlXSkudGhlbiAoZGF0YSkgLT5cbiAgICAjICAgICBjb25zb2xlLmxvZyAncmVzb2x2aW5nIGdldE5vZGUnXG4gICAgIyAgICAgZGVmZXJyZWQucmVzb2x2ZShzZWVrTm9kZShub2RlaWQsIGN1cnJlbnRQbGFuLm5vZGVzKSlcblxuICAgICRxLmFsbChbZGVmZXJyZWRzLnBsYW4ucHJvbWlzZSwgZGVmZXJyZWRzLmpvYi5wcm9taXNlXSkudGhlbiAoZGF0YSkgPT5cbiAgICAgIGZvdW5kTm9kZSA9IHNlZWtOb2RlKG5vZGVpZCwgY3VycmVudFBsYW4ubm9kZXMpXG5cbiAgICAgICMgVE9ETyBsaW5rIHRvIHJlYWwgdmVydGV4LiBmb3Igbm93IHRoZXJlIGlzIG5vIHdheSB0byBnZXQgdGhlIHJpZ2h0IG9uZSwgc28gd2UgYXJlIHNob3dpbmcgdGhlIGZpcnN0IG9uZSAtIGp1c3QgZm9yIHRlc3RpbmdcbiAgICAgIEBnZXRWZXJ0ZXgoY3VycmVudEpvYi5qaWQsIGN1cnJlbnRKb2Iub2xkVi5ncm91cHZlcnRpY2VzWzBdLmdyb3VwdmVydGV4aWQpLnRoZW4gKHZlcnRleCkgLT5cbiAgICAgICAgZm91bmROb2RlLnZlcnRleCA9IHZlcnRleFxuICAgICAgICBkZWZlcnJlZC5yZXNvbHZlKGZvdW5kTm9kZSlcblxuICAgIGRlZmVycmVkLnByb21pc2VcblxuXG4gIEBnZXRWZXJ0ZXggPSAoam9iSWQsIHZlcnRleElkKSAtPlxuICAgIGRlZmVycmVkID0gJHEuZGVmZXIoKVxuXG4gICAgJGh0dHAuZ2V0IGZsaW5rQ29uZmlnLmpvYlNlcnZlciArIFwiL2pvYnNJbmZvP2dldD1ncm91cHZlcnRleCZqb2I9XCIgKyBqb2JJZCArIFwiJmdyb3VwdmVydGV4PVwiICsgdmVydGV4SWRcbiAgICAuc3VjY2VzcyAoZGF0YSkgLT5cbiAgICAgIGRlZmVycmVkLnJlc29sdmUoZGF0YSlcblxuICAgIGRlZmVycmVkLnByb21pc2VcblxuICBAXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5zZXJ2aWNlKCdKb2JzU2VydmljZScsIGZ1bmN0aW9uKCRodHRwLCBmbGlua0NvbmZpZywgJGxvZywgYW1Nb21lbnQsICRxLCAkdGltZW91dCkge1xuICB2YXIgY3VycmVudEpvYiwgY3VycmVudFBsYW4sIGRlZmVycmVkcywgam9iT2JzZXJ2ZXJzLCBqb2JzLCBub3RpZnlPYnNlcnZlcnM7XG4gIGN1cnJlbnRKb2IgPSBudWxsO1xuICBjdXJyZW50UGxhbiA9IG51bGw7XG4gIGRlZmVycmVkcyA9IHt9O1xuICBqb2JzID0ge1xuICAgIHJ1bm5pbmc6IFtdLFxuICAgIGZpbmlzaGVkOiBbXSxcbiAgICBjYW5jZWxsZWQ6IFtdLFxuICAgIGZhaWxlZDogW11cbiAgfTtcbiAgam9iT2JzZXJ2ZXJzID0gW107XG4gIG5vdGlmeU9ic2VydmVycyA9IGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiBhbmd1bGFyLmZvckVhY2goam9iT2JzZXJ2ZXJzLCBmdW5jdGlvbihjYWxsYmFjaykge1xuICAgICAgcmV0dXJuIGNhbGxiYWNrKCk7XG4gICAgfSk7XG4gIH07XG4gIHRoaXMucmVnaXN0ZXJPYnNlcnZlciA9IGZ1bmN0aW9uKGNhbGxiYWNrKSB7XG4gICAgcmV0dXJuIGpvYk9ic2VydmVycy5wdXNoKGNhbGxiYWNrKTtcbiAgfTtcbiAgdGhpcy51blJlZ2lzdGVyT2JzZXJ2ZXIgPSBmdW5jdGlvbihjYWxsYmFjaykge1xuICAgIHZhciBpbmRleDtcbiAgICBpbmRleCA9IGpvYk9ic2VydmVycy5pbmRleE9mKGNhbGxiYWNrKTtcbiAgICByZXR1cm4gam9iT2JzZXJ2ZXJzLnNwbGljZShpbmRleCwgMSk7XG4gIH07XG4gIHRoaXMuc3RhdGVMaXN0ID0gZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIFsnU0NIRURVTEVEJywgJ0RFUExPWUlORycsICdSVU5OSU5HJywgJ0ZJTklTSEVEJywgJ0ZBSUxFRCcsICdDQU5DRUxJTkcnLCAnQ0FOQ0VMRUQnXTtcbiAgfTtcbiAgdGhpcy50cmFuc2xhdGVMYWJlbFN0YXRlID0gZnVuY3Rpb24oc3RhdGUpIHtcbiAgICBzd2l0Y2ggKHN0YXRlLnRvTG93ZXJDYXNlKCkpIHtcbiAgICAgIGNhc2UgJ2ZpbmlzaGVkJzpcbiAgICAgICAgcmV0dXJuICdzdWNjZXNzJztcbiAgICAgIGNhc2UgJ2ZhaWxlZCc6XG4gICAgICAgIHJldHVybiAnZGFuZ2VyJztcbiAgICAgIGNhc2UgJ3NjaGVkdWxlZCc6XG4gICAgICAgIHJldHVybiAnZGVmYXVsdCc7XG4gICAgICBjYXNlICdkZXBsb3lpbmcnOlxuICAgICAgICByZXR1cm4gJ2luZm8nO1xuICAgICAgY2FzZSAncnVubmluZyc6XG4gICAgICAgIHJldHVybiAncHJpbWFyeSc7XG4gICAgICBjYXNlICdjYW5jZWxpbmcnOlxuICAgICAgICByZXR1cm4gJ3dhcm5pbmcnO1xuICAgICAgY2FzZSAncGVuZGluZyc6XG4gICAgICAgIHJldHVybiAnaW5mbyc7XG4gICAgICBjYXNlICd0b3RhbCc6XG4gICAgICAgIHJldHVybiAnYmxhY2snO1xuICAgICAgZGVmYXVsdDpcbiAgICAgICAgcmV0dXJuICdkZWZhdWx0JztcbiAgICB9XG4gIH07XG4gIHRoaXMubGlzdEpvYnMgPSBmdW5jdGlvbigpIHtcbiAgICB2YXIgZGVmZXJyZWQ7XG4gICAgZGVmZXJyZWQgPSAkcS5kZWZlcigpO1xuICAgICRodHRwLmdldChmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzXCIpLnN1Y2Nlc3MoZnVuY3Rpb24oZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIHtcbiAgICAgIGFuZ3VsYXIuZm9yRWFjaChkYXRhLCBmdW5jdGlvbihsaXN0LCBsaXN0S2V5KSB7XG4gICAgICAgIHN3aXRjaCAobGlzdEtleSkge1xuICAgICAgICAgIGNhc2UgJ2pvYnMtcnVubmluZyc6XG4gICAgICAgICAgICBqb2JzLnJ1bm5pbmcgPSBsaXN0O1xuICAgICAgICAgICAgYnJlYWs7XG4gICAgICAgICAgY2FzZSAnam9icy1maW5pc2hlZCc6XG4gICAgICAgICAgICBqb2JzLmZpbmlzaGVkID0gbGlzdDtcbiAgICAgICAgICAgIGJyZWFrO1xuICAgICAgICAgIGNhc2UgJ2pvYnMtY2FuY2VsbGVkJzpcbiAgICAgICAgICAgIGpvYnMuY2FuY2VsbGVkID0gbGlzdDtcbiAgICAgICAgICAgIGJyZWFrO1xuICAgICAgICAgIGNhc2UgJ2pvYnMtZmFpbGVkJzpcbiAgICAgICAgICAgIGpvYnMuZmFpbGVkID0gbGlzdDtcbiAgICAgICAgfVxuICAgICAgICByZXR1cm4gYW5ndWxhci5mb3JFYWNoKGxpc3QsIGZ1bmN0aW9uKGpvYmlkLCBpbmRleCkge1xuICAgICAgICAgIHJldHVybiAkaHR0cC5nZXQoZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkKS5zdWNjZXNzKGZ1bmN0aW9uKGRldGFpbHMpIHtcbiAgICAgICAgICAgIHJldHVybiBsaXN0W2luZGV4XSA9IGRldGFpbHM7XG4gICAgICAgICAgfSk7XG4gICAgICAgIH0pO1xuICAgICAgfSk7XG4gICAgICBkZWZlcnJlZC5yZXNvbHZlKGpvYnMpO1xuICAgICAgcmV0dXJuIG5vdGlmeU9ic2VydmVycygpO1xuICAgIH0pO1xuICAgIHJldHVybiBkZWZlcnJlZC5wcm9taXNlO1xuICB9O1xuICB0aGlzLmdldEpvYnMgPSBmdW5jdGlvbih0eXBlKSB7XG4gICAgcmV0dXJuIGpvYnNbdHlwZV07XG4gIH07XG4gIHRoaXMuZ2V0QWxsSm9icyA9IGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiBqb2JzO1xuICB9O1xuICB0aGlzLmxvYWRKb2IgPSBmdW5jdGlvbihqb2JpZCkge1xuICAgIGN1cnJlbnRKb2IgPSBudWxsO1xuICAgIGRlZmVycmVkcy5qb2IgPSAkcS5kZWZlcigpO1xuICAgICRodHRwLmdldChmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzL1wiICsgam9iaWQpLnN1Y2Nlc3MoZnVuY3Rpb24oZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIHtcbiAgICAgIGRhdGEudGltZSA9IERhdGUubm93KCk7XG4gICAgICByZXR1cm4gJGh0dHAuZ2V0KGZsaW5rQ29uZmlnLm5ld1NlcnZlciArIFwiL2pvYnMvXCIgKyBqb2JpZCArIFwiL3ZlcnRpY2VzXCIpLnN1Y2Nlc3MoZnVuY3Rpb24odmVydGljZXMpIHtcbiAgICAgICAgZGF0YSA9IGFuZ3VsYXIuZXh0ZW5kKGRhdGEsIHZlcnRpY2VzKTtcbiAgICAgICAgcmV0dXJuICRodHRwLmdldChmbGlua0NvbmZpZy5qb2JTZXJ2ZXIgKyBcIi9qb2JzSW5mbz9nZXQ9am9iJmpvYj1cIiArIGpvYmlkKS5zdWNjZXNzKGZ1bmN0aW9uKG9sZFZlcnRpY2VzKSB7XG4gICAgICAgICAgZGF0YS5vbGRWID0gb2xkVmVydGljZXNbMF07XG4gICAgICAgICAgY3VycmVudEpvYiA9IGRhdGE7XG4gICAgICAgICAgcmV0dXJuIGRlZmVycmVkcy5qb2IucmVzb2x2ZShkYXRhKTtcbiAgICAgICAgfSk7XG4gICAgICB9KTtcbiAgICB9KTtcbiAgICByZXR1cm4gZGVmZXJyZWRzLmpvYi5wcm9taXNlO1xuICB9O1xuICB0aGlzLmxvYWRQbGFuID0gZnVuY3Rpb24oam9iaWQpIHtcbiAgICBjdXJyZW50UGxhbiA9IG51bGw7XG4gICAgZGVmZXJyZWRzLnBsYW4gPSAkcS5kZWZlcigpO1xuICAgICRodHRwLmdldChmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzL1wiICsgam9iaWQgKyBcIi9wbGFuXCIpLnN1Y2Nlc3MoZnVuY3Rpb24oZGF0YSkge1xuICAgICAgY3VycmVudFBsYW4gPSBkYXRhO1xuICAgICAgcmV0dXJuIGRlZmVycmVkcy5wbGFuLnJlc29sdmUoZGF0YSk7XG4gICAgfSk7XG4gICAgcmV0dXJuIGRlZmVycmVkcy5wbGFuLnByb21pc2U7XG4gIH07XG4gIHRoaXMuZ2V0Tm9kZSA9IGZ1bmN0aW9uKG5vZGVpZCkge1xuICAgIHZhciBkZWZlcnJlZCwgc2Vla05vZGU7XG4gICAgc2Vla05vZGUgPSBmdW5jdGlvbihub2RlaWQsIGRhdGEpIHtcbiAgICAgIHZhciBpLCBsZW4sIG5vZGUsIHN1YjtcbiAgICAgIG5vZGVpZCA9IHBhcnNlSW50KG5vZGVpZCk7XG4gICAgICBmb3IgKGkgPSAwLCBsZW4gPSBkYXRhLmxlbmd0aDsgaSA8IGxlbjsgaSsrKSB7XG4gICAgICAgIG5vZGUgPSBkYXRhW2ldO1xuICAgICAgICBpZiAobm9kZS5pZCA9PT0gbm9kZWlkKSB7XG4gICAgICAgICAgcmV0dXJuIG5vZGU7XG4gICAgICAgIH1cbiAgICAgICAgaWYgKG5vZGUuc3RlcF9mdW5jdGlvbikge1xuICAgICAgICAgIHN1YiA9IHNlZWtOb2RlKG5vZGVpZCwgbm9kZS5zdGVwX2Z1bmN0aW9uKTtcbiAgICAgICAgfVxuICAgICAgICBpZiAoc3ViKSB7XG4gICAgICAgICAgcmV0dXJuIHN1YjtcbiAgICAgICAgfVxuICAgICAgfVxuICAgICAgcmV0dXJuIG51bGw7XG4gICAgfTtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJHEuYWxsKFtkZWZlcnJlZHMucGxhbi5wcm9taXNlLCBkZWZlcnJlZHMuam9iLnByb21pc2VdKS50aGVuKChmdW5jdGlvbihfdGhpcykge1xuICAgICAgcmV0dXJuIGZ1bmN0aW9uKGRhdGEpIHtcbiAgICAgICAgdmFyIGZvdW5kTm9kZTtcbiAgICAgICAgZm91bmROb2RlID0gc2Vla05vZGUobm9kZWlkLCBjdXJyZW50UGxhbi5ub2Rlcyk7XG4gICAgICAgIHJldHVybiBfdGhpcy5nZXRWZXJ0ZXgoY3VycmVudEpvYi5qaWQsIGN1cnJlbnRKb2Iub2xkVi5ncm91cHZlcnRpY2VzWzBdLmdyb3VwdmVydGV4aWQpLnRoZW4oZnVuY3Rpb24odmVydGV4KSB7XG4gICAgICAgICAgZm91bmROb2RlLnZlcnRleCA9IHZlcnRleDtcbiAgICAgICAgICByZXR1cm4gZGVmZXJyZWQucmVzb2x2ZShmb3VuZE5vZGUpO1xuICAgICAgICB9KTtcbiAgICAgIH07XG4gICAgfSkodGhpcykpO1xuICAgIHJldHVybiBkZWZlcnJlZC5wcm9taXNlO1xuICB9O1xuICB0aGlzLmdldFZlcnRleCA9IGZ1bmN0aW9uKGpvYklkLCB2ZXJ0ZXhJZCkge1xuICAgIHZhciBkZWZlcnJlZDtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJGh0dHAuZ2V0KGZsaW5rQ29uZmlnLmpvYlNlcnZlciArIFwiL2pvYnNJbmZvP2dldD1ncm91cHZlcnRleCZqb2I9XCIgKyBqb2JJZCArIFwiJmdyb3VwdmVydGV4PVwiICsgdmVydGV4SWQpLnN1Y2Nlc3MoZnVuY3Rpb24oZGF0YSkge1xuICAgICAgcmV0dXJuIGRlZmVycmVkLnJlc29sdmUoZGF0YSk7XG4gICAgfSk7XG4gICAgcmV0dXJuIGRlZmVycmVkLnByb21pc2U7XG4gIH07XG4gIHJldHVybiB0aGlzO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLmNvbnRyb2xsZXIgJ092ZXJ2aWV3Q29udHJvbGxlcicsICgkc2NvcGUsIE92ZXJ2aWV3U2VydmljZSwgSm9ic1NlcnZpY2UpIC0+XG4gICRzY29wZS5qb2JPYnNlcnZlciA9IC0+XG4gICAgJHNjb3BlLnJ1bm5pbmdKb2JzID0gSm9ic1NlcnZpY2UuZ2V0Sm9icygncnVubmluZycpXG4gICAgJHNjb3BlLmZpbmlzaGVkSm9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ2ZpbmlzaGVkJylcblxuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcbiAgJHNjb3BlLiRvbiAnJGRlc3Ryb3knLCAtPlxuICAgIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpXG5cbiAgJHNjb3BlLmpvYk9ic2VydmVyKClcbiIsImFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpLmNvbnRyb2xsZXIoJ092ZXJ2aWV3Q29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgT3ZlcnZpZXdTZXJ2aWNlLCBKb2JzU2VydmljZSkge1xuICAkc2NvcGUuam9iT2JzZXJ2ZXIgPSBmdW5jdGlvbigpIHtcbiAgICAkc2NvcGUucnVubmluZ0pvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdydW5uaW5nJyk7XG4gICAgcmV0dXJuICRzY29wZS5maW5pc2hlZEpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdmaW5pc2hlZCcpO1xuICB9O1xuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5qb2JPYnNlcnZlcigpO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLnNlcnZpY2UgJ092ZXJ2aWV3U2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRsb2cpIC0+XG4gIHNlcnZlclN0YXR1cyA9IHt9XG5cbiAgQGxvYWRTZXJ2ZXJTdGF0dXMgPSAtPlxuICAgICRodHRwLmdldChmbGlua0NvbmZpZy5qb2JTZXJ2ZXIgKyBcIi9tb25pdG9yL3N0YXR1c1wiKVxuICAgIC5zdWNjZXNzIChkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykgLT5cbiAgICAgICRsb2cgZGF0YVxuXG4gICAgLmVycm9yIChkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykgLT5cbiAgICAgIHJldHVyblxuXG4gICAgc2VydmVyU3RhdHVzXG5cbiAgQFxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuc2VydmljZSgnT3ZlcnZpZXdTZXJ2aWNlJywgZnVuY3Rpb24oJGh0dHAsIGZsaW5rQ29uZmlnLCAkbG9nKSB7XG4gIHZhciBzZXJ2ZXJTdGF0dXM7XG4gIHNlcnZlclN0YXR1cyA9IHt9O1xuICB0aGlzLmxvYWRTZXJ2ZXJTdGF0dXMgPSBmdW5jdGlvbigpIHtcbiAgICAkaHR0cC5nZXQoZmxpbmtDb25maWcuam9iU2VydmVyICsgXCIvbW9uaXRvci9zdGF0dXNcIikuc3VjY2VzcyhmdW5jdGlvbihkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykge1xuICAgICAgcmV0dXJuICRsb2coZGF0YSk7XG4gICAgfSkuZXJyb3IoZnVuY3Rpb24oZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIHt9KTtcbiAgICByZXR1cm4gc2VydmVyU3RhdHVzO1xuICB9O1xuICByZXR1cm4gdGhpcztcbn0pO1xuIl0sInNvdXJjZVJvb3QiOiIvc291cmNlLyJ9 \ No newline at end of file +//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsIm1vZHVsZXMvam9icy9qb2JzLmN0cmwuY29mZmVlIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5qcyIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5kaXIuanMiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JzL2pvYnMuc3ZjLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvb3ZlcnZpZXcvb3ZlcnZpZXcuY3RybC5qcyIsIm1vZHVsZXMvb3ZlcnZpZXcvb3ZlcnZpZXcuc3ZjLmNvZmZlZSIsIm1vZHVsZXMvb3ZlcnZpZXcvb3ZlcnZpZXcuc3ZjLmpzIl0sIm5hbWVzIjpbXSwibWFwcGluZ3MiOiJBQWtCQSxRQUFRLE9BQU8sWUFBWSxDQUFDLGFBQWEsa0JBSXhDLG1CQUFJLFNBQUMsWUFBRDtFQUNILFdBQVcsaUJBQWlCO0VDckI1QixPRHNCQSxXQUFXLGNBQWMsV0FBQTtJQUN2QixXQUFXLGlCQUFpQixDQUFDLFdBQVc7SUNyQnhDLE9Ec0JBLFdBQVcsZUFBZTs7SUFJN0IsU0FBUyxlQUFlO0VBQ3ZCLFdBQVc7RUFDWCxXQUFXO0VBR1gsaUJBQWlCO0dBS2xCLGdEQUFJLFNBQUMsYUFBYSxhQUFhLFdBQTNCO0VBQ0gsWUFBWTtFQzdCWixPRCtCQSxVQUFVLFdBQUE7SUM5QlIsT0QrQkEsWUFBWTtLQUNaLFlBQVk7SUFLZixnREFBTyxTQUFDLGdCQUFnQixvQkFBakI7RUFDTixlQUFlLE1BQU0sWUFDbkI7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sY0FDTDtJQUFBLEtBQUs7SUFDTCxVQUFVO0lBQ1YsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sbUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sd0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sdUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSw4QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsUUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7O0VDckJuQixPRHVCQSxtQkFBbUIsVUFBVTs7QUNyQi9CO0FDdEZBLFFBQVEsT0FBTyxZQUlkLFVBQVUsMkJBQVcsU0FBQyxhQUFEO0VDckJwQixPRHNCQTtJQUFBLFlBQVk7SUFDWixTQUFTO0lBQ1QsT0FDRTtNQUFBLGVBQWU7TUFDZixRQUFROztJQUVWLFVBQVU7SUFFVixNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01DckJGLE9Ec0JGLE1BQU0sZ0JBQWdCLFdBQUE7UUNyQmxCLE9Ec0JGLGlCQUFpQixZQUFZLG9CQUFvQixNQUFNOzs7O0lBSTVELFVBQVUsb0NBQW9CLFNBQUMsYUFBRDtFQ3JCN0IsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLGVBQWU7TUFDZixRQUFROztJQUVWLFVBQVU7SUFFVixNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01DckJGLE9Ec0JGLE1BQU0sZ0JBQWdCLFdBQUE7UUNyQmxCLE9Ec0JGLHNDQUFzQyxZQUFZLG9CQUFvQixNQUFNOzs7O0lBSWpGLFVBQVUsaUJBQWlCLFdBQUE7RUNyQjFCLE9Ec0JBO0lBQUEsU0FBUztJQUNULE9BQ0U7TUFBQSxPQUFPOztJQUVULFVBQVU7OztBQ2xCWjtBQ3BCQSxRQUFRLE9BQU8sWUFFZCxPQUFPLG9EQUE0QixTQUFDLHFCQUFEO0VBQ2xDLElBQUE7RUFBQSxpQ0FBaUMsU0FBQyxPQUFPLFFBQVEsZ0JBQWhCO0lBQy9CLElBQWMsT0FBTyxVQUFTLGVBQWUsVUFBUyxNQUF0RDtNQUFBLE9BQU87O0lDaEJQLE9Ea0JBLE9BQU8sU0FBUyxPQUFPLFFBQVEsT0FBTyxnQkFBZ0I7TUFBRSxNQUFNOzs7RUFFaEUsK0JBQStCLFlBQVksb0JBQW9CO0VDZi9ELE9EaUJBOztBQ2ZGO0FDS0EsUUFBUSxPQUFPLFlBRWQsV0FBVyw2RUFBeUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUNuQyxPQUFPLGNBQWMsV0FBQTtJQ25CbkIsT0RvQkEsT0FBTyxPQUFPLFlBQVksUUFBUTs7RUFFcEMsWUFBWSxpQkFBaUIsT0FBTztFQUNwQyxPQUFPLElBQUksWUFBWSxXQUFBO0lDbkJyQixPRG9CQSxZQUFZLG1CQUFtQixPQUFPOztFQ2xCeEMsT0RvQkEsT0FBTztJQUlSLFdBQVcsK0VBQTJCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDckMsT0FBTyxjQUFjLFdBQUE7SUN0Qm5CLE9EdUJBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ3RCckIsT0R1QkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNyQnhDLE9EdUJBLE9BQU87SUFJUixXQUFXLHlGQUF1QixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQWEsWUFBNUM7RUFDakMsT0FBTyxRQUFRLGFBQWE7RUFDNUIsV0FBVyxNQUFNO0VBRWpCLFlBQVksUUFBUSxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUMxQjNDLE9EMkJBLFdBQVcsTUFBTTs7RUN6Qm5CLE9EMkJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUMxQnJCLE9EMkJBLFdBQVcsTUFBTTs7SUFJcEIsV0FBVyx5RUFBcUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQzVCL0IsT0Q2QkEsWUFBWSxTQUFTLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtJQzVCNUMsT0Q2QkEsT0FBTyxPQUFPOztJQUlqQixXQUFXLDZFQUF5QixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ25DLE9BQU8sU0FBUyxhQUFhO0VBQzdCLE9BQU8sWUFBWSxZQUFZO0VDOUIvQixPRGdDQSxZQUFZLFFBQVEsT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO0lDL0J0QyxPRGdDQSxPQUFPLE9BQU87O0lBSWpCLFdBQVcsbUZBQStCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUNqQ3pDLE9Ea0NBLFlBQVksVUFBVSxhQUFhLE9BQU8sYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO0lDakNwRSxPRGtDQSxPQUFPLFNBQVM7OztBQy9CcEI7QUN4QkEsUUFBUSxPQUFPLFlBSWQsVUFBVSxxQkFBVSxTQUFDLFFBQUQ7RUNyQm5CLE9Ec0JBO0lBQUEsVUFBVTtJQUVWLE9BQ0U7TUFBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBLE9BQUE7TUFBQSxPQUFPLEdBQUcsU0FBUztNQUNuQixRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTLGFBQWE7TUFFbEQsY0FBYyxTQUFDLE1BQUQ7UUFDWixJQUFBLE1BQUEsT0FBQSxLQUFBLE1BQUE7UUFBQSxXQUFXO1FBRVgsUUFBUSxRQUFRLEtBQUssWUFBWSxjQUFjLFNBQUMsUUFBUSxHQUFUO1VBQzdDLElBQUE7VUFBQSxRQUFRLEtBQUssYUFBYSxPQUFPO1VDckIvQixPRHVCRixTQUFTLEtBQUs7WUFDWixPQUFVLE9BQU8scUJBQW1CLE9BQUksSUFBRTtZQUMxQyxPQUFPO2NBQ0w7Z0JBQ0UsT0FBTztnQkFDUCxPQUFPO2dCQUNQLGVBQWUsTUFBTSxlQUFlO2dCQUNwQyxhQUFhLE1BQU0sZUFBZTtpQkFFcEM7Z0JBQ0UsT0FBTztnQkFDUCxPQUFPO2dCQUNQLGVBQWUsTUFBTSxlQUFlO2dCQUNwQyxhQUFhLE1BQU0sYUFBYTtpQkFFbEM7Z0JBQ0UsT0FBTztnQkFDUCxPQUFPO2dCQUNQLGVBQWUsTUFBTSxhQUFhO2dCQUNsQyxhQUFhLE1BQU0sY0FBYzs7Ozs7UUFLekMsUUFBUSxHQUFHLFdBQVcsUUFBUSxXQUFXO1VBQ3ZDLFFBQVEsR0FBRyxLQUFLLE9BQU87VUFFdkIsY0FBYztVQUNkLFVBQVU7V0FDVCxZQUFZLFNBQUMsT0FBRDtVQ3pCWCxPRDBCRjtXQUNBLE9BQU87VUFBRSxNQUFNO1VBQUssT0FBTztVQUFHLEtBQUs7VUFBRyxRQUFROztRQUVoRCxNQUFNLEdBQUcsT0FBTyxPQUNmLE1BQU0sVUFDTixLQUFLLE9BQ0wsS0FBSztRQUVOLE9BQU8sSUFBSSxPQUFPO1FBRWxCLEtBQUssR0FBRyxRQUFRLFdBQUE7VUFDZCxJQUFBO1VBQUEsS0FBSyxHQUFHO1VBRVIsS0FBSyxVQUFVLFFBQVEsS0FBSyxhQUFhLGVBQWUsR0FBRyxVQUFVLEtBQUssZUFBZSxHQUFHLFFBQVE7VUMxQmxHLE9EMkJGLEtBQUssVUFBVSxRQUFRLEtBQUssYUFBYSxlQUFlLEdBQUcsVUFBVSxLQUFLLGVBQWUsR0FBRyxRQUFROztRQUd0RyxPQUFPLEtBQUssR0FBRyxHQUFHO1FDM0JoQixPRDRCRixJQUFJLEtBQUssVUFBVSxLQUFLLFNBQVM7O01BRW5DLFlBQVksTUFBTTs7O0lBTXJCLFVBQVUsdUJBQVksU0FBQyxRQUFEO0VDOUJyQixPRCtCQTtJQUFBLFVBQVU7SUFFVixPQUNFO01BQUEsS0FBSzs7SUFFUCxNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLGFBQUEsWUFBQSxPQUFBO01BQUEsT0FBTyxHQUFHLFNBQVM7TUFDbkIsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUyxhQUFhO01BRWxELGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxNQUFBLE9BQUEsS0FBQSxNQUFBO1FBQUEsV0FBVztRQUVYLFFBQVEsUUFBUSxLQUFLLEtBQUssZUFBZSxTQUFDLFFBQUQ7VUFDdkMsSUFBQTtVQUFBLFFBQVEsS0FBSyxLQUFLLGtCQUFrQixPQUFPO1VDOUJ6QyxPRGtDRixTQUFTLEtBQ1A7WUFBQSxPQUFPO2NBQ0w7Z0JBQUEsT0FBTyxPQUFPO2dCQUNkLE9BQU87Z0JBQ1AsZUFBZSxNQUFNO2dCQUNyQixhQUFhLE1BQU07Z0JBQ25CLE1BQU0sT0FBTzs7Ozs7UUFHbkIsUUFBUSxHQUFHLFdBQVcsUUFBUSxNQUFNLFNBQUMsR0FBRyxHQUFHLE9BQVA7VUM5QmhDLE9EK0JGLE9BQU8sR0FBRyw4QkFBOEI7WUFBRSxPQUFPLEtBQUs7WUFBSyxVQUFVLEVBQUU7O1dBRXZFLFdBQVc7VUFDWCxRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLGNBQWM7VUFDZCxVQUFVO1dBQ1QsT0FBTztVQUFFLE1BQU07VUFBRyxPQUFPO1VBQUcsS0FBSztVQUFHLFFBQVE7O1FBRS9DLE1BQU0sR0FBRyxPQUFPLE9BQ2YsTUFBTSxVQUNOLEtBQUssT0FDTCxLQUFLO1FBRU4sT0FBTyxJQUFJLE9BQU87UUFFbEIsS0FBSyxHQUFHLFFBQVEsV0FBQTtVQUNkLElBQUE7VUFBQSxLQUFLLEdBQUc7VUFFUixLQUFLLFVBQVUsUUFBUSxLQUFLLGFBQWEsZUFBZSxHQUFHLFVBQVUsS0FBSyxlQUFlLEdBQUcsUUFBUTtVQzlCbEcsT0QrQkYsS0FBSyxVQUFVLFFBQVEsS0FBSyxhQUFhLGVBQWUsR0FBRyxVQUFVLEtBQUssZUFBZSxHQUFHLFFBQVE7O1FBR3RHLE9BQU8sS0FBSyxHQUFHLEdBQUc7UUMvQmhCLE9EZ0NGLElBQUksS0FBSyxVQUFVLEtBQUssU0FBUzs7TUFFbkMsTUFBTSxPQUFPLE1BQU0sS0FBSyxTQUFDLE1BQUQ7UUFDdEIsSUFBcUIsTUFBckI7VUMvQkksT0QrQkosWUFBWTs7Ozs7SUFNakIsVUFBVSx3QkFBVyxTQUFDLFVBQUQ7RUMvQnBCLE9EZ0NBO0lBQUEsVUFBVTtJQVFWLE9BQ0U7TUFBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsWUFBQSxZQUFBLGlCQUFBLGlCQUFBLFlBQUEsV0FBQSxZQUFBLFVBQUEsV0FBQSw2QkFBQSxhQUFBLHdCQUFBLE9BQUEsaUJBQUEsT0FBQSxnQkFBQSxnQkFBQSxVQUFBLGVBQUEsZUFBQTtNQUFBLFdBQVcsR0FBRyxTQUFTO01BQ3ZCLFlBQVk7TUFDWixRQUFRLE1BQU07TUFFZCxpQkFBaUIsS0FBSyxXQUFXO01BQ2pDLFFBQVEsS0FBSyxXQUFXLFdBQVc7TUFDbkMsaUJBQWlCLEtBQUssV0FBVztNQUVqQyxZQUFZLEdBQUcsT0FBTztNQUN0QixhQUFhLEdBQUcsT0FBTztNQUN2QixXQUFXLEdBQUcsT0FBTztNQUlyQixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLEtBQUssV0FBVyxJQUFJLE1BQU07TUFFMUMsTUFBTSxTQUFTLFdBQUE7UUFDYixJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsWUFBWSxTQUFTO1VBQ3JCLEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxTQUFTLE1BQU0sU0FBUyxVQUFVO1VBQ2xDLFNBQVMsVUFBVSxDQUFFLElBQUk7VUMzQ3ZCLE9EOENGLFdBQVcsS0FBSyxhQUFhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUVoRyxNQUFNLFVBQVUsV0FBQTtRQUNkLElBQUEsV0FBQSxJQUFBO1FBQUEsSUFBRyxTQUFTLFVBQVUsTUFBdEI7VUFHRSxTQUFTLE1BQU0sU0FBUyxVQUFVO1VBQ2xDLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxVQUFVLENBQUUsSUFBSTtVQzdDdkIsT0RnREYsV0FBVyxLQUFLLGFBQWEsZUFBZSxLQUFLLE1BQU0sS0FBSyxhQUFhLFNBQVMsVUFBVTs7O01BR2hHLGtCQUFrQixTQUFDLElBQUQ7UUFDaEIsSUFBQTtRQUFBLGFBQWE7UUFDYixJQUFHLENBQUEsR0FBQSxpQkFBQSxVQUFxQixHQUFBLGtCQUFBLE9BQXhCO1VBQ0UsY0FBYztVQUNkLElBQW1DLEdBQUEsaUJBQUEsTUFBbkM7WUFBQSxjQUFjLEdBQUc7O1VBQ2pCLElBQWdELEdBQUcsY0FBYSxXQUFoRTtZQUFBLGNBQWMsT0FBTyxHQUFHLFlBQVk7O1VBQ3BDLElBQWtELEdBQUcsbUJBQWtCLFdBQXZFO1lBQUEsY0FBYyxVQUFVLEdBQUc7O1VBQzNCLGNBQWM7O1FDdkNkLE9Ed0NGOztNQUlGLHlCQUF5QixTQUFDLE1BQUQ7UUN6Q3JCLE9EMENELFNBQVEscUJBQXFCLFNBQVEseUJBQXlCLFNBQVEsYUFBYSxTQUFRLGlCQUFpQixTQUFRLGlCQUFpQixTQUFROztNQUVoSixjQUFjLFNBQUMsSUFBSSxNQUFMO1FBQ1osSUFBRyxTQUFRLFVBQVg7VUN6Q0ksT0QwQ0Y7ZUFFRyxJQUFHLHVCQUF1QixPQUExQjtVQzFDRCxPRDJDRjtlQURHO1VBSUgsSUFBRyxHQUFHLFNBQVEsZUFBZDtZQzNDSSxPRDRDRjtpQkFDRyxJQUFHLEdBQUcsU0FBUSxhQUFkO1lDM0NELE9ENENGO2lCQURHO1lDekNELE9ENENGOzs7O01BR04sa0JBQWtCLFNBQUMsSUFBSSxNQUFNLE1BQU0sTUFBakI7UUFDaEIsSUFBQSxZQUFBO1FBQUEsYUFBYSxxQkFBcUIsUUFBUSxNQUFNLEdBQUcsS0FBSyx5QkFBeUIsWUFBWSxJQUFJLFFBQVE7UUFHekcsSUFBRyxTQUFRLFVBQVg7VUFDRSxjQUFjLHFDQUFxQyxHQUFHLE9BQU87ZUFEL0Q7VUFHRSxjQUFjLDJCQUEyQixHQUFHLE9BQU87O1FBQ3JELElBQUcsR0FBRyxhQUFZLElBQWxCO1VBQ0UsY0FBYztlQURoQjtVQUdFLFdBQVcsR0FBRztVQUdkLFdBQVcsY0FBYztVQUN6QixjQUFjLDJCQUEyQixXQUFXOztRQUd0RCxJQUFHLEdBQUEsaUJBQUEsTUFBSDtVQUNFLGNBQWMsNEJBQTRCLEdBQUcsSUFBSSxNQUFNO2VBRHpEO1VBS0UsSUFBK0MsdUJBQXVCLE9BQXRFO1lBQUEsY0FBYyxTQUFTLE9BQU87O1VBQzlCLElBQXFFLEdBQUcsZ0JBQWUsSUFBdkY7WUFBQSxjQUFjLHNCQUFzQixHQUFHLGNBQWM7O1VBQ3JELElBQTJGLEdBQUcsb0JBQW1CLFdBQWpIO1lBQUEsY0FBYywwQkFBMEIsY0FBYyxHQUFHLG1CQUFtQjs7O1FBRTlFLGNBQWM7UUN6Q1osT0QwQ0Y7O01BR0YsOEJBQThCLFNBQUMsSUFBSSxNQUFNLE1BQVg7UUFDNUIsSUFBQSxZQUFBO1FBQUEsUUFBUSxTQUFTO1FBRWpCLGFBQWEsaUJBQWlCLFFBQVEsYUFBYSxPQUFPLGFBQWEsT0FBTztRQzFDNUUsT0QyQ0Y7O01BR0YsZ0JBQWdCLFNBQUMsR0FBRDtRQUVkLElBQUE7UUFBQSxJQUFHLEVBQUUsT0FBTyxPQUFNLEtBQWxCO1VBQ0UsSUFBSSxFQUFFLFFBQVEsS0FBSztVQUNuQixJQUFJLEVBQUUsUUFBUSxLQUFLOztRQUNyQixNQUFNO1FBQ04sT0FBTSxFQUFFLFNBQVMsSUFBakI7VUFDRSxNQUFNLE1BQU0sRUFBRSxVQUFVLEdBQUcsTUFBTTtVQUNqQyxJQUFJLEVBQUUsVUFBVSxJQUFJLEVBQUU7O1FBQ3hCLE1BQU0sTUFBTTtRQ3pDVixPRDBDRjs7TUFFRixhQUFhLFNBQUMsR0FBRyxNQUFNLElBQUksVUFBa0IsTUFBTSxNQUF0QztRQ3pDVCxJQUFJLFlBQVksTUFBTTtVRHlDQyxXQUFXOztRQUVwQyxJQUFHLEdBQUcsT0FBTSxLQUFLLGtCQUFqQjtVQ3ZDSSxPRHdDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxtQkFBbUIsTUFBTTtZQUNwRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssdUJBQWpCO1VDdkNELE9Ed0NGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLHVCQUF1QixNQUFNO1lBQ3hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxTQUFqQjtVQ3ZDRCxPRHdDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxXQUFXLE1BQU07WUFDNUMsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGNBQWpCO1VDdkNELE9Ed0NGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGVBQWUsTUFBTTtZQUNoRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssY0FBakI7VUN2Q0QsT0R3Q0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksZUFBZSxNQUFNO1lBQ2hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxnQkFBakI7VUN2Q0QsT0R3Q0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksaUJBQWlCLE1BQU07WUFDbEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUp0QjtVQ2pDRCxPRHdDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxJQUFJLE1BQU07WUFDckMsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOzs7O01BRTdCLGFBQWEsU0FBQyxHQUFHLE1BQU0sSUFBSSxlQUFlLE1BQTdCO1FBQ1gsSUFBQTtRQUFBLElBQU8sY0FBYyxRQUFRLEtBQUssUUFBTyxDQUFDLEdBQTFDO1VDcENJLE9EcUNGLEVBQUUsUUFBUSxLQUFLLElBQUksR0FBRyxJQUNwQjtZQUFBLE9BQU8sZ0JBQWdCO1lBQ3ZCLFdBQVc7WUFDWCxXQUFXOztlQUpmO1VBT0UsY0FBYyxjQUFjLE1BQU0sS0FBSztVQUN2QyxJQUFBLEVBQU8sQ0FBQyxlQUFlLFlBQVksaUJBQWdCLE9BQW5EO1lBQ0UsWUFBWSxlQUFlO1lBQzNCLEVBQUUsUUFBUSxZQUFZLElBQ3BCO2NBQUEsT0FBTyxnQkFBZ0IsYUFBYTtjQUNwQyxXQUFXO2NBQ1gsU0FBTyxZQUFZLGFBQWE7O1lDbkNoQyxPRHFDRixFQUFFLFFBQVEsWUFBWSxJQUFJLEdBQUcsSUFDM0I7Y0FBQSxPQUFPLGdCQUFnQjtjQUN2QixXQUFXOzs7OztNQUVuQixrQkFBa0IsU0FBQyxHQUFHLE1BQUo7UUFDaEIsSUFBQSxJQUFBLGVBQUEsVUFBQSxHQUFBLEdBQUEsS0FBQSxNQUFBLE1BQUEsTUFBQSxNQUFBLEdBQUEsS0FBQSxJQUFBO1FBQUEsZ0JBQWdCO1FBRWhCLElBQUcsS0FBQSxTQUFBLE1BQUg7VUFFRSxZQUFZLEtBQUs7ZUFGbkI7VUFNRSxZQUFZLEtBQUs7VUFDakIsV0FBVzs7UUFFYixLQUFBLElBQUEsR0FBQSxNQUFBLFVBQUEsUUFBQSxJQUFBLEtBQUEsS0FBQTtVQ3BDSSxLQUFLLFVBQVU7VURxQ2pCLE9BQU87VUFDUCxPQUFPO1VBRVAsSUFBRyxHQUFHLGVBQU47WUFDRSxLQUFTLElBQUEsUUFBUSxTQUFTLE1BQU07Y0FBRSxZQUFZO2NBQU0sVUFBVTtlQUFRLFNBQVM7Y0FDN0UsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTOztZQUdYLFVBQVUsR0FBRyxNQUFNO1lBRW5CLGdCQUFnQixJQUFJO1lBRXBCLElBQVEsSUFBQSxRQUFRO1lBQ2hCLFNBQVMsT0FBTyxLQUFLLEtBQUssR0FBRztZQUM3QixPQUFPLEdBQUcsUUFBUTtZQUNsQixPQUFPLEdBQUcsUUFBUTtZQUVsQixRQUFRLFFBQVEsZ0JBQWdCOztVQUVsQyxXQUFXLEdBQUcsTUFBTSxJQUFJLFVBQVUsTUFBTTtVQUV4QyxjQUFjLEtBQUssR0FBRztVQUd0QixJQUFHLEdBQUEsZ0JBQUEsTUFBSDtZQUNFLE1BQUEsR0FBQTtZQUFBLEtBQUEsSUFBQSxHQUFBLE9BQUEsSUFBQSxRQUFBLElBQUEsTUFBQSxLQUFBO2NDdkNJLE9BQU8sSUFBSTtjRHdDYixXQUFXLEdBQUcsTUFBTSxJQUFJLGVBQWU7Ozs7UUNuQzNDLE9EcUNGOztNQUdGLGdCQUFnQixTQUFDLE1BQU0sUUFBUDtRQUNkLElBQUEsSUFBQSxHQUFBO1FBQUEsS0FBQSxLQUFBLEtBQUEsT0FBQTtVQUNFLEtBQUssS0FBSyxNQUFNO1VBQ2hCLElBQWMsR0FBRyxPQUFNLFFBQXZCO1lBQUEsT0FBTzs7VUFHUCxJQUFHLEdBQUEsaUJBQUEsTUFBSDtZQUNFLEtBQUEsS0FBQSxHQUFBLGVBQUE7Y0FDRSxJQUErQixHQUFHLGNBQWMsR0FBRyxPQUFNLFFBQXpEO2dCQUFBLE9BQU8sR0FBRyxjQUFjOzs7Ozs7TUFFaEMsWUFBWSxTQUFDLE1BQUQ7UUFDVixJQUFBLEdBQUEsR0FBQSxVQUFBLFVBQUEsSUFBQSxlQUFBO1FBQUEsSUFBUSxJQUFBLFFBQVEsU0FBUyxNQUFNO1VBQUUsWUFBWTtVQUFNLFVBQVU7V0FBUSxTQUFTO1VBQzVFLFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUzs7UUFHWCxnQkFBZ0IsR0FBRztRQUVuQixXQUFlLElBQUEsUUFBUTtRQUN2QixXQUFXLEtBQUssVUFBVTtRQUUxQixLQUFBLEtBQUEsV0FBQTtVQzlCSSxLQUFLLFVBQVU7VUQrQmpCLFVBQVUsT0FBTyxhQUFhLElBQUksTUFBTSxLQUFLLFVBQVU7O1FBRXpELFdBQVc7UUFFWCxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixVQUFVLEVBQUUsUUFBUSxRQUFRLFlBQVk7UUFDcEcsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsV0FBVyxFQUFFLFFBQVEsU0FBUyxZQUFZO1FBRXRHLFNBQVMsTUFBTSxVQUFVLFVBQVUsQ0FBQyxlQUFlO1FBRW5ELFdBQVcsS0FBSyxhQUFhLGVBQWUsZ0JBQWdCLE9BQU8sZ0JBQWdCLGFBQWEsU0FBUyxVQUFVO1FBRW5ILFNBQVMsR0FBRyxRQUFRLFdBQUE7VUFDbEIsSUFBQTtVQUFBLEtBQUssR0FBRztVQ2hDTixPRGlDRixXQUFXLEtBQUssYUFBYSxlQUFlLEdBQUcsWUFBWSxhQUFhLEdBQUcsUUFBUTs7UUMvQm5GLE9EaUNGLFNBQVM7O01BRVgsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLFNBQUQ7UUFDdkIsSUFBc0IsU0FBdEI7VUNoQ0ksT0RnQ0osVUFBVTs7Ozs7O0FDMUJoQjtBQzVaQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDhFQUFlLFNBQUMsT0FBTyxhQUFhLE1BQU0sVUFBVSxJQUFJLFVBQXpDO0VBQ3RCLElBQUEsWUFBQSxhQUFBLFdBQUEsY0FBQSxNQUFBO0VBQUEsYUFBYTtFQUNiLGNBQWM7RUFDZCxZQUFZO0VBQ1osT0FBTztJQUNMLFNBQVM7SUFDVCxVQUFVO0lBQ1YsV0FBVztJQUNYLFFBQVE7O0VBR1YsZUFBZTtFQUVmLGtCQUFrQixXQUFBO0lDcEJoQixPRHFCQSxRQUFRLFFBQVEsY0FBYyxTQUFDLFVBQUQ7TUNwQjVCLE9EcUJBOzs7RUFFSixLQUFDLG1CQUFtQixTQUFDLFVBQUQ7SUNuQmxCLE9Eb0JBLGFBQWEsS0FBSzs7RUFFcEIsS0FBQyxxQkFBcUIsU0FBQyxVQUFEO0lBQ3BCLElBQUE7SUFBQSxRQUFRLGFBQWEsUUFBUTtJQ2xCN0IsT0RtQkEsYUFBYSxPQUFPLE9BQU87O0VBRTdCLEtBQUMsWUFBWSxXQUFBO0lDbEJYLE9EbUJBLENBRUUsYUFDQSxhQUNBLFdBQ0EsWUFDQSxVQUNBLGFBQ0E7O0VBR0osS0FBQyxzQkFBc0IsU0FBQyxPQUFEO0lBQ3JCLFFBQU8sTUFBTTtNQUFiLEtBQ087UUMzQkgsT0QyQm1CO01BRHZCLEtBRU87UUMxQkgsT0QwQmlCO01BRnJCLEtBR087UUN6QkgsT0R5Qm9CO01BSHhCLEtBSU87UUN4QkgsT0R3Qm9CO01BSnhCLEtBS087UUN2QkgsT0R1QmtCO01BTHRCLEtBTU87UUN0QkgsT0RzQm9CO01BTnhCLEtBT087UUNyQkgsT0RxQmtCO01BUHRCLEtBUU87UUNwQkgsT0RvQmdCO01BUnBCO1FDVkksT0RtQkc7OztFQUVULEtBQUMsV0FBVyxXQUFBO0lBQ1YsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxZQUFZLFlBQVksU0FDakMsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BRVAsUUFBUSxRQUFRLE1BQU0sU0FBQyxNQUFNLFNBQVA7UUFFcEIsUUFBTztVQUFQLEtBQ087WUFBb0IsS0FBSyxVQUFVO1lBQW5DO1VBRFAsS0FFTztZQUFxQixLQUFLLFdBQVc7WUFBckM7VUFGUCxLQUdPO1lBQXNCLEtBQUssWUFBWTtZQUF2QztVQUhQLEtBSU87WUFBbUIsS0FBSyxTQUFTOztRQ1p4QyxPRGNBLFFBQVEsUUFBUSxNQUFNLFNBQUMsT0FBTyxPQUFSO1VDYnBCLE9EY0EsTUFBTSxJQUFJLFlBQVksWUFBWSxXQUFXLE9BQzVDLFFBQVEsU0FBQyxTQUFEO1lDZFAsT0RlQSxLQUFLLFNBQVM7Ozs7TUFFcEIsU0FBUyxRQUFRO01DWmpCLE9EYUE7O0lDWEYsT0RhQSxTQUFTOztFQUVYLEtBQUMsVUFBVSxTQUFDLE1BQUQ7SUNaVCxPRGFBLEtBQUs7O0VBRVAsS0FBQyxhQUFhLFdBQUE7SUNaWixPRGFBOztFQUVGLEtBQUMsVUFBVSxTQUFDLE9BQUQ7SUFDVCxhQUFhO0lBQ2IsVUFBVSxNQUFNLEdBQUc7SUFFbkIsTUFBTSxJQUFJLFlBQVksWUFBWSxXQUFXLE9BQzVDLFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLEtBQUssT0FBTyxLQUFLO01DZGpCLE9EZ0JBLE1BQU0sSUFBSSxZQUFZLFlBQVksV0FBVyxRQUFRLGFBQ3BELFFBQVEsU0FBQyxVQUFEO1FBQ1AsT0FBTyxRQUFRLE9BQU8sTUFBTTtRQ2hCNUIsT0RrQkEsTUFBTSxJQUFJLFlBQVksWUFBWSwyQkFBMkIsT0FDNUQsUUFBUSxTQUFDLGFBQUQ7VUFDUCxLQUFLLE9BQU8sWUFBWTtVQ2xCeEIsT0RvQkEsTUFBTSxJQUFJLFlBQVksWUFBWSxXQUFXLFFBQVEsV0FDcEQsUUFBUSxTQUFDLFdBQUQ7WUFDUCxPQUFPLFFBQVEsT0FBTyxNQUFNO1lBRTVCLGFBQWE7WUNyQmIsT0RzQkEsVUFBVSxJQUFJLFFBQVE7Ozs7O0lDakI5QixPRG1CQSxVQUFVLElBQUk7O0VBRWhCLEtBQUMsV0FBVyxTQUFDLE9BQUQ7SUFDVixjQUFjO0lBQ2QsVUFBVSxPQUFPLEdBQUc7SUFFcEIsTUFBTSxJQUFJLFlBQVksWUFBWSxXQUFXLFFBQVEsU0FDcEQsUUFBUSxTQUFDLE1BQUQ7TUFDUCxjQUFjO01DcEJkLE9Ec0JBLFVBQVUsS0FBSyxRQUFROztJQ3BCekIsT0RzQkEsVUFBVSxLQUFLOztFQUVqQixLQUFDLFVBQVUsU0FBQyxRQUFEO0lBQ1QsSUFBQSxVQUFBO0lBQUEsV0FBVyxTQUFDLFFBQVEsTUFBVDtNQUNULElBQUEsR0FBQSxLQUFBLE1BQUE7TUFBQSxTQUFTLFNBQVM7TUFFbEIsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7UUNwQkUsT0FBTyxLQUFLO1FEcUJaLElBQWUsS0FBSyxPQUFNLFFBQTFCO1VBQUEsT0FBTzs7UUFDUCxJQUE4QyxLQUFLLGVBQW5EO1VBQUEsTUFBTSxTQUFTLFFBQVEsS0FBSzs7UUFDNUIsSUFBYyxLQUFkO1VBQUEsT0FBTzs7O01DWlQsT0RjQTs7SUFFRixXQUFXLEdBQUc7SUFVZCxHQUFHLElBQUksQ0FBQyxVQUFVLEtBQUssU0FBUyxVQUFVLElBQUksVUFBVSxLQUFLLENBQUEsU0FBQSxPQUFBO01DdEIzRCxPRHNCMkQsU0FBQyxNQUFEO1FBQzNELElBQUE7UUFBQSxZQUFZLFNBQVMsUUFBUSxZQUFZO1FDcEJ2QyxPRHVCRixNQUFDLFVBQVUsV0FBVyxLQUFLLFdBQVcsS0FBSyxjQUFjLEdBQUcsZUFBZSxLQUFLLFNBQUMsUUFBRDtVQUM5RSxVQUFVLFNBQVM7VUN0QmpCLE9EdUJGLFNBQVMsUUFBUTs7O09BTndDO0lDYjdELE9EcUJBLFNBQVM7O0VBR1gsS0FBQyxZQUFZLFNBQUMsT0FBTyxVQUFSO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxZQUFZLFlBQVksbUNBQW1DLFFBQVEsa0JBQWtCLFVBQzlGLFFBQVEsU0FBQyxNQUFEO01DdEJQLE9EdUJBLFNBQVMsUUFBUTs7SUNyQm5CLE9EdUJBLFNBQVM7O0VDckJYLE9EdUJBOztBQ3JCRjtBQ3pJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLG1FQUFzQixTQUFDLFFBQVEsaUJBQWlCLGFBQTFCO0VBQ2hDLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNuQnpDLE9Eb0JBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87O0FDbEJUO0FDT0EsUUFBUSxPQUFPLFlBRWQsUUFBUSxvREFBbUIsU0FBQyxPQUFPLGFBQWEsTUFBckI7RUFDMUIsSUFBQTtFQUFBLGVBQWU7RUFFZixLQUFDLG1CQUFtQixXQUFBO0lBQ2xCLE1BQU0sSUFBSSxZQUFZLFlBQVksbUJBQ2pDLFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQ3BCUCxPRHFCQSxLQUFLO09BRU4sTUFBTSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO0lDckJQLE9Ed0JBOztFQ3RCRixPRHdCQTs7QUN0QkYiLCJmaWxlIjoiaW5kZXguanMiLCJzb3VyY2VzQ29udGVudCI6WyIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJywgWyd1aS5yb3V0ZXInLCAnYW5ndWxhck1vbWVudCddKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5ydW4gKCRyb290U2NvcGUpIC0+XG4gICRyb290U2NvcGUuc2lkZWJhclZpc2libGUgPSBmYWxzZVxuICAkcm9vdFNjb3BlLnNob3dTaWRlYmFyID0gLT5cbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJDbGFzcyA9ICdmb3JjZS1zaG93J1xuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5jb25zdGFudCAnZmxpbmtDb25maWcnLCB7XG4gIGpvYlNlcnZlcjogJ2h0dHA6Ly9sb2NhbGhvc3Q6ODA4MSdcbiAgbmV3U2VydmVyOiAnaHR0cDovL2xvY2FsaG9zdDo4MDgxJ1xuIyAgam9iU2VydmVyOiAnaHR0cDovL2xvY2FsaG9zdDozMDAwL25ldy1zZXJ2ZXInXG4jICBuZXdTZXJ2ZXI6ICdodHRwOi8vbG9jYWxob3N0OjMwMDAvbmV3LXNlcnZlcidcbiAgcmVmcmVzaEludGVydmFsOiAxMDAwMFxufVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5ydW4gKEpvYnNTZXJ2aWNlLCBmbGlua0NvbmZpZywgJGludGVydmFsKSAtPlxuICBKb2JzU2VydmljZS5saXN0Sm9icygpXG5cbiAgJGludGVydmFsIC0+XG4gICAgSm9ic1NlcnZpY2UubGlzdEpvYnMoKVxuICAsIGZsaW5rQ29uZmlnLnJlZnJlc2hJbnRlcnZhbFxuXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbmZpZyAoJHN0YXRlUHJvdmlkZXIsICR1cmxSb3V0ZXJQcm92aWRlcikgLT5cbiAgJHN0YXRlUHJvdmlkZXIuc3RhdGUgXCJvdmVydmlld1wiLFxuICAgIHVybDogXCIvb3ZlcnZpZXdcIlxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvb3ZlcnZpZXcuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwicnVubmluZy1qb2JzXCIsXG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIlxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9ydW5uaW5nLWpvYnMuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdSdW5uaW5nSm9ic0NvbnRyb2xsZXInXG4gIFxuICAuc3RhdGUgXCJjb21wbGV0ZWQtam9ic1wiLFxuICAgIHVybDogXCIvY29tcGxldGVkLWpvYnNcIlxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9jb21wbGV0ZWQtam9icy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0NvbXBsZXRlZEpvYnNDb250cm9sbGVyJ1xuXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2JcIixcbiAgICB1cmw6IFwiL2pvYnMve2pvYmlkfVwiXG4gICAgYWJzdHJhY3Q6IHRydWVcbiAgICB2aWV3czpcbiAgICAgIG1haW46XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnU2luZ2xlSm9iQ29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW5cIixcbiAgICB1cmw6IFwiXCJcbiAgICB2aWV3czpcbiAgICAgIGRldGFpbHM6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuQ29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4ubm9kZVwiLFxuICAgIHVybDogXCIve25vZGVpZDppbnR9XCJcbiAgICB2aWV3czpcbiAgICAgIG5vZGU6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Ob2RlQ29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXG4gICAgdXJsOiBcIi90aW1lbGluZVwiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi50aW1lbGluZS5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLFxuICAgIHVybDogXCIve3ZlcnRleElkfVwiXG4gICAgdmlld3M6XG4gICAgICB2ZXJ0ZXg6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLnZlcnRleC5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnN0YXRpc3RpY3NcIixcbiAgICB1cmw6IFwiL3N0YXRpc3RpY3NcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2Iuc3RhdGlzdGljcy5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLmV4Y2VwdGlvbnNcIixcbiAgICB1cmw6IFwiL2V4Y2VwdGlvbnNcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IuZXhjZXB0aW9ucy5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLmNvbmZpZ1wiLFxuICAgIHVybDogXCIvY29uZmlnXCJcbiAgICB2aWV3czpcbiAgICAgIGRldGFpbHM6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcblxuICAkdXJsUm91dGVyUHJvdmlkZXIub3RoZXJ3aXNlIFwiL292ZXJ2aWV3XCJcbiIsImFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSkucnVuKGZ1bmN0aW9uKCRyb290U2NvcGUpIHtcbiAgJHJvb3RTY29wZS5zaWRlYmFyVmlzaWJsZSA9IGZhbHNlO1xuICByZXR1cm4gJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IGZ1bmN0aW9uKCkge1xuICAgICRyb290U2NvcGUuc2lkZWJhclZpc2libGUgPSAhJHJvb3RTY29wZS5zaWRlYmFyVmlzaWJsZTtcbiAgICByZXR1cm4gJHJvb3RTY29wZS5zaWRlYmFyQ2xhc3MgPSAnZm9yY2Utc2hvdyc7XG4gIH07XG59KS5jb25zdGFudCgnZmxpbmtDb25maWcnLCB7XG4gIGpvYlNlcnZlcjogJ2h0dHA6Ly9sb2NhbGhvc3Q6ODA4MScsXG4gIG5ld1NlcnZlcjogJ2h0dHA6Ly9sb2NhbGhvc3Q6ODA4MScsXG4gIHJlZnJlc2hJbnRlcnZhbDogMTAwMDBcbn0pLnJ1bihmdW5jdGlvbihKb2JzU2VydmljZSwgZmxpbmtDb25maWcsICRpbnRlcnZhbCkge1xuICBKb2JzU2VydmljZS5saXN0Sm9icygpO1xuICByZXR1cm4gJGludGVydmFsKGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiBKb2JzU2VydmljZS5saXN0Sm9icygpO1xuICB9LCBmbGlua0NvbmZpZy5yZWZyZXNoSW50ZXJ2YWwpO1xufSkuY29uZmlnKGZ1bmN0aW9uKCRzdGF0ZVByb3ZpZGVyLCAkdXJsUm91dGVyUHJvdmlkZXIpIHtcbiAgJHN0YXRlUHJvdmlkZXIuc3RhdGUoXCJvdmVydmlld1wiLCB7XG4gICAgdXJsOiBcIi9vdmVydmlld1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ092ZXJ2aWV3Q29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwicnVubmluZy1qb2JzXCIsIHtcbiAgICB1cmw6IFwiL3J1bm5pbmctam9ic1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvcnVubmluZy1qb2JzLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwiY29tcGxldGVkLWpvYnNcIiwge1xuICAgIHVybDogXCIvY29tcGxldGVkLWpvYnNcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0NvbXBsZXRlZEpvYnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iXCIsIHtcbiAgICB1cmw6IFwiL2pvYnMve2pvYmlkfVwiLFxuICAgIGFic3RyYWN0OiB0cnVlLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ1NpbmdsZUpvYkNvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IucGxhblwiLCB7XG4gICAgdXJsOiBcIlwiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkNvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IucGxhbi5ub2RlXCIsIHtcbiAgICB1cmw6IFwiL3tub2RlaWQ6aW50fVwiLFxuICAgIHZpZXdzOiB7XG4gICAgICBub2RlOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuTm9kZUNvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IudGltZWxpbmVcIiwge1xuICAgIHVybDogXCIvdGltZWxpbmVcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi50aW1lbGluZS5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi50aW1lbGluZS52ZXJ0ZXhcIiwge1xuICAgIHVybDogXCIve3ZlcnRleElkfVwiLFxuICAgIHZpZXdzOiB7XG4gICAgICB2ZXJ0ZXg6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4Lmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5zdGF0aXN0aWNzXCIsIHtcbiAgICB1cmw6IFwiL3N0YXRpc3RpY3NcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5zdGF0aXN0aWNzLmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLmV4Y2VwdGlvbnNcIiwge1xuICAgIHVybDogXCIvZXhjZXB0aW9uc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IuY29uZmlnXCIsIHtcbiAgICB1cmw6IFwiL2NvbmZpZ1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pO1xuICByZXR1cm4gJHVybFJvdXRlclByb3ZpZGVyLm90aGVyd2lzZShcIi9vdmVydmlld1wiKTtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uZGlyZWN0aXZlICdic0xhYmVsJywgKEpvYnNTZXJ2aWNlKSAtPlxuICB0cmFuc2NsdWRlOiB0cnVlXG4gIHJlcGxhY2U6IHRydWVcbiAgc2NvcGU6IFxuICAgIGdldExhYmVsQ2xhc3M6IFwiJlwiXG4gICAgc3RhdHVzOiBcIkBcIlxuXG4gIHRlbXBsYXRlOiBcIjxzcGFuIHRpdGxlPSd7e3N0YXR1c319JyBuZy1jbGFzcz0nZ2V0TGFiZWxDbGFzcygpJz48bmctdHJhbnNjbHVkZT48L25nLXRyYW5zY2x1ZGU+PC9zcGFuPlwiXG4gIFxuICBsaW5rOiAoc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSAtPlxuICAgIHNjb3BlLmdldExhYmVsQ2xhc3MgPSAtPlxuICAgICAgJ2xhYmVsIGxhYmVsLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ2luZGljYXRvclByaW1hcnknLCAoSm9ic1NlcnZpY2UpIC0+XG4gIHJlcGxhY2U6IHRydWVcbiAgc2NvcGU6IFxuICAgIGdldExhYmVsQ2xhc3M6IFwiJlwiXG4gICAgc3RhdHVzOiAnQCdcblxuICB0ZW1wbGF0ZTogXCI8aSB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKScgLz5cIlxuICBcbiAgbGluazogKHNjb3BlLCBlbGVtZW50LCBhdHRycykgLT5cbiAgICBzY29wZS5nZXRMYWJlbENsYXNzID0gLT5cbiAgICAgICdmYSBmYS1jaXJjbGUgaW5kaWNhdG9yIGluZGljYXRvci0nICsgSm9ic1NlcnZpY2UudHJhbnNsYXRlTGFiZWxTdGF0ZShhdHRycy5zdGF0dXMpXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uZGlyZWN0aXZlICd0YWJsZVByb3BlcnR5JywgLT5cbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTpcbiAgICB2YWx1ZTogJz0nXG5cbiAgdGVtcGxhdGU6IFwiPHRkIHRpdGxlPVxcXCJ7e3ZhbHVlIHx8ICdOb25lJ319XFxcIj57e3ZhbHVlIHx8ICdOb25lJ319PC90ZD5cIlxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuZGlyZWN0aXZlKCdic0xhYmVsJywgZnVuY3Rpb24oSm9ic1NlcnZpY2UpIHtcbiAgcmV0dXJuIHtcbiAgICB0cmFuc2NsdWRlOiB0cnVlLFxuICAgIHJlcGxhY2U6IHRydWUsXG4gICAgc2NvcGU6IHtcbiAgICAgIGdldExhYmVsQ2xhc3M6IFwiJlwiLFxuICAgICAgc3RhdHVzOiBcIkBcIlxuICAgIH0sXG4gICAgdGVtcGxhdGU6IFwiPHNwYW4gdGl0bGU9J3t7c3RhdHVzfX0nIG5nLWNsYXNzPSdnZXRMYWJlbENsYXNzKCknPjxuZy10cmFuc2NsdWRlPjwvbmctdHJhbnNjbHVkZT48L3NwYW4+XCIsXG4gICAgbGluazogZnVuY3Rpb24oc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSB7XG4gICAgICByZXR1cm4gc2NvcGUuZ2V0TGFiZWxDbGFzcyA9IGZ1bmN0aW9uKCkge1xuICAgICAgICByZXR1cm4gJ2xhYmVsIGxhYmVsLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cyk7XG4gICAgICB9O1xuICAgIH1cbiAgfTtcbn0pLmRpcmVjdGl2ZSgnaW5kaWNhdG9yUHJpbWFyeScsIGZ1bmN0aW9uKEpvYnNTZXJ2aWNlKSB7XG4gIHJldHVybiB7XG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgZ2V0TGFiZWxDbGFzczogXCImXCIsXG4gICAgICBzdGF0dXM6ICdAJ1xuICAgIH0sXG4gICAgdGVtcGxhdGU6IFwiPGkgdGl0bGU9J3t7c3RhdHVzfX0nIG5nLWNsYXNzPSdnZXRMYWJlbENsYXNzKCknIC8+XCIsXG4gICAgbGluazogZnVuY3Rpb24oc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSB7XG4gICAgICByZXR1cm4gc2NvcGUuZ2V0TGFiZWxDbGFzcyA9IGZ1bmN0aW9uKCkge1xuICAgICAgICByZXR1cm4gJ2ZhIGZhLWNpcmNsZSBpbmRpY2F0b3IgaW5kaWNhdG9yLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cyk7XG4gICAgICB9O1xuICAgIH1cbiAgfTtcbn0pLmRpcmVjdGl2ZSgndGFibGVQcm9wZXJ0eScsIGZ1bmN0aW9uKCkge1xuICByZXR1cm4ge1xuICAgIHJlcGxhY2U6IHRydWUsXG4gICAgc2NvcGU6IHtcbiAgICAgIHZhbHVlOiAnPSdcbiAgICB9LFxuICAgIHRlbXBsYXRlOiBcIjx0ZCB0aXRsZT1cXFwie3t2YWx1ZSB8fCAnTm9uZSd9fVxcXCI+e3t2YWx1ZSB8fCAnTm9uZSd9fTwvdGQ+XCJcbiAgfTtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbi5maWx0ZXIgXCJhbUR1cmF0aW9uRm9ybWF0RXh0ZW5kZWRcIiwgKGFuZ3VsYXJNb21lbnRDb25maWcpIC0+XG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlciA9ICh2YWx1ZSwgZm9ybWF0LCBkdXJhdGlvbkZvcm1hdCkgLT5cbiAgICByZXR1cm4gXCJcIiAgaWYgdHlwZW9mIHZhbHVlIGlzIFwidW5kZWZpbmVkXCIgb3IgdmFsdWUgaXMgbnVsbFxuXG4gICAgbW9tZW50LmR1cmF0aW9uKHZhbHVlLCBmb3JtYXQpLmZvcm1hdChkdXJhdGlvbkZvcm1hdCwgeyB0cmltOiBmYWxzZSB9KVxuXG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlci4kc3RhdGVmdWwgPSBhbmd1bGFyTW9tZW50Q29uZmlnLnN0YXRlZnVsRmlsdGVyc1xuXG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlclxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuZmlsdGVyKFwiYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkXCIsIGZ1bmN0aW9uKGFuZ3VsYXJNb21lbnRDb25maWcpIHtcbiAgdmFyIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlcjtcbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyID0gZnVuY3Rpb24odmFsdWUsIGZvcm1hdCwgZHVyYXRpb25Gb3JtYXQpIHtcbiAgICBpZiAodHlwZW9mIHZhbHVlID09PSBcInVuZGVmaW5lZFwiIHx8IHZhbHVlID09PSBudWxsKSB7XG4gICAgICByZXR1cm4gXCJcIjtcbiAgICB9XG4gICAgcmV0dXJuIG1vbWVudC5kdXJhdGlvbih2YWx1ZSwgZm9ybWF0KS5mb3JtYXQoZHVyYXRpb25Gb3JtYXQsIHtcbiAgICAgIHRyaW06IGZhbHNlXG4gICAgfSk7XG4gIH07XG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlci4kc3RhdGVmdWwgPSBhbmd1bGFyTW9tZW50Q29uZmlnLnN0YXRlZnVsRmlsdGVycztcbiAgcmV0dXJuIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlcjtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbi5jb250cm9sbGVyICdSdW5uaW5nSm9ic0NvbnRyb2xsZXInLCAoJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIC0+XG4gICRzY29wZS5qb2JPYnNlcnZlciA9IC0+XG4gICAgJHNjb3BlLmpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdydW5uaW5nJylcblxuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcbiAgJHNjb3BlLiRvbiAnJGRlc3Ryb3knLCAtPlxuICAgIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpXG5cbiAgJHNjb3BlLmpvYk9ic2VydmVyKClcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29udHJvbGxlciAnQ29tcGxldGVkSm9ic0NvbnRyb2xsZXInLCAoJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIC0+XG4gICRzY29wZS5qb2JPYnNlcnZlciA9IC0+XG4gICAgJHNjb3BlLmpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdmaW5pc2hlZCcpXG5cbiAgSm9ic1NlcnZpY2UucmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpXG4gICRzY29wZS4kb24gJyRkZXN0cm95JywgLT5cbiAgICBKb2JzU2VydmljZS51blJlZ2lzdGVyT2JzZXJ2ZXIoJHNjb3BlLmpvYk9ic2VydmVyKVxuXG4gICRzY29wZS5qb2JPYnNlcnZlcigpXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbnRyb2xsZXIgJ1NpbmdsZUpvYkNvbnRyb2xsZXInLCAoJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UsICRyb290U2NvcGUpIC0+XG4gICRzY29wZS5qb2JpZCA9ICRzdGF0ZVBhcmFtcy5qb2JpZFxuICAkcm9vdFNjb3BlLmpvYiA9IG51bGxcblxuICBKb2JzU2VydmljZS5sb2FkSm9iKCRzdGF0ZVBhcmFtcy5qb2JpZCkudGhlbiAoZGF0YSkgLT5cbiAgICAkcm9vdFNjb3BlLmpvYiA9IGRhdGFcblxuICAkc2NvcGUuJG9uICckZGVzdHJveScsIC0+XG4gICAgJHJvb3RTY29wZS5qb2IgPSBudWxsXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbnRyb2xsZXIgJ0pvYlBsYW5Db250cm9sbGVyJywgKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSAtPlxuICBKb2JzU2VydmljZS5sb2FkUGxhbigkc3RhdGVQYXJhbXMuam9iaWQpLnRoZW4gKGRhdGEpIC0+XG4gICAgJHNjb3BlLnBsYW4gPSBkYXRhXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbnRyb2xsZXIgJ0pvYlBsYW5Ob2RlQ29udHJvbGxlcicsICgkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkgLT5cbiAgJHNjb3BlLm5vZGVpZCA9ICRzdGF0ZVBhcmFtcy5ub2RlaWRcbiAgJHNjb3BlLnN0YXRlTGlzdCA9IEpvYnNTZXJ2aWNlLnN0YXRlTGlzdCgpXG5cbiAgSm9ic1NlcnZpY2UuZ2V0Tm9kZSgkc2NvcGUubm9kZWlkKS50aGVuIChkYXRhKSAtPlxuICAgICRzY29wZS5ub2RlID0gZGF0YVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5jb250cm9sbGVyICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInLCAoJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIC0+XG4gIEpvYnNTZXJ2aWNlLmdldFZlcnRleCgkc3RhdGVQYXJhbXMuam9iaWQsICRzdGF0ZVBhcmFtcy52ZXJ0ZXhJZCkudGhlbiAoZGF0YSkgLT5cbiAgICAkc2NvcGUudmVydGV4ID0gZGF0YVxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuY29udHJvbGxlcignUnVubmluZ0pvYnNDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIHtcbiAgJHNjb3BlLmpvYk9ic2VydmVyID0gZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuICRzY29wZS5qb2JzID0gSm9ic1NlcnZpY2UuZ2V0Sm9icygncnVubmluZycpO1xuICB9O1xuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5qb2JPYnNlcnZlcigpO1xufSkuY29udHJvbGxlcignQ29tcGxldGVkSm9ic0NvbnRyb2xsZXInLCBmdW5jdGlvbigkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkge1xuICAkc2NvcGUuam9iT2JzZXJ2ZXIgPSBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gJHNjb3BlLmpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdmaW5pc2hlZCcpO1xuICB9O1xuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5qb2JPYnNlcnZlcigpO1xufSkuY29udHJvbGxlcignU2luZ2xlSm9iQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlLCAkcm9vdFNjb3BlKSB7XG4gICRzY29wZS5qb2JpZCA9ICRzdGF0ZVBhcmFtcy5qb2JpZDtcbiAgJHJvb3RTY29wZS5qb2IgPSBudWxsO1xuICBKb2JzU2VydmljZS5sb2FkSm9iKCRzdGF0ZVBhcmFtcy5qb2JpZCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgcmV0dXJuICRyb290U2NvcGUuam9iID0gZGF0YTtcbiAgfSk7XG4gIHJldHVybiAkc2NvcGUuJG9uKCckZGVzdHJveScsIGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiAkcm9vdFNjb3BlLmpvYiA9IG51bGw7XG4gIH0pO1xufSkuY29udHJvbGxlcignSm9iUGxhbkNvbnRyb2xsZXInLCBmdW5jdGlvbigkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkge1xuICByZXR1cm4gSm9ic1NlcnZpY2UubG9hZFBsYW4oJHN0YXRlUGFyYW1zLmpvYmlkKS50aGVuKGZ1bmN0aW9uKGRhdGEpIHtcbiAgICByZXR1cm4gJHNjb3BlLnBsYW4gPSBkYXRhO1xuICB9KTtcbn0pLmNvbnRyb2xsZXIoJ0pvYlBsYW5Ob2RlQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSB7XG4gICRzY29wZS5ub2RlaWQgPSAkc3RhdGVQYXJhbXMubm9kZWlkO1xuICAkc2NvcGUuc3RhdGVMaXN0ID0gSm9ic1NlcnZpY2Uuc3RhdGVMaXN0KCk7XG4gIHJldHVybiBKb2JzU2VydmljZS5nZXROb2RlKCRzY29wZS5ub2RlaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgIHJldHVybiAkc2NvcGUubm9kZSA9IGRhdGE7XG4gIH0pO1xufSkuY29udHJvbGxlcignSm9iVGltZWxpbmVWZXJ0ZXhDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIHtcbiAgcmV0dXJuIEpvYnNTZXJ2aWNlLmdldFZlcnRleCgkc3RhdGVQYXJhbXMuam9iaWQsICRzdGF0ZVBhcmFtcy52ZXJ0ZXhJZCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgcmV0dXJuICRzY29wZS52ZXJ0ZXggPSBkYXRhO1xuICB9KTtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uZGlyZWN0aXZlICd2ZXJ0ZXgnLCAoJHN0YXRlKSAtPlxuICB0ZW1wbGF0ZTogXCI8c3ZnIGNsYXNzPSd0aW1lbGluZSBzZWNvbmRhcnknIHdpZHRoPScwJyBoZWlnaHQ9JzAnPjwvc3ZnPlwiXG5cbiAgc2NvcGU6XG4gICAgZGF0YTogXCI9XCJcblxuICBsaW5rOiAoc2NvcGUsIGVsZW0sIGF0dHJzKSAtPlxuICAgIHpvb20gPSBkMy5iZWhhdmlvci56b29tKClcbiAgICBzdmdFbCA9IGVsZW0uY2hpbGRyZW4oKVswXVxuXG4gICAgY29udGFpbmVyVyA9IGVsZW0ud2lkdGgoKVxuICAgIGFuZ3VsYXIuZWxlbWVudChzdmdFbCkuYXR0cignd2lkdGgnLCBjb250YWluZXJXIC0gMTYpXG5cbiAgICBhbmFseXplVGltZSA9IChkYXRhKSAtPlxuICAgICAgdGVzdERhdGEgPSBbXVxuXG4gICAgICBhbmd1bGFyLmZvckVhY2ggZGF0YS5ncm91cHZlcnRleC5ncm91cG1lbWJlcnMsICh2ZXJ0ZXgsIGkpIC0+XG4gICAgICAgIHZUaW1lID0gZGF0YS52ZXJ0aWNldGltZXNbdmVydGV4LnZlcnRleGlkXVxuXG4gICAgICAgIHRlc3REYXRhLnB1c2gge1xuICAgICAgICAgIGxhYmVsOiBcIiN7dmVydGV4LnZlcnRleGluc3RhbmNlbmFtZX0gKCN7aX0pXCJcbiAgICAgICAgICB0aW1lczogW1xuICAgICAgICAgICAge1xuICAgICAgICAgICAgICBsYWJlbDogXCJTY2hlZHVsZWRcIlxuICAgICAgICAgICAgICBjb2xvcjogXCIjNjY2XCJcbiAgICAgICAgICAgICAgc3RhcnRpbmdfdGltZTogdlRpbWVbXCJTQ0hFRFVMRURcIl0gKiAxMDBcbiAgICAgICAgICAgICAgZW5kaW5nX3RpbWU6IHZUaW1lW1wiREVQTE9ZSU5HXCJdICogMTAwXG4gICAgICAgICAgICB9XG4gICAgICAgICAgICB7XG4gICAgICAgICAgICAgIGxhYmVsOiBcIkRlcGxveWluZ1wiXG4gICAgICAgICAgICAgIGNvbG9yOiBcIiNhYWFcIlxuICAgICAgICAgICAgICBzdGFydGluZ190aW1lOiB2VGltZVtcIkRFUExPWUlOR1wiXSAqIDEwMFxuICAgICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJSVU5OSU5HXCJdICogMTAwXG4gICAgICAgICAgICB9XG4gICAgICAgICAgICB7XG4gICAgICAgICAgICAgIGxhYmVsOiBcIlJ1bm5pbmdcIlxuICAgICAgICAgICAgICBjb2xvcjogXCIjZGRkXCJcbiAgICAgICAgICAgICAgc3RhcnRpbmdfdGltZTogdlRpbWVbXCJSVU5OSU5HXCJdICogMTAwXG4gICAgICAgICAgICAgIGVuZGluZ190aW1lOiB2VGltZVtcIkZJTklTSEVEXCJdICogMTAwXG4gICAgICAgICAgICB9XG4gICAgICAgICAgXVxuICAgICAgICB9XG5cbiAgICAgIGNoYXJ0ID0gZDMudGltZWxpbmUoKS5zdGFjaygpLnRpY2tGb3JtYXQoe1xuICAgICAgICBmb3JtYXQ6IGQzLnRpbWUuZm9ybWF0KFwiJVNcIiksXG4gICAgICAgICMgdGlja1RpbWU6IGQzLnRpbWUubWlsbGlzZWNvbmRzLFxuICAgICAgICB0aWNrSW50ZXJ2YWw6IDEsXG4gICAgICAgIHRpY2tTaXplOiAxXG4gICAgICB9KS5sYWJlbEZvcm1hdCgobGFiZWwpIC0+XG4gICAgICAgIGxhYmVsXG4gICAgICApLm1hcmdpbih7IGxlZnQ6IDEwMCwgcmlnaHQ6IDAsIHRvcDogMCwgYm90dG9tOiAwIH0pXG5cbiAgICAgIHN2ZyA9IGQzLnNlbGVjdChzdmdFbClcbiAgICAgIC5kYXR1bSh0ZXN0RGF0YSlcbiAgICAgIC5jYWxsKGNoYXJ0KVxuICAgICAgLmNhbGwoem9vbSlcblxuICAgICAgc3ZnRyA9IHN2Zy5zZWxlY3QoXCJnXCIpXG5cbiAgICAgIHpvb20ub24oXCJ6b29tXCIsIC0+XG4gICAgICAgIGV2ID0gZDMuZXZlbnRcblxuICAgICAgICBzdmdHLnNlbGVjdEFsbCgncmVjdCcpLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGVbMF0gKyBcIiwwKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIsMSlcIilcbiAgICAgICAgc3ZnRy5zZWxlY3RBbGwoJ3RleHQnKS5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlWzBdICsgXCIsMCkgc2NhbGUoXCIgKyBldi5zY2FsZSArIFwiLDEpXCIpXG4gICAgICApXG5cbiAgICAgIGJib3ggPSBzdmdHWzBdWzBdLmdldEJCb3goKVxuICAgICAgc3ZnLmF0dHIoJ2hlaWdodCcsIGJib3guaGVpZ2h0ICsgMzApXG5cbiAgICBhbmFseXplVGltZShzY29wZS5kYXRhKVxuXG4gICAgcmV0dXJuXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uZGlyZWN0aXZlICd0aW1lbGluZScsICgkc3RhdGUpIC0+XG4gIHRlbXBsYXRlOiBcIjxzdmcgY2xhc3M9J3RpbWVsaW5lJyB3aWR0aD0nMCcgaGVpZ2h0PScwJz48L3N2Zz5cIlxuXG4gIHNjb3BlOlxuICAgIGpvYjogXCI9XCJcblxuICBsaW5rOiAoc2NvcGUsIGVsZW0sIGF0dHJzKSAtPlxuICAgIHpvb20gPSBkMy5iZWhhdmlvci56b29tKClcbiAgICBzdmdFbCA9IGVsZW0uY2hpbGRyZW4oKVswXVxuXG4gICAgY29udGFpbmVyVyA9IGVsZW0ud2lkdGgoKVxuICAgIGFuZ3VsYXIuZWxlbWVudChzdmdFbCkuYXR0cignd2lkdGgnLCBjb250YWluZXJXIC0gMTYpXG5cbiAgICBhbmFseXplVGltZSA9IChkYXRhKSAtPlxuICAgICAgdGVzdERhdGEgPSBbXVxuXG4gICAgICBhbmd1bGFyLmZvckVhY2ggZGF0YS5vbGRWLmdyb3VwdmVydGljZXMsICh2ZXJ0ZXgpIC0+XG4gICAgICAgIHZUaW1lID0gZGF0YS5vbGRWLmdyb3VwdmVydGljZXRpbWVzW3ZlcnRleC5ncm91cHZlcnRleGlkXVxuXG4gICAgICAgICMgY29uc29sZS5sb2cgdlRpbWUsIHZlcnRleC5ncm91cHZlcnRleGlkXG5cbiAgICAgICAgdGVzdERhdGEucHVzaCBcbiAgICAgICAgICB0aW1lczogW1xuICAgICAgICAgICAgbGFiZWw6IHZlcnRleC5ncm91cHZlcnRleG5hbWVcbiAgICAgICAgICAgIGNvbG9yOiBcIiMzZmI2ZDhcIlxuICAgICAgICAgICAgc3RhcnRpbmdfdGltZTogdlRpbWVbXCJTVEFSVEVEXCJdXG4gICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJFTkRFRFwiXVxuICAgICAgICAgICAgbGluazogdmVydGV4Lmdyb3VwdmVydGV4aWRcbiAgICAgICAgICBdXG5cbiAgICAgIGNoYXJ0ID0gZDMudGltZWxpbmUoKS5zdGFjaygpLmNsaWNrKChkLCBpLCBkYXR1bSkgLT5cbiAgICAgICAgJHN0YXRlLmdvIFwic2luZ2xlLWpvYi50aW1lbGluZS52ZXJ0ZXhcIiwgeyBqb2JpZDogZGF0YS5qaWQsIHZlcnRleElkOiBkLmxpbmsgfVxuXG4gICAgICApLnRpY2tGb3JtYXQoe1xuICAgICAgICBmb3JtYXQ6IGQzLnRpbWUuZm9ybWF0KFwiJVNcIilcbiAgICAgICAgIyB0aWNrVGltZTogZDMudGltZS5taWxsaXNlY29uZHNcbiAgICAgICAgdGlja0ludGVydmFsOiAxXG4gICAgICAgIHRpY2tTaXplOiAxXG4gICAgICB9KS5tYXJnaW4oeyBsZWZ0OiAwLCByaWdodDogMCwgdG9wOiAwLCBib3R0b206IDAgfSlcblxuICAgICAgc3ZnID0gZDMuc2VsZWN0KHN2Z0VsKVxuICAgICAgLmRhdHVtKHRlc3REYXRhKVxuICAgICAgLmNhbGwoY2hhcnQpXG4gICAgICAuY2FsbCh6b29tKVxuXG4gICAgICBzdmdHID0gc3ZnLnNlbGVjdChcImdcIilcblxuICAgICAgem9vbS5vbihcInpvb21cIiwgLT5cbiAgICAgICAgZXYgPSBkMy5ldmVudFxuXG4gICAgICAgIHN2Z0cuc2VsZWN0QWxsKCdyZWN0JykuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIGV2LnRyYW5zbGF0ZVswXSArIFwiLDApIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIiwxKVwiKVxuICAgICAgICBzdmdHLnNlbGVjdEFsbCgndGV4dCcpLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGVbMF0gKyBcIiwwKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIsMSlcIilcbiAgICAgIClcblxuICAgICAgYmJveCA9IHN2Z0dbMF1bMF0uZ2V0QkJveCgpXG4gICAgICBzdmcuYXR0cignaGVpZ2h0JywgYmJveC5oZWlnaHQgKyAzMClcblxuICAgIHNjb3BlLiR3YXRjaCBhdHRycy5qb2IsIChkYXRhKSAtPlxuICAgICAgYW5hbHl6ZVRpbWUoZGF0YSkgaWYgZGF0YVxuXG4gICAgcmV0dXJuXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uZGlyZWN0aXZlICdqb2JQbGFuJywgKCR0aW1lb3V0KSAtPlxuICB0ZW1wbGF0ZTogXCJcbiAgICA8c3ZnIGNsYXNzPSdncmFwaCcgd2lkdGg9JzUwMCcgaGVpZ2h0PSc0MDAnPjxnIC8+PC9zdmc+XG4gICAgPHN2ZyBjbGFzcz0ndG1wJyB3aWR0aD0nMScgaGVpZ2h0PScxJz48ZyAvPjwvc3ZnPlxuICAgIDxkaXYgY2xhc3M9J2J0bi1ncm91cCB6b29tLWJ1dHRvbnMnPlxuICAgICAgPGEgY2xhc3M9J2J0biBidG4tZGVmYXVsdCB6b29tLWluJyBuZy1jbGljaz0nem9vbUluKCknPjxpIGNsYXNzPSdmYSBmYS1wbHVzJyAvPjwvYT5cbiAgICAgIDxhIGNsYXNzPSdidG4gYnRuLWRlZmF1bHQgem9vbS1vdXQnIG5nLWNsaWNrPSd6b29tT3V0KCknPjxpIGNsYXNzPSdmYSBmYS1taW51cycgLz48L2E+XG4gICAgPC9kaXY+XCJcblxuICBzY29wZTpcbiAgICBwbGFuOiAnPSdcblxuICBsaW5rOiAoc2NvcGUsIGVsZW0sIGF0dHJzKSAtPlxuICAgIG1haW5ab29tID0gZDMuYmVoYXZpb3Iuem9vbSgpXG4gICAgc3ViZ3JhcGhzID0gW11cbiAgICBqb2JpZCA9IGF0dHJzLmpvYmlkXG5cbiAgICBtYWluU3ZnRWxlbWVudCA9IGVsZW0uY2hpbGRyZW4oKVswXVxuICAgIG1haW5HID0gZWxlbS5jaGlsZHJlbigpLmNoaWxkcmVuKClbMF1cbiAgICBtYWluVG1wRWxlbWVudCA9IGVsZW0uY2hpbGRyZW4oKVsxXVxuXG4gICAgZDNtYWluU3ZnID0gZDMuc2VsZWN0KG1haW5TdmdFbGVtZW50KVxuICAgIGQzbWFpblN2Z0cgPSBkMy5zZWxlY3QobWFpbkcpXG4gICAgZDN0bXBTdmcgPSBkMy5zZWxlY3QobWFpblRtcEVsZW1lbnQpXG5cbiAgICAjIGFuZ3VsYXIuZWxlbWVudChtYWluRykuZW1wdHkoKVxuXG4gICAgY29udGFpbmVyVyA9IGVsZW0ud2lkdGgoKVxuICAgIGFuZ3VsYXIuZWxlbWVudChlbGVtLmNoaWxkcmVuKClbMF0pLndpZHRoKGNvbnRhaW5lclcpXG5cbiAgICBzY29wZS56b29tSW4gPSAtPlxuICAgICAgaWYgbWFpblpvb20uc2NhbGUoKSA8IDIuOTlcbiAgICAgICAgXG4gICAgICAgICMgQ2FsY3VsYXRlIGFuZCBzdG9yZSBuZXcgdmFsdWVzIGluIHpvb20gb2JqZWN0XG4gICAgICAgIHRyYW5zbGF0ZSA9IG1haW5ab29tLnRyYW5zbGF0ZSgpXG4gICAgICAgIHYxID0gdHJhbnNsYXRlWzBdICogKG1haW5ab29tLnNjYWxlKCkgKyAwLjEgLyAobWFpblpvb20uc2NhbGUoKSkpXG4gICAgICAgIHYyID0gdHJhbnNsYXRlWzFdICogKG1haW5ab29tLnNjYWxlKCkgKyAwLjEgLyAobWFpblpvb20uc2NhbGUoKSkpXG4gICAgICAgIG1haW5ab29tLnNjYWxlIG1haW5ab29tLnNjYWxlKCkgKyAwLjFcbiAgICAgICAgbWFpblpvb20udHJhbnNsYXRlIFsgdjEsIHYyIF1cbiAgICAgICAgXG4gICAgICAgICMgVHJhbnNmb3JtIHN2Z1xuICAgICAgICBkM21haW5TdmdHLmF0dHIgXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyB2MSArIFwiLFwiICsgdjIgKyBcIikgc2NhbGUoXCIgKyBtYWluWm9vbS5zY2FsZSgpICsgXCIpXCJcblxuICAgIHNjb3BlLnpvb21PdXQgPSAtPlxuICAgICAgaWYgbWFpblpvb20uc2NhbGUoKSA+IDAuMzFcbiAgICAgICAgXG4gICAgICAgICMgQ2FsY3VsYXRlIGFuZCBzdG9yZSBuZXcgdmFsdWVzIGluIG1haW5ab29tIG9iamVjdFxuICAgICAgICBtYWluWm9vbS5zY2FsZSBtYWluWm9vbS5zY2FsZSgpIC0gMC4xXG4gICAgICAgIHRyYW5zbGF0ZSA9IG1haW5ab29tLnRyYW5zbGF0ZSgpXG4gICAgICAgIHYxID0gdHJhbnNsYXRlWzBdICogKG1haW5ab29tLnNjYWxlKCkgLSAwLjEgLyAobWFpblpvb20uc2NhbGUoKSkpXG4gICAgICAgIHYyID0gdHJhbnNsYXRlWzFdICogKG1haW5ab29tLnNjYWxlKCkgLSAwLjEgLyAobWFpblpvb20uc2NhbGUoKSkpXG4gICAgICAgIG1haW5ab29tLnRyYW5zbGF0ZSBbIHYxLCB2MiBdXG4gICAgICAgIFxuICAgICAgICAjIFRyYW5zZm9ybSBzdmdcbiAgICAgICAgZDNtYWluU3ZnRy5hdHRyIFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgdjEgKyBcIixcIiArIHYyICsgXCIpIHNjYWxlKFwiICsgbWFpblpvb20uc2NhbGUoKSArIFwiKVwiXG5cbiAgICAjY3JlYXRlIGEgbGFiZWwgb2YgYW4gZWRnZVxuICAgIGNyZWF0ZUxhYmVsRWRnZSA9IChlbCkgLT5cbiAgICAgIGxhYmVsVmFsdWUgPSBcIlwiXG4gICAgICBpZiBlbC5zaGlwX3N0cmF0ZWd5PyBvciBlbC5sb2NhbF9zdHJhdGVneT9cbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxkaXYgY2xhc3M9J2VkZ2UtbGFiZWwnPlwiXG4gICAgICAgIGxhYmVsVmFsdWUgKz0gZWwuc2hpcF9zdHJhdGVneSAgaWYgZWwuc2hpcF9zdHJhdGVneT9cbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIiAoXCIgKyBlbC50ZW1wX21vZGUgKyBcIilcIiAgdW5sZXNzIGVsLnRlbXBfbW9kZSBpcyBgdW5kZWZpbmVkYFxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiLDxicj5cIiArIGVsLmxvY2FsX3N0cmF0ZWd5ICB1bmxlc3MgZWwubG9jYWxfc3RyYXRlZ3kgaXMgYHVuZGVmaW5lZGBcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjwvZGl2PlwiXG4gICAgICBsYWJlbFZhbHVlXG5cblxuICAgICMgdHJ1ZSwgaWYgdGhlIG5vZGUgaXMgYSBzcGVjaWFsIG5vZGUgZnJvbSBhbiBpdGVyYXRpb25cbiAgICBpc1NwZWNpYWxJdGVyYXRpb25Ob2RlID0gKGluZm8pIC0+XG4gICAgICAoaW5mbyBpcyBcInBhcnRpYWxTb2x1dGlvblwiIG9yIGluZm8gaXMgXCJuZXh0UGFydGlhbFNvbHV0aW9uXCIgb3IgaW5mbyBpcyBcIndvcmtzZXRcIiBvciBpbmZvIGlzIFwibmV4dFdvcmtzZXRcIiBvciBpbmZvIGlzIFwic29sdXRpb25TZXRcIiBvciBpbmZvIGlzIFwic29sdXRpb25EZWx0YVwiKVxuXG4gICAgZ2V0Tm9kZVR5cGUgPSAoZWwsIGluZm8pIC0+XG4gICAgICBpZiBpbmZvIGlzIFwibWlycm9yXCJcbiAgICAgICAgJ25vZGUtbWlycm9yJ1xuXG4gICAgICBlbHNlIGlmIGlzU3BlY2lhbEl0ZXJhdGlvbk5vZGUoaW5mbylcbiAgICAgICAgJ25vZGUtaXRlcmF0aW9uJ1xuXG4gICAgICBlbHNlXG4gICAgICAgIGlmIGVsLnBhY3QgaXMgXCJEYXRhIFNvdXJjZVwiXG4gICAgICAgICAgJ25vZGUtc291cmNlJ1xuICAgICAgICBlbHNlIGlmIGVsLnBhY3QgaXMgXCJEYXRhIFNpbmtcIlxuICAgICAgICAgICdub2RlLXNpbmsnXG4gICAgICAgIGVsc2VcbiAgICAgICAgICAnbm9kZS1ub3JtYWwnXG4gICAgICBcbiAgICAjIGNyZWF0ZXMgdGhlIGxhYmVsIG9mIGEgbm9kZSwgaW4gaW5mbyBpcyBzdG9yZWQsIHdoZXRoZXIgaXQgaXMgYSBzcGVjaWFsIG5vZGUgKGxpa2UgYSBtaXJyb3IgaW4gYW4gaXRlcmF0aW9uKVxuICAgIGNyZWF0ZUxhYmVsTm9kZSA9IChlbCwgaW5mbywgbWF4VywgbWF4SCkgLT5cbiAgICAgIGxhYmVsVmFsdWUgPSBcIjxhIGhyZWY9JyMvam9icy9cIiArIGpvYmlkICsgXCIvXCIgKyBlbC5pZCArIFwiJyBjbGFzcz0nbm9kZS1sYWJlbCBcIiArIGdldE5vZGVUeXBlKGVsLCBpbmZvKSArIFwiJz5cIlxuXG4gICAgICAjIE5vZGVuYW1lXG4gICAgICBpZiBpbmZvIGlzIFwibWlycm9yXCJcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoMyBjbGFzcz0nbm9kZS1uYW1lJz5NaXJyb3Igb2YgXCIgKyBlbC5wYWN0ICsgXCI8L2gzPlwiXG4gICAgICBlbHNlXG4gICAgICAgIGxhYmVsVmFsdWUgKz0gXCI8aDMgY2xhc3M9J25vZGUtbmFtZSc+XCIgKyBlbC5wYWN0ICsgXCI8L2gzPlwiXG4gICAgICBpZiBlbC5jb250ZW50cyBpcyBcIlwiXG4gICAgICAgIGxhYmVsVmFsdWUgKz0gXCJcIlxuICAgICAgZWxzZVxuICAgICAgICBzdGVwTmFtZSA9IGVsLmNvbnRlbnRzXG4gICAgICAgIFxuICAgICAgICAjIGNsZWFuIHN0ZXBOYW1lXG4gICAgICAgIHN0ZXBOYW1lID0gc2hvcnRlblN0cmluZyhzdGVwTmFtZSlcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoNCBjbGFzcz0nc3RlcC1uYW1lJz5cIiArIHN0ZXBOYW1lICsgXCI8L2g0PlwiXG4gICAgICBcbiAgICAgICMgSWYgdGhpcyBub2RlIGlzIGFuIFwiaXRlcmF0aW9uXCIgd2UgbmVlZCBhIGRpZmZlcmVudCBwYW5lbC1ib2R5XG4gICAgICBpZiBlbC5zdGVwX2Z1bmN0aW9uP1xuICAgICAgICBsYWJlbFZhbHVlICs9IGV4dGVuZExhYmVsTm9kZUZvckl0ZXJhdGlvbihlbC5pZCwgbWF4VywgbWF4SClcbiAgICAgIGVsc2VcbiAgICAgICAgXG4gICAgICAgICMgT3RoZXJ3aXNlIGFkZCBpbmZvcyAgICBcbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoNT5cIiArIGluZm8gKyBcIiBOb2RlPC9oNT5cIiAgaWYgaXNTcGVjaWFsSXRlcmF0aW9uTm9kZShpbmZvKVxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg1PlBhcmFsbGVsaXNtOiBcIiArIGVsLnBhcmFsbGVsaXNtICsgXCI8L2g1PlwiICB1bmxlc3MgZWwucGFyYWxsZWxpc20gaXMgXCJcIlxuICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg1PkRyaXZlciBTdHJhdGVneTogXCIgKyBzaG9ydGVuU3RyaW5nKGVsLmRyaXZlcl9zdHJhdGVneSkgKyBcIjwvaDVcIiAgdW5sZXNzIGVsLmRyaXZlcl9zdHJhdGVneSBpcyBgdW5kZWZpbmVkYFxuICAgICAgXG4gICAgICBsYWJlbFZhbHVlICs9IFwiPC9hPlwiXG4gICAgICBsYWJlbFZhbHVlXG5cbiAgICAjIEV4dGVuZHMgdGhlIGxhYmVsIG9mIGEgbm9kZSB3aXRoIGFuIGFkZGl0aW9uYWwgc3ZnIEVsZW1lbnQgdG8gcHJlc2VudCB0aGUgaXRlcmF0aW9uLlxuICAgIGV4dGVuZExhYmVsTm9kZUZvckl0ZXJhdGlvbiA9IChpZCwgbWF4VywgbWF4SCkgLT5cbiAgICAgIHN2Z0lEID0gXCJzdmctXCIgKyBpZFxuXG4gICAgICBsYWJlbFZhbHVlID0gXCI8c3ZnIGNsYXNzPSdcIiArIHN2Z0lEICsgXCInIHdpZHRoPVwiICsgbWF4VyArIFwiIGhlaWdodD1cIiArIG1heEggKyBcIj48ZyAvPjwvc3ZnPlwiXG4gICAgICBsYWJlbFZhbHVlXG5cbiAgICAjIFNwbGl0IGEgc3RyaW5nIGludG8gbXVsdGlwbGUgbGluZXMgc28gdGhhdCBlYWNoIGxpbmUgaGFzIGxlc3MgdGhhbiAzMCBsZXR0ZXJzLlxuICAgIHNob3J0ZW5TdHJpbmcgPSAocykgLT5cbiAgICAgICMgbWFrZSBzdXJlIHRoYXQgbmFtZSBkb2VzIG5vdCBjb250YWluIGEgPCAoYmVjYXVzZSBvZiBodG1sKVxuICAgICAgaWYgcy5jaGFyQXQoMCkgaXMgXCI8XCJcbiAgICAgICAgcyA9IHMucmVwbGFjZShcIjxcIiwgXCImbHQ7XCIpXG4gICAgICAgIHMgPSBzLnJlcGxhY2UoXCI+XCIsIFwiJmd0O1wiKVxuICAgICAgc2JyID0gXCJcIlxuICAgICAgd2hpbGUgcy5sZW5ndGggPiAzMFxuICAgICAgICBzYnIgPSBzYnIgKyBzLnN1YnN0cmluZygwLCAzMCkgKyBcIjxicj5cIlxuICAgICAgICBzID0gcy5zdWJzdHJpbmcoMzAsIHMubGVuZ3RoKVxuICAgICAgc2JyID0gc2JyICsgc1xuICAgICAgc2JyXG5cbiAgICBjcmVhdGVOb2RlID0gKGcsIGRhdGEsIGVsLCBpc1BhcmVudCA9IGZhbHNlLCBtYXhXLCBtYXhIKSAtPlxuICAgICAgIyBjcmVhdGUgbm9kZSwgc2VuZCBhZGRpdGlvbmFsIGluZm9ybWF0aW9ucyBhYm91dCB0aGUgbm9kZSBpZiBpdCBpcyBhIHNwZWNpYWwgb25lXG4gICAgICBpZiBlbC5pZCBpcyBkYXRhLnBhcnRpYWxfc29sdXRpb25cbiAgICAgICAgZy5zZXROb2RlIGVsLmlkLFxuICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwicGFydGlhbFNvbHV0aW9uXCIsIG1heFcsIG1heEgpXG4gICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICBjbGFzczogZ2V0Tm9kZVR5cGUoZWwsIFwicGFydGlhbFNvbHV0aW9uXCIpXG5cbiAgICAgIGVsc2UgaWYgZWwuaWQgaXMgZGF0YS5uZXh0X3BhcnRpYWxfc29sdXRpb25cbiAgICAgICAgZy5zZXROb2RlIGVsLmlkLFxuICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwibmV4dFBhcnRpYWxTb2x1dGlvblwiLCBtYXhXLCBtYXhIKVxuICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgY2xhc3M6IGdldE5vZGVUeXBlKGVsLCBcIm5leHRQYXJ0aWFsU29sdXRpb25cIilcblxuICAgICAgZWxzZSBpZiBlbC5pZCBpcyBkYXRhLndvcmtzZXRcbiAgICAgICAgZy5zZXROb2RlIGVsLmlkLFxuICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwid29ya3NldFwiLCBtYXhXLCBtYXhIKVxuICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgY2xhc3M6IGdldE5vZGVUeXBlKGVsLCBcIndvcmtzZXRcIilcblxuICAgICAgZWxzZSBpZiBlbC5pZCBpcyBkYXRhLm5leHRfd29ya3NldFxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJuZXh0V29ya3NldFwiLCBtYXhXLCBtYXhIKVxuICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgY2xhc3M6IGdldE5vZGVUeXBlKGVsLCBcIm5leHRXb3Jrc2V0XCIpXG5cbiAgICAgIGVsc2UgaWYgZWwuaWQgaXMgZGF0YS5zb2x1dGlvbl9zZXRcbiAgICAgICAgZy5zZXROb2RlIGVsLmlkLFxuICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwic29sdXRpb25TZXRcIiwgbWF4VywgbWF4SClcbiAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJ1xuICAgICAgICAgIGNsYXNzOiBnZXROb2RlVHlwZShlbCwgXCJzb2x1dGlvblNldFwiKVxuXG4gICAgICBlbHNlIGlmIGVsLmlkIGlzIGRhdGEuc29sdXRpb25fZGVsdGFcbiAgICAgICAgZy5zZXROb2RlIGVsLmlkLFxuICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwic29sdXRpb25EZWx0YVwiLCBtYXhXLCBtYXhIKVxuICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgY2xhc3M6IGdldE5vZGVUeXBlKGVsLCBcInNvbHV0aW9uRGVsdGFcIilcblxuICAgICAgZWxzZVxuICAgICAgICBnLnNldE5vZGUgZWwuaWQsXG4gICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJcIiwgbWF4VywgbWF4SClcbiAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJ1xuICAgICAgICAgIGNsYXNzOiBnZXROb2RlVHlwZShlbCwgXCJcIilcblxuICAgIGNyZWF0ZUVkZ2UgPSAoZywgZGF0YSwgZWwsIGV4aXN0aW5nTm9kZXMsIHByZWQpIC0+XG4gICAgICB1bmxlc3MgZXhpc3RpbmdOb2Rlcy5pbmRleE9mKHByZWQuaWQpIGlzIC0xXG4gICAgICAgIGcuc2V0RWRnZSBwcmVkLmlkLCBlbC5pZCxcbiAgICAgICAgICBsYWJlbDogY3JlYXRlTGFiZWxFZGdlKHByZWQpXG4gICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCdcbiAgICAgICAgICBhcnJvd2hlYWQ6ICdub3JtYWwnXG5cbiAgICAgIGVsc2VcbiAgICAgICAgbWlzc2luZ05vZGUgPSBzZWFyY2hGb3JOb2RlKGRhdGEsIHByZWQuaWQpXG4gICAgICAgIHVubGVzcyAhbWlzc2luZ05vZGUgb3IgbWlzc2luZ05vZGUuYWxyZWFkeUFkZGVkIGlzIHRydWVcbiAgICAgICAgICBtaXNzaW5nTm9kZS5hbHJlYWR5QWRkZWQgPSB0cnVlXG4gICAgICAgICAgZy5zZXROb2RlIG1pc3NpbmdOb2RlLmlkLFxuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShtaXNzaW5nTm9kZSwgXCJtaXJyb3JcIilcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgICBjbGFzczogZ2V0Tm9kZVR5cGUobWlzc2luZ05vZGUsICdtaXJyb3InKVxuXG4gICAgICAgICAgZy5zZXRFZGdlIG1pc3NpbmdOb2RlLmlkLCBlbC5pZCxcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbEVkZ2UobWlzc2luZ05vZGUpXG4gICAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJ1xuXG4gICAgbG9hZEpzb25Ub0RhZ3JlID0gKGcsIGRhdGEpIC0+XG4gICAgICBleGlzdGluZ05vZGVzID0gW11cblxuICAgICAgaWYgZGF0YS5ub2Rlcz9cbiAgICAgICAgIyBUaGlzIGlzIHRoZSBub3JtYWwganNvbiBkYXRhXG4gICAgICAgIHRvSXRlcmF0ZSA9IGRhdGEubm9kZXNcblxuICAgICAgZWxzZVxuICAgICAgICAjIFRoaXMgaXMgYW4gaXRlcmF0aW9uLCB3ZSBub3cgc3RvcmUgc3BlY2lhbCBpdGVyYXRpb24gbm9kZXMgaWYgcG9zc2libGVcbiAgICAgICAgdG9JdGVyYXRlID0gZGF0YS5zdGVwX2Z1bmN0aW9uXG4gICAgICAgIGlzUGFyZW50ID0gdHJ1ZVxuXG4gICAgICBmb3IgZWwgaW4gdG9JdGVyYXRlXG4gICAgICAgIG1heFcgPSAwXG4gICAgICAgIG1heEggPSAwXG5cbiAgICAgICAgaWYgZWwuc3RlcF9mdW5jdGlvblxuICAgICAgICAgIHNnID0gbmV3IGRhZ3JlRDMuZ3JhcGhsaWIuR3JhcGgoeyBtdWx0aWdyYXBoOiB0cnVlLCBjb21wb3VuZDogdHJ1ZSB9KS5zZXRHcmFwaCh7XG4gICAgICAgICAgICBub2Rlc2VwOiAyMFxuICAgICAgICAgICAgZWRnZXNlcDogMFxuICAgICAgICAgICAgcmFua3NlcDogMjBcbiAgICAgICAgICAgIHJhbmtkaXI6IFwiTFJcIlxuICAgICAgICAgICAgbWFyZ2lueDogMTBcbiAgICAgICAgICAgIG1hcmdpbnk6IDEwXG4gICAgICAgICAgICB9KVxuXG4gICAgICAgICAgc3ViZ3JhcGhzW2VsLmlkXSA9IHNnXG5cbiAgICAgICAgICBsb2FkSnNvblRvRGFncmUoc2csIGVsKVxuXG4gICAgICAgICAgciA9IG5ldyBkYWdyZUQzLnJlbmRlcigpXG4gICAgICAgICAgZDN0bXBTdmcuc2VsZWN0KCdnJykuY2FsbChyLCBzZylcbiAgICAgICAgICBtYXhXID0gc2cuZ3JhcGgoKS53aWR0aFxuICAgICAgICAgIG1heEggPSBzZy5ncmFwaCgpLmhlaWdodFxuXG4gICAgICAgICAgYW5ndWxhci5lbGVtZW50KG1haW5UbXBFbGVtZW50KS5lbXB0eSgpXG5cbiAgICAgICAgY3JlYXRlTm9kZShnLCBkYXRhLCBlbCwgaXNQYXJlbnQsIG1heFcsIG1heEgpXG5cbiAgICAgICAgZXhpc3RpbmdOb2Rlcy5wdXNoIGVsLmlkXG4gICAgICAgIFxuICAgICAgICAjIGNyZWF0ZSBlZGdlcyBmcm9tIHByZWRlY2Vzc29ycyB0byBjdXJyZW50IG5vZGVcbiAgICAgICAgaWYgZWwucHJlZGVjZXNzb3JzP1xuICAgICAgICAgIGZvciBwcmVkIGluIGVsLnByZWRlY2Vzc29yc1xuICAgICAgICAgICAgY3JlYXRlRWRnZShnLCBkYXRhLCBlbCwgZXhpc3RpbmdOb2RlcywgcHJlZClcblxuICAgICAgZ1xuXG4gICAgIyBzZWFyY2hlcyBpbiB0aGUgZ2xvYmFsIEpTT05EYXRhIGZvciB0aGUgbm9kZSB3aXRoIHRoZSBnaXZlbiBpZFxuICAgIHNlYXJjaEZvck5vZGUgPSAoZGF0YSwgbm9kZUlEKSAtPlxuICAgICAgZm9yIGkgb2YgZGF0YS5ub2Rlc1xuICAgICAgICBlbCA9IGRhdGEubm9kZXNbaV1cbiAgICAgICAgcmV0dXJuIGVsICBpZiBlbC5pZCBpcyBub2RlSURcbiAgICAgICAgXG4gICAgICAgICMgbG9vayBmb3Igbm9kZXMgdGhhdCBhcmUgaW4gaXRlcmF0aW9uc1xuICAgICAgICBpZiBlbC5zdGVwX2Z1bmN0aW9uP1xuICAgICAgICAgIGZvciBqIG9mIGVsLnN0ZXBfZnVuY3Rpb25cbiAgICAgICAgICAgIHJldHVybiBlbC5zdGVwX2Z1bmN0aW9uW2pdICBpZiBlbC5zdGVwX2Z1bmN0aW9uW2pdLmlkIGlzIG5vZGVJRFxuXG4gICAgZHJhd0dyYXBoID0gKGRhdGEpIC0+XG4gICAgICBnID0gbmV3IGRhZ3JlRDMuZ3JhcGhsaWIuR3JhcGgoeyBtdWx0aWdyYXBoOiB0cnVlLCBjb21wb3VuZDogdHJ1ZSB9KS5zZXRHcmFwaCh7XG4gICAgICAgIG5vZGVzZXA6IDcwXG4gICAgICAgIGVkZ2VzZXA6IDBcbiAgICAgICAgcmFua3NlcDogNTBcbiAgICAgICAgcmFua2RpcjogXCJMUlwiXG4gICAgICAgIG1hcmdpbng6IDQwXG4gICAgICAgIG1hcmdpbnk6IDQwXG4gICAgICAgIH0pXG5cbiAgICAgIGxvYWRKc29uVG9EYWdyZShnLCBkYXRhKVxuXG4gICAgICByZW5kZXJlciA9IG5ldyBkYWdyZUQzLnJlbmRlcigpXG4gICAgICBkM21haW5TdmdHLmNhbGwocmVuZGVyZXIsIGcpXG5cbiAgICAgIGZvciBpLCBzZyBvZiBzdWJncmFwaHNcbiAgICAgICAgZDNtYWluU3ZnLnNlbGVjdCgnc3ZnLnN2Zy0nICsgaSArICcgZycpLmNhbGwocmVuZGVyZXIsIHNnKVxuXG4gICAgICBuZXdTY2FsZSA9IDAuNVxuXG4gICAgICB4Q2VudGVyT2Zmc2V0ID0gTWF0aC5mbG9vcigoYW5ndWxhci5lbGVtZW50KG1haW5TdmdFbGVtZW50KS53aWR0aCgpIC0gZy5ncmFwaCgpLndpZHRoICogbmV3U2NhbGUpIC8gMilcbiAgICAgIHlDZW50ZXJPZmZzZXQgPSBNYXRoLmZsb29yKChhbmd1bGFyLmVsZW1lbnQobWFpblN2Z0VsZW1lbnQpLmhlaWdodCgpIC0gZy5ncmFwaCgpLmhlaWdodCAqIG5ld1NjYWxlKSAvIDIpXG5cbiAgICAgIG1haW5ab29tLnNjYWxlKG5ld1NjYWxlKS50cmFuc2xhdGUoW3hDZW50ZXJPZmZzZXQsIHlDZW50ZXJPZmZzZXRdKVxuXG4gICAgICBkM21haW5TdmdHLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyB4Q2VudGVyT2Zmc2V0ICsgXCIsIFwiICsgeUNlbnRlck9mZnNldCArIFwiKSBzY2FsZShcIiArIG1haW5ab29tLnNjYWxlKCkgKyBcIilcIilcblxuICAgICAgbWFpblpvb20ub24oXCJ6b29tXCIsIC0+XG4gICAgICAgIGV2ID0gZDMuZXZlbnRcbiAgICAgICAgZDNtYWluU3ZnRy5hdHRyIFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlICsgXCIpIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIilcIlxuICAgICAgKVxuICAgICAgbWFpblpvb20oZDNtYWluU3ZnKVxuXG4gICAgc2NvcGUuJHdhdGNoIGF0dHJzLnBsYW4sIChuZXdQbGFuKSAtPlxuICAgICAgZHJhd0dyYXBoKG5ld1BsYW4pIGlmIG5ld1BsYW5cblxuICAgIHJldHVyblxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuZGlyZWN0aXZlKCd2ZXJ0ZXgnLCBmdW5jdGlvbigkc3RhdGUpIHtcbiAgcmV0dXJuIHtcbiAgICB0ZW1wbGF0ZTogXCI8c3ZnIGNsYXNzPSd0aW1lbGluZSBzZWNvbmRhcnknIHdpZHRoPScwJyBoZWlnaHQ9JzAnPjwvc3ZnPlwiLFxuICAgIHNjb3BlOiB7XG4gICAgICBkYXRhOiBcIj1cIlxuICAgIH0sXG4gICAgbGluazogZnVuY3Rpb24oc2NvcGUsIGVsZW0sIGF0dHJzKSB7XG4gICAgICB2YXIgYW5hbHl6ZVRpbWUsIGNvbnRhaW5lclcsIHN2Z0VsLCB6b29tO1xuICAgICAgem9vbSA9IGQzLmJlaGF2aW9yLnpvb20oKTtcbiAgICAgIHN2Z0VsID0gZWxlbS5jaGlsZHJlbigpWzBdO1xuICAgICAgY29udGFpbmVyVyA9IGVsZW0ud2lkdGgoKTtcbiAgICAgIGFuZ3VsYXIuZWxlbWVudChzdmdFbCkuYXR0cignd2lkdGgnLCBjb250YWluZXJXIC0gMTYpO1xuICAgICAgYW5hbHl6ZVRpbWUgPSBmdW5jdGlvbihkYXRhKSB7XG4gICAgICAgIHZhciBiYm94LCBjaGFydCwgc3ZnLCBzdmdHLCB0ZXN0RGF0YTtcbiAgICAgICAgdGVzdERhdGEgPSBbXTtcbiAgICAgICAgYW5ndWxhci5mb3JFYWNoKGRhdGEuZ3JvdXB2ZXJ0ZXguZ3JvdXBtZW1iZXJzLCBmdW5jdGlvbih2ZXJ0ZXgsIGkpIHtcbiAgICAgICAgICB2YXIgdlRpbWU7XG4gICAgICAgICAgdlRpbWUgPSBkYXRhLnZlcnRpY2V0aW1lc1t2ZXJ0ZXgudmVydGV4aWRdO1xuICAgICAgICAgIHJldHVybiB0ZXN0RGF0YS5wdXNoKHtcbiAgICAgICAgICAgIGxhYmVsOiB2ZXJ0ZXgudmVydGV4aW5zdGFuY2VuYW1lICsgXCIgKFwiICsgaSArIFwiKVwiLFxuICAgICAgICAgICAgdGltZXM6IFtcbiAgICAgICAgICAgICAge1xuICAgICAgICAgICAgICAgIGxhYmVsOiBcIlNjaGVkdWxlZFwiLFxuICAgICAgICAgICAgICAgIGNvbG9yOiBcIiM2NjZcIixcbiAgICAgICAgICAgICAgICBzdGFydGluZ190aW1lOiB2VGltZVtcIlNDSEVEVUxFRFwiXSAqIDEwMCxcbiAgICAgICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJERVBMT1lJTkdcIl0gKiAxMDBcbiAgICAgICAgICAgICAgfSwge1xuICAgICAgICAgICAgICAgIGxhYmVsOiBcIkRlcGxveWluZ1wiLFxuICAgICAgICAgICAgICAgIGNvbG9yOiBcIiNhYWFcIixcbiAgICAgICAgICAgICAgICBzdGFydGluZ190aW1lOiB2VGltZVtcIkRFUExPWUlOR1wiXSAqIDEwMCxcbiAgICAgICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJSVU5OSU5HXCJdICogMTAwXG4gICAgICAgICAgICAgIH0sIHtcbiAgICAgICAgICAgICAgICBsYWJlbDogXCJSdW5uaW5nXCIsXG4gICAgICAgICAgICAgICAgY29sb3I6IFwiI2RkZFwiLFxuICAgICAgICAgICAgICAgIHN0YXJ0aW5nX3RpbWU6IHZUaW1lW1wiUlVOTklOR1wiXSAqIDEwMCxcbiAgICAgICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJGSU5JU0hFRFwiXSAqIDEwMFxuICAgICAgICAgICAgICB9XG4gICAgICAgICAgICBdXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0pO1xuICAgICAgICBjaGFydCA9IGQzLnRpbWVsaW5lKCkuc3RhY2soKS50aWNrRm9ybWF0KHtcbiAgICAgICAgICBmb3JtYXQ6IGQzLnRpbWUuZm9ybWF0KFwiJVNcIiksXG4gICAgICAgICAgdGlja0ludGVydmFsOiAxLFxuICAgICAgICAgIHRpY2tTaXplOiAxXG4gICAgICAgIH0pLmxhYmVsRm9ybWF0KGZ1bmN0aW9uKGxhYmVsKSB7XG4gICAgICAgICAgcmV0dXJuIGxhYmVsO1xuICAgICAgICB9KS5tYXJnaW4oe1xuICAgICAgICAgIGxlZnQ6IDEwMCxcbiAgICAgICAgICByaWdodDogMCxcbiAgICAgICAgICB0b3A6IDAsXG4gICAgICAgICAgYm90dG9tOiAwXG4gICAgICAgIH0pO1xuICAgICAgICBzdmcgPSBkMy5zZWxlY3Qoc3ZnRWwpLmRhdHVtKHRlc3REYXRhKS5jYWxsKGNoYXJ0KS5jYWxsKHpvb20pO1xuICAgICAgICBzdmdHID0gc3ZnLnNlbGVjdChcImdcIik7XG4gICAgICAgIHpvb20ub24oXCJ6b29tXCIsIGZ1bmN0aW9uKCkge1xuICAgICAgICAgIHZhciBldjtcbiAgICAgICAgICBldiA9IGQzLmV2ZW50O1xuICAgICAgICAgIHN2Z0cuc2VsZWN0QWxsKCdyZWN0JykuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIGV2LnRyYW5zbGF0ZVswXSArIFwiLDApIHNjYWxlKFwiICsgZXYuc2NhbGUgKyBcIiwxKVwiKTtcbiAgICAgICAgICByZXR1cm4gc3ZnRy5zZWxlY3RBbGwoJ3RleHQnKS5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlWzBdICsgXCIsMCkgc2NhbGUoXCIgKyBldi5zY2FsZSArIFwiLDEpXCIpO1xuICAgICAgICB9KTtcbiAgICAgICAgYmJveCA9IHN2Z0dbMF1bMF0uZ2V0QkJveCgpO1xuICAgICAgICByZXR1cm4gc3ZnLmF0dHIoJ2hlaWdodCcsIGJib3guaGVpZ2h0ICsgMzApO1xuICAgICAgfTtcbiAgICAgIGFuYWx5emVUaW1lKHNjb3BlLmRhdGEpO1xuICAgIH1cbiAgfTtcbn0pLmRpcmVjdGl2ZSgndGltZWxpbmUnLCBmdW5jdGlvbigkc3RhdGUpIHtcbiAgcmV0dXJuIHtcbiAgICB0ZW1wbGF0ZTogXCI8c3ZnIGNsYXNzPSd0aW1lbGluZScgd2lkdGg9JzAnIGhlaWdodD0nMCc+PC9zdmc+XCIsXG4gICAgc2NvcGU6IHtcbiAgICAgIGpvYjogXCI9XCJcbiAgICB9LFxuICAgIGxpbms6IGZ1bmN0aW9uKHNjb3BlLCBlbGVtLCBhdHRycykge1xuICAgICAgdmFyIGFuYWx5emVUaW1lLCBjb250YWluZXJXLCBzdmdFbCwgem9vbTtcbiAgICAgIHpvb20gPSBkMy5iZWhhdmlvci56b29tKCk7XG4gICAgICBzdmdFbCA9IGVsZW0uY2hpbGRyZW4oKVswXTtcbiAgICAgIGNvbnRhaW5lclcgPSBlbGVtLndpZHRoKCk7XG4gICAgICBhbmd1bGFyLmVsZW1lbnQoc3ZnRWwpLmF0dHIoJ3dpZHRoJywgY29udGFpbmVyVyAtIDE2KTtcbiAgICAgIGFuYWx5emVUaW1lID0gZnVuY3Rpb24oZGF0YSkge1xuICAgICAgICB2YXIgYmJveCwgY2hhcnQsIHN2Zywgc3ZnRywgdGVzdERhdGE7XG4gICAgICAgIHRlc3REYXRhID0gW107XG4gICAgICAgIGFuZ3VsYXIuZm9yRWFjaChkYXRhLm9sZFYuZ3JvdXB2ZXJ0aWNlcywgZnVuY3Rpb24odmVydGV4KSB7XG4gICAgICAgICAgdmFyIHZUaW1lO1xuICAgICAgICAgIHZUaW1lID0gZGF0YS5vbGRWLmdyb3VwdmVydGljZXRpbWVzW3ZlcnRleC5ncm91cHZlcnRleGlkXTtcbiAgICAgICAgICByZXR1cm4gdGVzdERhdGEucHVzaCh7XG4gICAgICAgICAgICB0aW1lczogW1xuICAgICAgICAgICAgICB7XG4gICAgICAgICAgICAgICAgbGFiZWw6IHZlcnRleC5ncm91cHZlcnRleG5hbWUsXG4gICAgICAgICAgICAgICAgY29sb3I6IFwiIzNmYjZkOFwiLFxuICAgICAgICAgICAgICAgIHN0YXJ0aW5nX3RpbWU6IHZUaW1lW1wiU1RBUlRFRFwiXSxcbiAgICAgICAgICAgICAgICBlbmRpbmdfdGltZTogdlRpbWVbXCJFTkRFRFwiXSxcbiAgICAgICAgICAgICAgICBsaW5rOiB2ZXJ0ZXguZ3JvdXB2ZXJ0ZXhpZFxuICAgICAgICAgICAgICB9XG4gICAgICAgICAgICBdXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0pO1xuICAgICAgICBjaGFydCA9IGQzLnRpbWVsaW5lKCkuc3RhY2soKS5jbGljayhmdW5jdGlvbihkLCBpLCBkYXR1bSkge1xuICAgICAgICAgIHJldHVybiAkc3RhdGUuZ28oXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLCB7XG4gICAgICAgICAgICBqb2JpZDogZGF0YS5qaWQsXG4gICAgICAgICAgICB2ZXJ0ZXhJZDogZC5saW5rXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0pLnRpY2tGb3JtYXQoe1xuICAgICAgICAgIGZvcm1hdDogZDMudGltZS5mb3JtYXQoXCIlU1wiKSxcbiAgICAgICAgICB0aWNrSW50ZXJ2YWw6IDEsXG4gICAgICAgICAgdGlja1NpemU6IDFcbiAgICAgICAgfSkubWFyZ2luKHtcbiAgICAgICAgICBsZWZ0OiAwLFxuICAgICAgICAgIHJpZ2h0OiAwLFxuICAgICAgICAgIHRvcDogMCxcbiAgICAgICAgICBib3R0b206IDBcbiAgICAgICAgfSk7XG4gICAgICAgIHN2ZyA9IGQzLnNlbGVjdChzdmdFbCkuZGF0dW0odGVzdERhdGEpLmNhbGwoY2hhcnQpLmNhbGwoem9vbSk7XG4gICAgICAgIHN2Z0cgPSBzdmcuc2VsZWN0KFwiZ1wiKTtcbiAgICAgICAgem9vbS5vbihcInpvb21cIiwgZnVuY3Rpb24oKSB7XG4gICAgICAgICAgdmFyIGV2O1xuICAgICAgICAgIGV2ID0gZDMuZXZlbnQ7XG4gICAgICAgICAgc3ZnRy5zZWxlY3RBbGwoJ3JlY3QnKS5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgZXYudHJhbnNsYXRlWzBdICsgXCIsMCkgc2NhbGUoXCIgKyBldi5zY2FsZSArIFwiLDEpXCIpO1xuICAgICAgICAgIHJldHVybiBzdmdHLnNlbGVjdEFsbCgndGV4dCcpLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyBldi50cmFuc2xhdGVbMF0gKyBcIiwwKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIsMSlcIik7XG4gICAgICAgIH0pO1xuICAgICAgICBiYm94ID0gc3ZnR1swXVswXS5nZXRCQm94KCk7XG4gICAgICAgIHJldHVybiBzdmcuYXR0cignaGVpZ2h0JywgYmJveC5oZWlnaHQgKyAzMCk7XG4gICAgICB9O1xuICAgICAgc2NvcGUuJHdhdGNoKGF0dHJzLmpvYiwgZnVuY3Rpb24oZGF0YSkge1xuICAgICAgICBpZiAoZGF0YSkge1xuICAgICAgICAgIHJldHVybiBhbmFseXplVGltZShkYXRhKTtcbiAgICAgICAgfVxuICAgICAgfSk7XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCdqb2JQbGFuJywgZnVuY3Rpb24oJHRpbWVvdXQpIHtcbiAgcmV0dXJuIHtcbiAgICB0ZW1wbGF0ZTogXCI8c3ZnIGNsYXNzPSdncmFwaCcgd2lkdGg9JzUwMCcgaGVpZ2h0PSc0MDAnPjxnIC8+PC9zdmc+IDxzdmcgY2xhc3M9J3RtcCcgd2lkdGg9JzEnIGhlaWdodD0nMSc+PGcgLz48L3N2Zz4gPGRpdiBjbGFzcz0nYnRuLWdyb3VwIHpvb20tYnV0dG9ucyc+IDxhIGNsYXNzPSdidG4gYnRuLWRlZmF1bHQgem9vbS1pbicgbmctY2xpY2s9J3pvb21JbigpJz48aSBjbGFzcz0nZmEgZmEtcGx1cycgLz48L2E+IDxhIGNsYXNzPSdidG4gYnRuLWRlZmF1bHQgem9vbS1vdXQnIG5nLWNsaWNrPSd6b29tT3V0KCknPjxpIGNsYXNzPSdmYSBmYS1taW51cycgLz48L2E+IDwvZGl2PlwiLFxuICAgIHNjb3BlOiB7XG4gICAgICBwbGFuOiAnPSdcbiAgICB9LFxuICAgIGxpbms6IGZ1bmN0aW9uKHNjb3BlLCBlbGVtLCBhdHRycykge1xuICAgICAgdmFyIGNvbnRhaW5lclcsIGNyZWF0ZUVkZ2UsIGNyZWF0ZUxhYmVsRWRnZSwgY3JlYXRlTGFiZWxOb2RlLCBjcmVhdGVOb2RlLCBkM21haW5TdmcsIGQzbWFpblN2Z0csIGQzdG1wU3ZnLCBkcmF3R3JhcGgsIGV4dGVuZExhYmVsTm9kZUZvckl0ZXJhdGlvbiwgZ2V0Tm9kZVR5cGUsIGlzU3BlY2lhbEl0ZXJhdGlvbk5vZGUsIGpvYmlkLCBsb2FkSnNvblRvRGFncmUsIG1haW5HLCBtYWluU3ZnRWxlbWVudCwgbWFpblRtcEVsZW1lbnQsIG1haW5ab29tLCBzZWFyY2hGb3JOb2RlLCBzaG9ydGVuU3RyaW5nLCBzdWJncmFwaHM7XG4gICAgICBtYWluWm9vbSA9IGQzLmJlaGF2aW9yLnpvb20oKTtcbiAgICAgIHN1YmdyYXBocyA9IFtdO1xuICAgICAgam9iaWQgPSBhdHRycy5qb2JpZDtcbiAgICAgIG1haW5TdmdFbGVtZW50ID0gZWxlbS5jaGlsZHJlbigpWzBdO1xuICAgICAgbWFpbkcgPSBlbGVtLmNoaWxkcmVuKCkuY2hpbGRyZW4oKVswXTtcbiAgICAgIG1haW5UbXBFbGVtZW50ID0gZWxlbS5jaGlsZHJlbigpWzFdO1xuICAgICAgZDNtYWluU3ZnID0gZDMuc2VsZWN0KG1haW5TdmdFbGVtZW50KTtcbiAgICAgIGQzbWFpblN2Z0cgPSBkMy5zZWxlY3QobWFpbkcpO1xuICAgICAgZDN0bXBTdmcgPSBkMy5zZWxlY3QobWFpblRtcEVsZW1lbnQpO1xuICAgICAgY29udGFpbmVyVyA9IGVsZW0ud2lkdGgoKTtcbiAgICAgIGFuZ3VsYXIuZWxlbWVudChlbGVtLmNoaWxkcmVuKClbMF0pLndpZHRoKGNvbnRhaW5lclcpO1xuICAgICAgc2NvcGUuem9vbUluID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHZhciB0cmFuc2xhdGUsIHYxLCB2MjtcbiAgICAgICAgaWYgKG1haW5ab29tLnNjYWxlKCkgPCAyLjk5KSB7XG4gICAgICAgICAgdHJhbnNsYXRlID0gbWFpblpvb20udHJhbnNsYXRlKCk7XG4gICAgICAgICAgdjEgPSB0cmFuc2xhdGVbMF0gKiAobWFpblpvb20uc2NhbGUoKSArIDAuMSAvIChtYWluWm9vbS5zY2FsZSgpKSk7XG4gICAgICAgICAgdjIgPSB0cmFuc2xhdGVbMV0gKiAobWFpblpvb20uc2NhbGUoKSArIDAuMSAvIChtYWluWm9vbS5zY2FsZSgpKSk7XG4gICAgICAgICAgbWFpblpvb20uc2NhbGUobWFpblpvb20uc2NhbGUoKSArIDAuMSk7XG4gICAgICAgICAgbWFpblpvb20udHJhbnNsYXRlKFt2MSwgdjJdKTtcbiAgICAgICAgICByZXR1cm4gZDNtYWluU3ZnRy5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgdjEgKyBcIixcIiArIHYyICsgXCIpIHNjYWxlKFwiICsgbWFpblpvb20uc2NhbGUoKSArIFwiKVwiKTtcbiAgICAgICAgfVxuICAgICAgfTtcbiAgICAgIHNjb3BlLnpvb21PdXQgPSBmdW5jdGlvbigpIHtcbiAgICAgICAgdmFyIHRyYW5zbGF0ZSwgdjEsIHYyO1xuICAgICAgICBpZiAobWFpblpvb20uc2NhbGUoKSA+IDAuMzEpIHtcbiAgICAgICAgICBtYWluWm9vbS5zY2FsZShtYWluWm9vbS5zY2FsZSgpIC0gMC4xKTtcbiAgICAgICAgICB0cmFuc2xhdGUgPSBtYWluWm9vbS50cmFuc2xhdGUoKTtcbiAgICAgICAgICB2MSA9IHRyYW5zbGF0ZVswXSAqIChtYWluWm9vbS5zY2FsZSgpIC0gMC4xIC8gKG1haW5ab29tLnNjYWxlKCkpKTtcbiAgICAgICAgICB2MiA9IHRyYW5zbGF0ZVsxXSAqIChtYWluWm9vbS5zY2FsZSgpIC0gMC4xIC8gKG1haW5ab29tLnNjYWxlKCkpKTtcbiAgICAgICAgICBtYWluWm9vbS50cmFuc2xhdGUoW3YxLCB2Ml0pO1xuICAgICAgICAgIHJldHVybiBkM21haW5TdmdHLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgXCJ0cmFuc2xhdGUoXCIgKyB2MSArIFwiLFwiICsgdjIgKyBcIikgc2NhbGUoXCIgKyBtYWluWm9vbS5zY2FsZSgpICsgXCIpXCIpO1xuICAgICAgICB9XG4gICAgICB9O1xuICAgICAgY3JlYXRlTGFiZWxFZGdlID0gZnVuY3Rpb24oZWwpIHtcbiAgICAgICAgdmFyIGxhYmVsVmFsdWU7XG4gICAgICAgIGxhYmVsVmFsdWUgPSBcIlwiO1xuICAgICAgICBpZiAoKGVsLnNoaXBfc3RyYXRlZ3kgIT0gbnVsbCkgfHwgKGVsLmxvY2FsX3N0cmF0ZWd5ICE9IG51bGwpKSB7XG4gICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxkaXYgY2xhc3M9J2VkZ2UtbGFiZWwnPlwiO1xuICAgICAgICAgIGlmIChlbC5zaGlwX3N0cmF0ZWd5ICE9IG51bGwpIHtcbiAgICAgICAgICAgIGxhYmVsVmFsdWUgKz0gZWwuc2hpcF9zdHJhdGVneTtcbiAgICAgICAgICB9XG4gICAgICAgICAgaWYgKGVsLnRlbXBfbW9kZSAhPT0gdW5kZWZpbmVkKSB7XG4gICAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiIChcIiArIGVsLnRlbXBfbW9kZSArIFwiKVwiO1xuICAgICAgICAgIH1cbiAgICAgICAgICBpZiAoZWwubG9jYWxfc3RyYXRlZ3kgIT09IHVuZGVmaW5lZCkge1xuICAgICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIiw8YnI+XCIgKyBlbC5sb2NhbF9zdHJhdGVneTtcbiAgICAgICAgICB9XG4gICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjwvZGl2PlwiO1xuICAgICAgICB9XG4gICAgICAgIHJldHVybiBsYWJlbFZhbHVlO1xuICAgICAgfTtcbiAgICAgIGlzU3BlY2lhbEl0ZXJhdGlvbk5vZGUgPSBmdW5jdGlvbihpbmZvKSB7XG4gICAgICAgIHJldHVybiBpbmZvID09PSBcInBhcnRpYWxTb2x1dGlvblwiIHx8IGluZm8gPT09IFwibmV4dFBhcnRpYWxTb2x1dGlvblwiIHx8IGluZm8gPT09IFwid29ya3NldFwiIHx8IGluZm8gPT09IFwibmV4dFdvcmtzZXRcIiB8fCBpbmZvID09PSBcInNvbHV0aW9uU2V0XCIgfHwgaW5mbyA9PT0gXCJzb2x1dGlvbkRlbHRhXCI7XG4gICAgICB9O1xuICAgICAgZ2V0Tm9kZVR5cGUgPSBmdW5jdGlvbihlbCwgaW5mbykge1xuICAgICAgICBpZiAoaW5mbyA9PT0gXCJtaXJyb3JcIikge1xuICAgICAgICAgIHJldHVybiAnbm9kZS1taXJyb3InO1xuICAgICAgICB9IGVsc2UgaWYgKGlzU3BlY2lhbEl0ZXJhdGlvbk5vZGUoaW5mbykpIHtcbiAgICAgICAgICByZXR1cm4gJ25vZGUtaXRlcmF0aW9uJztcbiAgICAgICAgfSBlbHNlIHtcbiAgICAgICAgICBpZiAoZWwucGFjdCA9PT0gXCJEYXRhIFNvdXJjZVwiKSB7XG4gICAgICAgICAgICByZXR1cm4gJ25vZGUtc291cmNlJztcbiAgICAgICAgICB9IGVsc2UgaWYgKGVsLnBhY3QgPT09IFwiRGF0YSBTaW5rXCIpIHtcbiAgICAgICAgICAgIHJldHVybiAnbm9kZS1zaW5rJztcbiAgICAgICAgICB9IGVsc2Uge1xuICAgICAgICAgICAgcmV0dXJuICdub2RlLW5vcm1hbCc7XG4gICAgICAgICAgfVxuICAgICAgICB9XG4gICAgICB9O1xuICAgICAgY3JlYXRlTGFiZWxOb2RlID0gZnVuY3Rpb24oZWwsIGluZm8sIG1heFcsIG1heEgpIHtcbiAgICAgICAgdmFyIGxhYmVsVmFsdWUsIHN0ZXBOYW1lO1xuICAgICAgICBsYWJlbFZhbHVlID0gXCI8YSBocmVmPScjL2pvYnMvXCIgKyBqb2JpZCArIFwiL1wiICsgZWwuaWQgKyBcIicgY2xhc3M9J25vZGUtbGFiZWwgXCIgKyBnZXROb2RlVHlwZShlbCwgaW5mbykgKyBcIic+XCI7XG4gICAgICAgIGlmIChpbmZvID09PSBcIm1pcnJvclwiKSB7XG4gICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoMyBjbGFzcz0nbm9kZS1uYW1lJz5NaXJyb3Igb2YgXCIgKyBlbC5wYWN0ICsgXCI8L2gzPlwiO1xuICAgICAgICB9IGVsc2Uge1xuICAgICAgICAgIGxhYmVsVmFsdWUgKz0gXCI8aDMgY2xhc3M9J25vZGUtbmFtZSc+XCIgKyBlbC5wYWN0ICsgXCI8L2gzPlwiO1xuICAgICAgICB9XG4gICAgICAgIGlmIChlbC5jb250ZW50cyA9PT0gXCJcIikge1xuICAgICAgICAgIGxhYmVsVmFsdWUgKz0gXCJcIjtcbiAgICAgICAgfSBlbHNlIHtcbiAgICAgICAgICBzdGVwTmFtZSA9IGVsLmNvbnRlbnRzO1xuICAgICAgICAgIHN0ZXBOYW1lID0gc2hvcnRlblN0cmluZyhzdGVwTmFtZSk7XG4gICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoNCBjbGFzcz0nc3RlcC1uYW1lJz5cIiArIHN0ZXBOYW1lICsgXCI8L2g0PlwiO1xuICAgICAgICB9XG4gICAgICAgIGlmIChlbC5zdGVwX2Z1bmN0aW9uICE9IG51bGwpIHtcbiAgICAgICAgICBsYWJlbFZhbHVlICs9IGV4dGVuZExhYmVsTm9kZUZvckl0ZXJhdGlvbihlbC5pZCwgbWF4VywgbWF4SCk7XG4gICAgICAgIH0gZWxzZSB7XG4gICAgICAgICAgaWYgKGlzU3BlY2lhbEl0ZXJhdGlvbk5vZGUoaW5mbykpIHtcbiAgICAgICAgICAgIGxhYmVsVmFsdWUgKz0gXCI8aDU+XCIgKyBpbmZvICsgXCIgTm9kZTwvaDU+XCI7XG4gICAgICAgICAgfVxuICAgICAgICAgIGlmIChlbC5wYXJhbGxlbGlzbSAhPT0gXCJcIikge1xuICAgICAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjxoNT5QYXJhbGxlbGlzbTogXCIgKyBlbC5wYXJhbGxlbGlzbSArIFwiPC9oNT5cIjtcbiAgICAgICAgICB9XG4gICAgICAgICAgaWYgKGVsLmRyaXZlcl9zdHJhdGVneSAhPT0gdW5kZWZpbmVkKSB7XG4gICAgICAgICAgICBsYWJlbFZhbHVlICs9IFwiPGg1PkRyaXZlciBTdHJhdGVneTogXCIgKyBzaG9ydGVuU3RyaW5nKGVsLmRyaXZlcl9zdHJhdGVneSkgKyBcIjwvaDVcIjtcbiAgICAgICAgICB9XG4gICAgICAgIH1cbiAgICAgICAgbGFiZWxWYWx1ZSArPSBcIjwvYT5cIjtcbiAgICAgICAgcmV0dXJuIGxhYmVsVmFsdWU7XG4gICAgICB9O1xuICAgICAgZXh0ZW5kTGFiZWxOb2RlRm9ySXRlcmF0aW9uID0gZnVuY3Rpb24oaWQsIG1heFcsIG1heEgpIHtcbiAgICAgICAgdmFyIGxhYmVsVmFsdWUsIHN2Z0lEO1xuICAgICAgICBzdmdJRCA9IFwic3ZnLVwiICsgaWQ7XG4gICAgICAgIGxhYmVsVmFsdWUgPSBcIjxzdmcgY2xhc3M9J1wiICsgc3ZnSUQgKyBcIicgd2lkdGg9XCIgKyBtYXhXICsgXCIgaGVpZ2h0PVwiICsgbWF4SCArIFwiPjxnIC8+PC9zdmc+XCI7XG4gICAgICAgIHJldHVybiBsYWJlbFZhbHVlO1xuICAgICAgfTtcbiAgICAgIHNob3J0ZW5TdHJpbmcgPSBmdW5jdGlvbihzKSB7XG4gICAgICAgIHZhciBzYnI7XG4gICAgICAgIGlmIChzLmNoYXJBdCgwKSA9PT0gXCI8XCIpIHtcbiAgICAgICAgICBzID0gcy5yZXBsYWNlKFwiPFwiLCBcIiZsdDtcIik7XG4gICAgICAgICAgcyA9IHMucmVwbGFjZShcIj5cIiwgXCImZ3Q7XCIpO1xuICAgICAgICB9XG4gICAgICAgIHNiciA9IFwiXCI7XG4gICAgICAgIHdoaWxlIChzLmxlbmd0aCA+IDMwKSB7XG4gICAgICAgICAgc2JyID0gc2JyICsgcy5zdWJzdHJpbmcoMCwgMzApICsgXCI8YnI+XCI7XG4gICAgICAgICAgcyA9IHMuc3Vic3RyaW5nKDMwLCBzLmxlbmd0aCk7XG4gICAgICAgIH1cbiAgICAgICAgc2JyID0gc2JyICsgcztcbiAgICAgICAgcmV0dXJuIHNicjtcbiAgICAgIH07XG4gICAgICBjcmVhdGVOb2RlID0gZnVuY3Rpb24oZywgZGF0YSwgZWwsIGlzUGFyZW50LCBtYXhXLCBtYXhIKSB7XG4gICAgICAgIGlmIChpc1BhcmVudCA9PSBudWxsKSB7XG4gICAgICAgICAgaXNQYXJlbnQgPSBmYWxzZTtcbiAgICAgICAgfVxuICAgICAgICBpZiAoZWwuaWQgPT09IGRhdGEucGFydGlhbF9zb2x1dGlvbikge1xuICAgICAgICAgIHJldHVybiBnLnNldE5vZGUoZWwuaWQsIHtcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwicGFydGlhbFNvbHV0aW9uXCIsIG1heFcsIG1heEgpLFxuICAgICAgICAgICAgbGFiZWxUeXBlOiAnaHRtbCcsXG4gICAgICAgICAgICBcImNsYXNzXCI6IGdldE5vZGVUeXBlKGVsLCBcInBhcnRpYWxTb2x1dGlvblwiKVxuICAgICAgICAgIH0pO1xuICAgICAgICB9IGVsc2UgaWYgKGVsLmlkID09PSBkYXRhLm5leHRfcGFydGlhbF9zb2x1dGlvbikge1xuICAgICAgICAgIHJldHVybiBnLnNldE5vZGUoZWwuaWQsIHtcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwibmV4dFBhcnRpYWxTb2x1dGlvblwiLCBtYXhXLCBtYXhIKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgXCJjbGFzc1wiOiBnZXROb2RlVHlwZShlbCwgXCJuZXh0UGFydGlhbFNvbHV0aW9uXCIpXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0gZWxzZSBpZiAoZWwuaWQgPT09IGRhdGEud29ya3NldCkge1xuICAgICAgICAgIHJldHVybiBnLnNldE5vZGUoZWwuaWQsIHtcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwid29ya3NldFwiLCBtYXhXLCBtYXhIKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgXCJjbGFzc1wiOiBnZXROb2RlVHlwZShlbCwgXCJ3b3Jrc2V0XCIpXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0gZWxzZSBpZiAoZWwuaWQgPT09IGRhdGEubmV4dF93b3Jrc2V0KSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJuZXh0V29ya3NldFwiLCBtYXhXLCBtYXhIKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgXCJjbGFzc1wiOiBnZXROb2RlVHlwZShlbCwgXCJuZXh0V29ya3NldFwiKVxuICAgICAgICAgIH0pO1xuICAgICAgICB9IGVsc2UgaWYgKGVsLmlkID09PSBkYXRhLnNvbHV0aW9uX3NldCkge1xuICAgICAgICAgIHJldHVybiBnLnNldE5vZGUoZWwuaWQsIHtcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwic29sdXRpb25TZXRcIiwgbWF4VywgbWF4SCksXG4gICAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJyxcbiAgICAgICAgICAgIFwiY2xhc3NcIjogZ2V0Tm9kZVR5cGUoZWwsIFwic29sdXRpb25TZXRcIilcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSBlbHNlIGlmIChlbC5pZCA9PT0gZGF0YS5zb2x1dGlvbl9kZWx0YSkge1xuICAgICAgICAgIHJldHVybiBnLnNldE5vZGUoZWwuaWQsIHtcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbE5vZGUoZWwsIFwic29sdXRpb25EZWx0YVwiLCBtYXhXLCBtYXhIKSxcbiAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgXCJjbGFzc1wiOiBnZXROb2RlVHlwZShlbCwgXCJzb2x1dGlvbkRlbHRhXCIpXG4gICAgICAgICAgfSk7XG4gICAgICAgIH0gZWxzZSB7XG4gICAgICAgICAgcmV0dXJuIGcuc2V0Tm9kZShlbC5pZCwge1xuICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShlbCwgXCJcIiwgbWF4VywgbWF4SCksXG4gICAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJyxcbiAgICAgICAgICAgIFwiY2xhc3NcIjogZ2V0Tm9kZVR5cGUoZWwsIFwiXCIpXG4gICAgICAgICAgfSk7XG4gICAgICAgIH1cbiAgICAgIH07XG4gICAgICBjcmVhdGVFZGdlID0gZnVuY3Rpb24oZywgZGF0YSwgZWwsIGV4aXN0aW5nTm9kZXMsIHByZWQpIHtcbiAgICAgICAgdmFyIG1pc3NpbmdOb2RlO1xuICAgICAgICBpZiAoZXhpc3RpbmdOb2Rlcy5pbmRleE9mKHByZWQuaWQpICE9PSAtMSkge1xuICAgICAgICAgIHJldHVybiBnLnNldEVkZ2UocHJlZC5pZCwgZWwuaWQsIHtcbiAgICAgICAgICAgIGxhYmVsOiBjcmVhdGVMYWJlbEVkZ2UocHJlZCksXG4gICAgICAgICAgICBsYWJlbFR5cGU6ICdodG1sJyxcbiAgICAgICAgICAgIGFycm93aGVhZDogJ25vcm1hbCdcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSBlbHNlIHtcbiAgICAgICAgICBtaXNzaW5nTm9kZSA9IHNlYXJjaEZvck5vZGUoZGF0YSwgcHJlZC5pZCk7XG4gICAgICAgICAgaWYgKCEoIW1pc3NpbmdOb2RlIHx8IG1pc3NpbmdOb2RlLmFscmVhZHlBZGRlZCA9PT0gdHJ1ZSkpIHtcbiAgICAgICAgICAgIG1pc3NpbmdOb2RlLmFscmVhZHlBZGRlZCA9IHRydWU7XG4gICAgICAgICAgICBnLnNldE5vZGUobWlzc2luZ05vZGUuaWQsIHtcbiAgICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsTm9kZShtaXNzaW5nTm9kZSwgXCJtaXJyb3JcIiksXG4gICAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnLFxuICAgICAgICAgICAgICBcImNsYXNzXCI6IGdldE5vZGVUeXBlKG1pc3NpbmdOb2RlLCAnbWlycm9yJylcbiAgICAgICAgICAgIH0pO1xuICAgICAgICAgICAgcmV0dXJuIGcuc2V0RWRnZShtaXNzaW5nTm9kZS5pZCwgZWwuaWQsIHtcbiAgICAgICAgICAgICAgbGFiZWw6IGNyZWF0ZUxhYmVsRWRnZShtaXNzaW5nTm9kZSksXG4gICAgICAgICAgICAgIGxhYmVsVHlwZTogJ2h0bWwnXG4gICAgICAgICAgICB9KTtcbiAgICAgICAgICB9XG4gICAgICAgIH1cbiAgICAgIH07XG4gICAgICBsb2FkSnNvblRvRGFncmUgPSBmdW5jdGlvbihnLCBkYXRhKSB7XG4gICAgICAgIHZhciBlbCwgZXhpc3RpbmdOb2RlcywgaXNQYXJlbnQsIGssIGwsIGxlbiwgbGVuMSwgbWF4SCwgbWF4VywgcHJlZCwgciwgcmVmLCBzZywgdG9JdGVyYXRlO1xuICAgICAgICBleGlzdGluZ05vZGVzID0gW107XG4gICAgICAgIGlmIChkYXRhLm5vZGVzICE9IG51bGwpIHtcbiAgICAgICAgICB0b0l0ZXJhdGUgPSBkYXRhLm5vZGVzO1xuICAgICAgICB9IGVsc2Uge1xuICAgICAgICAgIHRvSXRlcmF0ZSA9IGRhdGEuc3RlcF9mdW5jdGlvbjtcbiAgICAgICAgICBpc1BhcmVudCA9IHRydWU7XG4gICAgICAgIH1cbiAgICAgICAgZm9yIChrID0gMCwgbGVuID0gdG9JdGVyYXRlLmxlbmd0aDsgayA8IGxlbjsgaysrKSB7XG4gICAgICAgICAgZWwgPSB0b0l0ZXJhdGVba107XG4gICAgICAgICAgbWF4VyA9IDA7XG4gICAgICAgICAgbWF4SCA9IDA7XG4gICAgICAgICAgaWYgKGVsLnN0ZXBfZnVuY3Rpb24pIHtcbiAgICAgICAgICAgIHNnID0gbmV3IGRhZ3JlRDMuZ3JhcGhsaWIuR3JhcGgoe1xuICAgICAgICAgICAgICBtdWx0aWdyYXBoOiB0cnVlLFxuICAgICAgICAgICAgICBjb21wb3VuZDogdHJ1ZVxuICAgICAgICAgICAgfSkuc2V0R3JhcGgoe1xuICAgICAgICAgICAgICBub2Rlc2VwOiAyMCxcbiAgICAgICAgICAgICAgZWRnZXNlcDogMCxcbiAgICAgICAgICAgICAgcmFua3NlcDogMjAsXG4gICAgICAgICAgICAgIHJhbmtkaXI6IFwiTFJcIixcbiAgICAgICAgICAgICAgbWFyZ2lueDogMTAsXG4gICAgICAgICAgICAgIG1hcmdpbnk6IDEwXG4gICAgICAgICAgICB9KTtcbiAgICAgICAgICAgIHN1YmdyYXBoc1tlbC5pZF0gPSBzZztcbiAgICAgICAgICAgIGxvYWRKc29uVG9EYWdyZShzZywgZWwpO1xuICAgICAgICAgICAgciA9IG5ldyBkYWdyZUQzLnJlbmRlcigpO1xuICAgICAgICAgICAgZDN0bXBTdmcuc2VsZWN0KCdnJykuY2FsbChyLCBzZyk7XG4gICAgICAgICAgICBtYXhXID0gc2cuZ3JhcGgoKS53aWR0aDtcbiAgICAgICAgICAgIG1heEggPSBzZy5ncmFwaCgpLmhlaWdodDtcbiAgICAgICAgICAgIGFuZ3VsYXIuZWxlbWVudChtYWluVG1wRWxlbWVudCkuZW1wdHkoKTtcbiAgICAgICAgICB9XG4gICAgICAgICAgY3JlYXRlTm9kZShnLCBkYXRhLCBlbCwgaXNQYXJlbnQsIG1heFcsIG1heEgpO1xuICAgICAgICAgIGV4aXN0aW5nTm9kZXMucHVzaChlbC5pZCk7XG4gICAgICAgICAgaWYgKGVsLnByZWRlY2Vzc29ycyAhPSBudWxsKSB7XG4gICAgICAgICAgICByZWYgPSBlbC5wcmVkZWNlc3NvcnM7XG4gICAgICAgICAgICBmb3IgKGwgPSAwLCBsZW4xID0gcmVmLmxlbmd0aDsgbCA8IGxlbjE7IGwrKykge1xuICAgICAgICAgICAgICBwcmVkID0gcmVmW2xdO1xuICAgICAgICAgICAgICBjcmVhdGVFZGdlKGcsIGRhdGEsIGVsLCBleGlzdGluZ05vZGVzLCBwcmVkKTtcbiAgICAgICAgICAgIH1cbiAgICAgICAgICB9XG4gICAgICAgIH1cbiAgICAgICAgcmV0dXJuIGc7XG4gICAgICB9O1xuICAgICAgc2VhcmNoRm9yTm9kZSA9IGZ1bmN0aW9uKGRhdGEsIG5vZGVJRCkge1xuICAgICAgICB2YXIgZWwsIGksIGo7XG4gICAgICAgIGZvciAoaSBpbiBkYXRhLm5vZGVzKSB7XG4gICAgICAgICAgZWwgPSBkYXRhLm5vZGVzW2ldO1xuICAgICAgICAgIGlmIChlbC5pZCA9PT0gbm9kZUlEKSB7XG4gICAgICAgICAgICByZXR1cm4gZWw7XG4gICAgICAgICAgfVxuICAgICAgICAgIGlmIChlbC5zdGVwX2Z1bmN0aW9uICE9IG51bGwpIHtcbiAgICAgICAgICAgIGZvciAoaiBpbiBlbC5zdGVwX2Z1bmN0aW9uKSB7XG4gICAgICAgICAgICAgIGlmIChlbC5zdGVwX2Z1bmN0aW9uW2pdLmlkID09PSBub2RlSUQpIHtcbiAgICAgICAgICAgICAgICByZXR1cm4gZWwuc3RlcF9mdW5jdGlvbltqXTtcbiAgICAgICAgICAgICAgfVxuICAgICAgICAgICAgfVxuICAgICAgICAgIH1cbiAgICAgICAgfVxuICAgICAgfTtcbiAgICAgIGRyYXdHcmFwaCA9IGZ1bmN0aW9uKGRhdGEpIHtcbiAgICAgICAgdmFyIGcsIGksIG5ld1NjYWxlLCByZW5kZXJlciwgc2csIHhDZW50ZXJPZmZzZXQsIHlDZW50ZXJPZmZzZXQ7XG4gICAgICAgIGcgPSBuZXcgZGFncmVEMy5ncmFwaGxpYi5HcmFwaCh7XG4gICAgICAgICAgbXVsdGlncmFwaDogdHJ1ZSxcbiAgICAgICAgICBjb21wb3VuZDogdHJ1ZVxuICAgICAgICB9KS5zZXRHcmFwaCh7XG4gICAgICAgICAgbm9kZXNlcDogNzAsXG4gICAgICAgICAgZWRnZXNlcDogMCxcbiAgICAgICAgICByYW5rc2VwOiA1MCxcbiAgICAgICAgICByYW5rZGlyOiBcIkxSXCIsXG4gICAgICAgICAgbWFyZ2lueDogNDAsXG4gICAgICAgICAgbWFyZ2lueTogNDBcbiAgICAgICAgfSk7XG4gICAgICAgIGxvYWRKc29uVG9EYWdyZShnLCBkYXRhKTtcbiAgICAgICAgcmVuZGVyZXIgPSBuZXcgZGFncmVEMy5yZW5kZXIoKTtcbiAgICAgICAgZDNtYWluU3ZnRy5jYWxsKHJlbmRlcmVyLCBnKTtcbiAgICAgICAgZm9yIChpIGluIHN1YmdyYXBocykge1xuICAgICAgICAgIHNnID0gc3ViZ3JhcGhzW2ldO1xuICAgICAgICAgIGQzbWFpblN2Zy5zZWxlY3QoJ3N2Zy5zdmctJyArIGkgKyAnIGcnKS5jYWxsKHJlbmRlcmVyLCBzZyk7XG4gICAgICAgIH1cbiAgICAgICAgbmV3U2NhbGUgPSAwLjU7XG4gICAgICAgIHhDZW50ZXJPZmZzZXQgPSBNYXRoLmZsb29yKChhbmd1bGFyLmVsZW1lbnQobWFpblN2Z0VsZW1lbnQpLndpZHRoKCkgLSBnLmdyYXBoKCkud2lkdGggKiBuZXdTY2FsZSkgLyAyKTtcbiAgICAgICAgeUNlbnRlck9mZnNldCA9IE1hdGguZmxvb3IoKGFuZ3VsYXIuZWxlbWVudChtYWluU3ZnRWxlbWVudCkuaGVpZ2h0KCkgLSBnLmdyYXBoKCkuaGVpZ2h0ICogbmV3U2NhbGUpIC8gMik7XG4gICAgICAgIG1haW5ab29tLnNjYWxlKG5ld1NjYWxlKS50cmFuc2xhdGUoW3hDZW50ZXJPZmZzZXQsIHlDZW50ZXJPZmZzZXRdKTtcbiAgICAgICAgZDNtYWluU3ZnRy5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICsgeENlbnRlck9mZnNldCArIFwiLCBcIiArIHlDZW50ZXJPZmZzZXQgKyBcIikgc2NhbGUoXCIgKyBtYWluWm9vbS5zY2FsZSgpICsgXCIpXCIpO1xuICAgICAgICBtYWluWm9vbS5vbihcInpvb21cIiwgZnVuY3Rpb24oKSB7XG4gICAgICAgICAgdmFyIGV2O1xuICAgICAgICAgIGV2ID0gZDMuZXZlbnQ7XG4gICAgICAgICAgcmV0dXJuIGQzbWFpblN2Z0cuYXR0cihcInRyYW5zZm9ybVwiLCBcInRyYW5zbGF0ZShcIiArIGV2LnRyYW5zbGF0ZSArIFwiKSBzY2FsZShcIiArIGV2LnNjYWxlICsgXCIpXCIpO1xuICAgICAgICB9KTtcbiAgICAgICAgcmV0dXJuIG1haW5ab29tKGQzbWFpblN2Zyk7XG4gICAgICB9O1xuICAgICAgc2NvcGUuJHdhdGNoKGF0dHJzLnBsYW4sIGZ1bmN0aW9uKG5ld1BsYW4pIHtcbiAgICAgICAgaWYgKG5ld1BsYW4pIHtcbiAgICAgICAgICByZXR1cm4gZHJhd0dyYXBoKG5ld1BsYW4pO1xuICAgICAgICB9XG4gICAgICB9KTtcbiAgICB9XG4gIH07XG59KTtcbiIsIiNcbiMgTGljZW5zZWQgdG8gdGhlIEFwYWNoZSBTb2Z0d2FyZSBGb3VuZGF0aW9uIChBU0YpIHVuZGVyIG9uZVxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcbiMgZGlzdHJpYnV0ZWQgd2l0aCB0aGlzIHdvcmsgZm9yIGFkZGl0aW9uYWwgaW5mb3JtYXRpb25cbiMgcmVnYXJkaW5nIGNvcHlyaWdodCBvd25lcnNoaXAuICBUaGUgQVNGIGxpY2Vuc2VzIHRoaXMgZmlsZVxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXG4jIFwiTGljZW5zZVwiKTsgeW91IG1heSBub3QgdXNlIHRoaXMgZmlsZSBleGNlcHQgaW4gY29tcGxpYW5jZVxuIyB3aXRoIHRoZSBMaWNlbnNlLiAgWW91IG1heSBvYnRhaW4gYSBjb3B5IG9mIHRoZSBMaWNlbnNlIGF0XG4jXG4jICAgICBodHRwOi8vd3d3LmFwYWNoZS5vcmcvbGljZW5zZXMvTElDRU5TRS0yLjBcbiNcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxuIyBkaXN0cmlidXRlZCB1bmRlciB0aGUgTGljZW5zZSBpcyBkaXN0cmlidXRlZCBvbiBhbiBcIkFTIElTXCIgQkFTSVMsXG4jIFdJVEhPVVQgV0FSUkFOVElFUyBPUiBDT05ESVRJT05TIE9GIEFOWSBLSU5ELCBlaXRoZXIgZXhwcmVzcyBvciBpbXBsaWVkLlxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXG4jIGxpbWl0YXRpb25zIHVuZGVyIHRoZSBMaWNlbnNlLlxuI1xuXG5hbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKVxuXG4uc2VydmljZSAnSm9ic1NlcnZpY2UnLCAoJGh0dHAsIGZsaW5rQ29uZmlnLCAkbG9nLCBhbU1vbWVudCwgJHEsICR0aW1lb3V0KSAtPlxuICBjdXJyZW50Sm9iID0gbnVsbFxuICBjdXJyZW50UGxhbiA9IG51bGxcbiAgZGVmZXJyZWRzID0ge31cbiAgam9icyA9IHtcbiAgICBydW5uaW5nOiBbXVxuICAgIGZpbmlzaGVkOiBbXVxuICAgIGNhbmNlbGxlZDogW11cbiAgICBmYWlsZWQ6IFtdXG4gIH1cblxuICBqb2JPYnNlcnZlcnMgPSBbXVxuXG4gIG5vdGlmeU9ic2VydmVycyA9IC0+XG4gICAgYW5ndWxhci5mb3JFYWNoIGpvYk9ic2VydmVycywgKGNhbGxiYWNrKSAtPlxuICAgICAgY2FsbGJhY2soKVxuXG4gIEByZWdpc3Rlck9ic2VydmVyID0gKGNhbGxiYWNrKSAtPlxuICAgIGpvYk9ic2VydmVycy5wdXNoKGNhbGxiYWNrKVxuXG4gIEB1blJlZ2lzdGVyT2JzZXJ2ZXIgPSAoY2FsbGJhY2spIC0+XG4gICAgaW5kZXggPSBqb2JPYnNlcnZlcnMuaW5kZXhPZihjYWxsYmFjaylcbiAgICBqb2JPYnNlcnZlcnMuc3BsaWNlKGluZGV4LCAxKVxuXG4gIEBzdGF0ZUxpc3QgPSAtPlxuICAgIFsgXG4gICAgICAjICdDUkVBVEVEJ1xuICAgICAgJ1NDSEVEVUxFRCdcbiAgICAgICdERVBMT1lJTkcnXG4gICAgICAnUlVOTklORydcbiAgICAgICdGSU5JU0hFRCdcbiAgICAgICdGQUlMRUQnXG4gICAgICAnQ0FOQ0VMSU5HJ1xuICAgICAgJ0NBTkNFTEVEJ1xuICAgIF1cblxuICBAdHJhbnNsYXRlTGFiZWxTdGF0ZSA9IChzdGF0ZSkgLT5cbiAgICBzd2l0Y2ggc3RhdGUudG9Mb3dlckNhc2UoKVxuICAgICAgd2hlbiAnZmluaXNoZWQnIHRoZW4gJ3N1Y2Nlc3MnXG4gICAgICB3aGVuICdmYWlsZWQnIHRoZW4gJ2RhbmdlcidcbiAgICAgIHdoZW4gJ3NjaGVkdWxlZCcgdGhlbiAnZGVmYXVsdCdcbiAgICAgIHdoZW4gJ2RlcGxveWluZycgdGhlbiAnaW5mbydcbiAgICAgIHdoZW4gJ3J1bm5pbmcnIHRoZW4gJ3ByaW1hcnknXG4gICAgICB3aGVuICdjYW5jZWxpbmcnIHRoZW4gJ3dhcm5pbmcnXG4gICAgICB3aGVuICdwZW5kaW5nJyB0aGVuICdpbmZvJ1xuICAgICAgd2hlbiAndG90YWwnIHRoZW4gJ2JsYWNrJ1xuICAgICAgZWxzZSAnZGVmYXVsdCdcblxuICBAbGlzdEpvYnMgPSAtPlxuICAgIGRlZmVycmVkID0gJHEuZGVmZXIoKVxuXG4gICAgJGh0dHAuZ2V0IGZsaW5rQ29uZmlnLm5ld1NlcnZlciArIFwiL2pvYnNcIlxuICAgIC5zdWNjZXNzIChkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykgLT5cblxuICAgICAgYW5ndWxhci5mb3JFYWNoIGRhdGEsIChsaXN0LCBsaXN0S2V5KSAtPlxuXG4gICAgICAgIHN3aXRjaCBsaXN0S2V5XG4gICAgICAgICAgd2hlbiAnam9icy1ydW5uaW5nJyB0aGVuIGpvYnMucnVubmluZyA9IGxpc3RcbiAgICAgICAgICB3aGVuICdqb2JzLWZpbmlzaGVkJyB0aGVuIGpvYnMuZmluaXNoZWQgPSBsaXN0XG4gICAgICAgICAgd2hlbiAnam9icy1jYW5jZWxsZWQnIHRoZW4gam9icy5jYW5jZWxsZWQgPSBsaXN0XG4gICAgICAgICAgd2hlbiAnam9icy1mYWlsZWQnIHRoZW4gam9icy5mYWlsZWQgPSBsaXN0XG5cbiAgICAgICAgYW5ndWxhci5mb3JFYWNoIGxpc3QsIChqb2JpZCwgaW5kZXgpIC0+XG4gICAgICAgICAgJGh0dHAuZ2V0IGZsaW5rQ29uZmlnLm5ld1NlcnZlciArIFwiL2pvYnMvXCIgKyBqb2JpZFxuICAgICAgICAgIC5zdWNjZXNzIChkZXRhaWxzKSAtPlxuICAgICAgICAgICAgbGlzdFtpbmRleF0gPSBkZXRhaWxzXG5cbiAgICAgIGRlZmVycmVkLnJlc29sdmUoam9icylcbiAgICAgIG5vdGlmeU9ic2VydmVycygpXG5cbiAgICBkZWZlcnJlZC5wcm9taXNlXG5cbiAgQGdldEpvYnMgPSAodHlwZSkgLT5cbiAgICBqb2JzW3R5cGVdXG5cbiAgQGdldEFsbEpvYnMgPSAtPlxuICAgIGpvYnNcblxuICBAbG9hZEpvYiA9IChqb2JpZCkgLT5cbiAgICBjdXJyZW50Sm9iID0gbnVsbFxuICAgIGRlZmVycmVkcy5qb2IgPSAkcS5kZWZlcigpXG5cbiAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkXG4gICAgLnN1Y2Nlc3MgKGRhdGEsIHN0YXR1cywgaGVhZGVycywgY29uZmlnKSAtPlxuICAgICAgZGF0YS50aW1lID0gRGF0ZS5ub3coKVxuXG4gICAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkICsgXCIvdmVydGljZXNcIlxuICAgICAgLnN1Y2Nlc3MgKHZlcnRpY2VzKSAtPlxuICAgICAgICBkYXRhID0gYW5ndWxhci5leHRlbmQoZGF0YSwgdmVydGljZXMpXG5cbiAgICAgICAgJGh0dHAuZ2V0IGZsaW5rQ29uZmlnLmpvYlNlcnZlciArIFwiL2pvYnNJbmZvP2dldD1qb2Imam9iPVwiICsgam9iaWRcbiAgICAgICAgLnN1Y2Nlc3MgKG9sZFZlcnRpY2VzKSAtPlxuICAgICAgICAgIGRhdGEub2xkViA9IG9sZFZlcnRpY2VzWzBdXG5cbiAgICAgICAgICAkaHR0cC5nZXQgZmxpbmtDb25maWcuam9iU2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkICsgXCIvY29uZmlnXCJcbiAgICAgICAgICAuc3VjY2VzcyAoam9iQ29uZmlnKSAtPlxuICAgICAgICAgICAgZGF0YSA9IGFuZ3VsYXIuZXh0ZW5kKGRhdGEsIGpvYkNvbmZpZylcblxuICAgICAgICAgICAgY3VycmVudEpvYiA9IGRhdGFcbiAgICAgICAgICAgIGRlZmVycmVkcy5qb2IucmVzb2x2ZShkYXRhKVxuXG4gICAgZGVmZXJyZWRzLmpvYi5wcm9taXNlXG5cbiAgQGxvYWRQbGFuID0gKGpvYmlkKSAtPlxuICAgIGN1cnJlbnRQbGFuID0gbnVsbFxuICAgIGRlZmVycmVkcy5wbGFuID0gJHEuZGVmZXIoKVxuXG4gICAgJGh0dHAuZ2V0IGZsaW5rQ29uZmlnLm5ld1NlcnZlciArIFwiL2pvYnMvXCIgKyBqb2JpZCArIFwiL3BsYW5cIlxuICAgIC5zdWNjZXNzIChkYXRhKSAtPlxuICAgICAgY3VycmVudFBsYW4gPSBkYXRhXG5cbiAgICAgIGRlZmVycmVkcy5wbGFuLnJlc29sdmUoZGF0YSlcblxuICAgIGRlZmVycmVkcy5wbGFuLnByb21pc2VcblxuICBAZ2V0Tm9kZSA9IChub2RlaWQpIC0+XG4gICAgc2Vla05vZGUgPSAobm9kZWlkLCBkYXRhKSAtPlxuICAgICAgbm9kZWlkID0gcGFyc2VJbnQobm9kZWlkKVxuXG4gICAgICBmb3Igbm9kZSBpbiBkYXRhXG4gICAgICAgIHJldHVybiBub2RlIGlmIG5vZGUuaWQgaXMgbm9kZWlkXG4gICAgICAgIHN1YiA9IHNlZWtOb2RlKG5vZGVpZCwgbm9kZS5zdGVwX2Z1bmN0aW9uKSBpZiBub2RlLnN0ZXBfZnVuY3Rpb25cbiAgICAgICAgcmV0dXJuIHN1YiBpZiBzdWJcblxuICAgICAgbnVsbFxuXG4gICAgZGVmZXJyZWQgPSAkcS5kZWZlcigpXG5cbiAgICAjIGlmIGN1cnJlbnRQbGFuXG4gICAgIyAgIGRlZmVycmVkLnJlc29sdmUoc2Vla05vZGUobm9kZWlkLCBjdXJyZW50UGxhbi5ub2RlcykpXG4gICAgIyBlbHNlXG4gICAgIyAgICMgZGVmZXJyZWRzLnBsYW4ucHJvbWlzZS50aGVuIChkYXRhKSAtPlxuICAgICMgICAkcS5hbGwoW2RlZmVycmVkcy5wbGFuLnByb21pc2UsIGRlZmVycmVkcy5qb2IucHJvbWlzZV0pLnRoZW4gKGRhdGEpIC0+XG4gICAgIyAgICAgY29uc29sZS5sb2cgJ3Jlc29sdmluZyBnZXROb2RlJ1xuICAgICMgICAgIGRlZmVycmVkLnJlc29sdmUoc2Vla05vZGUobm9kZWlkLCBjdXJyZW50UGxhbi5ub2RlcykpXG5cbiAgICAkcS5hbGwoW2RlZmVycmVkcy5wbGFuLnByb21pc2UsIGRlZmVycmVkcy5qb2IucHJvbWlzZV0pLnRoZW4gKGRhdGEpID0+XG4gICAgICBmb3VuZE5vZGUgPSBzZWVrTm9kZShub2RlaWQsIGN1cnJlbnRQbGFuLm5vZGVzKVxuXG4gICAgICAjIFRPRE8gbGluayB0byByZWFsIHZlcnRleC4gZm9yIG5vdyB0aGVyZSBpcyBubyB3YXkgdG8gZ2V0IHRoZSByaWdodCBvbmUsIHNvIHdlIGFyZSBzaG93aW5nIHRoZSBmaXJzdCBvbmUgLSBqdXN0IGZvciB0ZXN0aW5nXG4gICAgICBAZ2V0VmVydGV4KGN1cnJlbnRKb2IuamlkLCBjdXJyZW50Sm9iLm9sZFYuZ3JvdXB2ZXJ0aWNlc1swXS5ncm91cHZlcnRleGlkKS50aGVuICh2ZXJ0ZXgpIC0+XG4gICAgICAgIGZvdW5kTm9kZS52ZXJ0ZXggPSB2ZXJ0ZXhcbiAgICAgICAgZGVmZXJyZWQucmVzb2x2ZShmb3VuZE5vZGUpXG5cbiAgICBkZWZlcnJlZC5wcm9taXNlXG5cblxuICBAZ2V0VmVydGV4ID0gKGpvYklkLCB2ZXJ0ZXhJZCkgLT5cbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKClcblxuICAgICRodHRwLmdldCBmbGlua0NvbmZpZy5qb2JTZXJ2ZXIgKyBcIi9qb2JzSW5mbz9nZXQ9Z3JvdXB2ZXJ0ZXgmam9iPVwiICsgam9iSWQgKyBcIiZncm91cHZlcnRleD1cIiArIHZlcnRleElkXG4gICAgLnN1Y2Nlc3MgKGRhdGEpIC0+XG4gICAgICBkZWZlcnJlZC5yZXNvbHZlKGRhdGEpXG5cbiAgICBkZWZlcnJlZC5wcm9taXNlXG5cbiAgQFxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuc2VydmljZSgnSm9ic1NlcnZpY2UnLCBmdW5jdGlvbigkaHR0cCwgZmxpbmtDb25maWcsICRsb2csIGFtTW9tZW50LCAkcSwgJHRpbWVvdXQpIHtcbiAgdmFyIGN1cnJlbnRKb2IsIGN1cnJlbnRQbGFuLCBkZWZlcnJlZHMsIGpvYk9ic2VydmVycywgam9icywgbm90aWZ5T2JzZXJ2ZXJzO1xuICBjdXJyZW50Sm9iID0gbnVsbDtcbiAgY3VycmVudFBsYW4gPSBudWxsO1xuICBkZWZlcnJlZHMgPSB7fTtcbiAgam9icyA9IHtcbiAgICBydW5uaW5nOiBbXSxcbiAgICBmaW5pc2hlZDogW10sXG4gICAgY2FuY2VsbGVkOiBbXSxcbiAgICBmYWlsZWQ6IFtdXG4gIH07XG4gIGpvYk9ic2VydmVycyA9IFtdO1xuICBub3RpZnlPYnNlcnZlcnMgPSBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gYW5ndWxhci5mb3JFYWNoKGpvYk9ic2VydmVycywgZnVuY3Rpb24oY2FsbGJhY2spIHtcbiAgICAgIHJldHVybiBjYWxsYmFjaygpO1xuICAgIH0pO1xuICB9O1xuICB0aGlzLnJlZ2lzdGVyT2JzZXJ2ZXIgPSBmdW5jdGlvbihjYWxsYmFjaykge1xuICAgIHJldHVybiBqb2JPYnNlcnZlcnMucHVzaChjYWxsYmFjayk7XG4gIH07XG4gIHRoaXMudW5SZWdpc3Rlck9ic2VydmVyID0gZnVuY3Rpb24oY2FsbGJhY2spIHtcbiAgICB2YXIgaW5kZXg7XG4gICAgaW5kZXggPSBqb2JPYnNlcnZlcnMuaW5kZXhPZihjYWxsYmFjayk7XG4gICAgcmV0dXJuIGpvYk9ic2VydmVycy5zcGxpY2UoaW5kZXgsIDEpO1xuICB9O1xuICB0aGlzLnN0YXRlTGlzdCA9IGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiBbJ1NDSEVEVUxFRCcsICdERVBMT1lJTkcnLCAnUlVOTklORycsICdGSU5JU0hFRCcsICdGQUlMRUQnLCAnQ0FOQ0VMSU5HJywgJ0NBTkNFTEVEJ107XG4gIH07XG4gIHRoaXMudHJhbnNsYXRlTGFiZWxTdGF0ZSA9IGZ1bmN0aW9uKHN0YXRlKSB7XG4gICAgc3dpdGNoIChzdGF0ZS50b0xvd2VyQ2FzZSgpKSB7XG4gICAgICBjYXNlICdmaW5pc2hlZCc6XG4gICAgICAgIHJldHVybiAnc3VjY2Vzcyc7XG4gICAgICBjYXNlICdmYWlsZWQnOlxuICAgICAgICByZXR1cm4gJ2Rhbmdlcic7XG4gICAgICBjYXNlICdzY2hlZHVsZWQnOlxuICAgICAgICByZXR1cm4gJ2RlZmF1bHQnO1xuICAgICAgY2FzZSAnZGVwbG95aW5nJzpcbiAgICAgICAgcmV0dXJuICdpbmZvJztcbiAgICAgIGNhc2UgJ3J1bm5pbmcnOlxuICAgICAgICByZXR1cm4gJ3ByaW1hcnknO1xuICAgICAgY2FzZSAnY2FuY2VsaW5nJzpcbiAgICAgICAgcmV0dXJuICd3YXJuaW5nJztcbiAgICAgIGNhc2UgJ3BlbmRpbmcnOlxuICAgICAgICByZXR1cm4gJ2luZm8nO1xuICAgICAgY2FzZSAndG90YWwnOlxuICAgICAgICByZXR1cm4gJ2JsYWNrJztcbiAgICAgIGRlZmF1bHQ6XG4gICAgICAgIHJldHVybiAnZGVmYXVsdCc7XG4gICAgfVxuICB9O1xuICB0aGlzLmxpc3RKb2JzID0gZnVuY3Rpb24oKSB7XG4gICAgdmFyIGRlZmVycmVkO1xuICAgIGRlZmVycmVkID0gJHEuZGVmZXIoKTtcbiAgICAkaHR0cC5nZXQoZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9ic1wiKS5zdWNjZXNzKGZ1bmN0aW9uKGRhdGEsIHN0YXR1cywgaGVhZGVycywgY29uZmlnKSB7XG4gICAgICBhbmd1bGFyLmZvckVhY2goZGF0YSwgZnVuY3Rpb24obGlzdCwgbGlzdEtleSkge1xuICAgICAgICBzd2l0Y2ggKGxpc3RLZXkpIHtcbiAgICAgICAgICBjYXNlICdqb2JzLXJ1bm5pbmcnOlxuICAgICAgICAgICAgam9icy5ydW5uaW5nID0gbGlzdDtcbiAgICAgICAgICAgIGJyZWFrO1xuICAgICAgICAgIGNhc2UgJ2pvYnMtZmluaXNoZWQnOlxuICAgICAgICAgICAgam9icy5maW5pc2hlZCA9IGxpc3Q7XG4gICAgICAgICAgICBicmVhaztcbiAgICAgICAgICBjYXNlICdqb2JzLWNhbmNlbGxlZCc6XG4gICAgICAgICAgICBqb2JzLmNhbmNlbGxlZCA9IGxpc3Q7XG4gICAgICAgICAgICBicmVhaztcbiAgICAgICAgICBjYXNlICdqb2JzLWZhaWxlZCc6XG4gICAgICAgICAgICBqb2JzLmZhaWxlZCA9IGxpc3Q7XG4gICAgICAgIH1cbiAgICAgICAgcmV0dXJuIGFuZ3VsYXIuZm9yRWFjaChsaXN0LCBmdW5jdGlvbihqb2JpZCwgaW5kZXgpIHtcbiAgICAgICAgICByZXR1cm4gJGh0dHAuZ2V0KGZsaW5rQ29uZmlnLm5ld1NlcnZlciArIFwiL2pvYnMvXCIgKyBqb2JpZCkuc3VjY2VzcyhmdW5jdGlvbihkZXRhaWxzKSB7XG4gICAgICAgICAgICByZXR1cm4gbGlzdFtpbmRleF0gPSBkZXRhaWxzO1xuICAgICAgICAgIH0pO1xuICAgICAgICB9KTtcbiAgICAgIH0pO1xuICAgICAgZGVmZXJyZWQucmVzb2x2ZShqb2JzKTtcbiAgICAgIHJldHVybiBub3RpZnlPYnNlcnZlcnMoKTtcbiAgICB9KTtcbiAgICByZXR1cm4gZGVmZXJyZWQucHJvbWlzZTtcbiAgfTtcbiAgdGhpcy5nZXRKb2JzID0gZnVuY3Rpb24odHlwZSkge1xuICAgIHJldHVybiBqb2JzW3R5cGVdO1xuICB9O1xuICB0aGlzLmdldEFsbEpvYnMgPSBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gam9icztcbiAgfTtcbiAgdGhpcy5sb2FkSm9iID0gZnVuY3Rpb24oam9iaWQpIHtcbiAgICBjdXJyZW50Sm9iID0gbnVsbDtcbiAgICBkZWZlcnJlZHMuam9iID0gJHEuZGVmZXIoKTtcbiAgICAkaHR0cC5nZXQoZmxpbmtDb25maWcubmV3U2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkKS5zdWNjZXNzKGZ1bmN0aW9uKGRhdGEsIHN0YXR1cywgaGVhZGVycywgY29uZmlnKSB7XG4gICAgICBkYXRhLnRpbWUgPSBEYXRlLm5vdygpO1xuICAgICAgcmV0dXJuICRodHRwLmdldChmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzL1wiICsgam9iaWQgKyBcIi92ZXJ0aWNlc1wiKS5zdWNjZXNzKGZ1bmN0aW9uKHZlcnRpY2VzKSB7XG4gICAgICAgIGRhdGEgPSBhbmd1bGFyLmV4dGVuZChkYXRhLCB2ZXJ0aWNlcyk7XG4gICAgICAgIHJldHVybiAkaHR0cC5nZXQoZmxpbmtDb25maWcuam9iU2VydmVyICsgXCIvam9ic0luZm8/Z2V0PWpvYiZqb2I9XCIgKyBqb2JpZCkuc3VjY2VzcyhmdW5jdGlvbihvbGRWZXJ0aWNlcykge1xuICAgICAgICAgIGRhdGEub2xkViA9IG9sZFZlcnRpY2VzWzBdO1xuICAgICAgICAgIHJldHVybiAkaHR0cC5nZXQoZmxpbmtDb25maWcuam9iU2VydmVyICsgXCIvam9icy9cIiArIGpvYmlkICsgXCIvY29uZmlnXCIpLnN1Y2Nlc3MoZnVuY3Rpb24oam9iQ29uZmlnKSB7XG4gICAgICAgICAgICBkYXRhID0gYW5ndWxhci5leHRlbmQoZGF0YSwgam9iQ29uZmlnKTtcbiAgICAgICAgICAgIGN1cnJlbnRKb2IgPSBkYXRhO1xuICAgICAgICAgICAgcmV0dXJuIGRlZmVycmVkcy5qb2IucmVzb2x2ZShkYXRhKTtcbiAgICAgICAgICB9KTtcbiAgICAgICAgfSk7XG4gICAgICB9KTtcbiAgICB9KTtcbiAgICByZXR1cm4gZGVmZXJyZWRzLmpvYi5wcm9taXNlO1xuICB9O1xuICB0aGlzLmxvYWRQbGFuID0gZnVuY3Rpb24oam9iaWQpIHtcbiAgICBjdXJyZW50UGxhbiA9IG51bGw7XG4gICAgZGVmZXJyZWRzLnBsYW4gPSAkcS5kZWZlcigpO1xuICAgICRodHRwLmdldChmbGlua0NvbmZpZy5uZXdTZXJ2ZXIgKyBcIi9qb2JzL1wiICsgam9iaWQgKyBcIi9wbGFuXCIpLnN1Y2Nlc3MoZnVuY3Rpb24oZGF0YSkge1xuICAgICAgY3VycmVudFBsYW4gPSBkYXRhO1xuICAgICAgcmV0dXJuIGRlZmVycmVkcy5wbGFuLnJlc29sdmUoZGF0YSk7XG4gICAgfSk7XG4gICAgcmV0dXJuIGRlZmVycmVkcy5wbGFuLnByb21pc2U7XG4gIH07XG4gIHRoaXMuZ2V0Tm9kZSA9IGZ1bmN0aW9uKG5vZGVpZCkge1xuICAgIHZhciBkZWZlcnJlZCwgc2Vla05vZGU7XG4gICAgc2Vla05vZGUgPSBmdW5jdGlvbihub2RlaWQsIGRhdGEpIHtcbiAgICAgIHZhciBpLCBsZW4sIG5vZGUsIHN1YjtcbiAgICAgIG5vZGVpZCA9IHBhcnNlSW50KG5vZGVpZCk7XG4gICAgICBmb3IgKGkgPSAwLCBsZW4gPSBkYXRhLmxlbmd0aDsgaSA8IGxlbjsgaSsrKSB7XG4gICAgICAgIG5vZGUgPSBkYXRhW2ldO1xuICAgICAgICBpZiAobm9kZS5pZCA9PT0gbm9kZWlkKSB7XG4gICAgICAgICAgcmV0dXJuIG5vZGU7XG4gICAgICAgIH1cbiAgICAgICAgaWYgKG5vZGUuc3RlcF9mdW5jdGlvbikge1xuICAgICAgICAgIHN1YiA9IHNlZWtOb2RlKG5vZGVpZCwgbm9kZS5zdGVwX2Z1bmN0aW9uKTtcbiAgICAgICAgfVxuICAgICAgICBpZiAoc3ViKSB7XG4gICAgICAgICAgcmV0dXJuIHN1YjtcbiAgICAgICAgfVxuICAgICAgfVxuICAgICAgcmV0dXJuIG51bGw7XG4gICAgfTtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJHEuYWxsKFtkZWZlcnJlZHMucGxhbi5wcm9taXNlLCBkZWZlcnJlZHMuam9iLnByb21pc2VdKS50aGVuKChmdW5jdGlvbihfdGhpcykge1xuICAgICAgcmV0dXJuIGZ1bmN0aW9uKGRhdGEpIHtcbiAgICAgICAgdmFyIGZvdW5kTm9kZTtcbiAgICAgICAgZm91bmROb2RlID0gc2Vla05vZGUobm9kZWlkLCBjdXJyZW50UGxhbi5ub2Rlcyk7XG4gICAgICAgIHJldHVybiBfdGhpcy5nZXRWZXJ0ZXgoY3VycmVudEpvYi5qaWQsIGN1cnJlbnRKb2Iub2xkVi5ncm91cHZlcnRpY2VzWzBdLmdyb3VwdmVydGV4aWQpLnRoZW4oZnVuY3Rpb24odmVydGV4KSB7XG4gICAgICAgICAgZm91bmROb2RlLnZlcnRleCA9IHZlcnRleDtcbiAgICAgICAgICByZXR1cm4gZGVmZXJyZWQucmVzb2x2ZShmb3VuZE5vZGUpO1xuICAgICAgICB9KTtcbiAgICAgIH07XG4gICAgfSkodGhpcykpO1xuICAgIHJldHVybiBkZWZlcnJlZC5wcm9taXNlO1xuICB9O1xuICB0aGlzLmdldFZlcnRleCA9IGZ1bmN0aW9uKGpvYklkLCB2ZXJ0ZXhJZCkge1xuICAgIHZhciBkZWZlcnJlZDtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJGh0dHAuZ2V0KGZsaW5rQ29uZmlnLmpvYlNlcnZlciArIFwiL2pvYnNJbmZvP2dldD1ncm91cHZlcnRleCZqb2I9XCIgKyBqb2JJZCArIFwiJmdyb3VwdmVydGV4PVwiICsgdmVydGV4SWQpLnN1Y2Nlc3MoZnVuY3Rpb24oZGF0YSkge1xuICAgICAgcmV0dXJuIGRlZmVycmVkLnJlc29sdmUoZGF0YSk7XG4gICAgfSk7XG4gICAgcmV0dXJuIGRlZmVycmVkLnByb21pc2U7XG4gIH07XG4gIHJldHVybiB0aGlzO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLmNvbnRyb2xsZXIgJ092ZXJ2aWV3Q29udHJvbGxlcicsICgkc2NvcGUsIE92ZXJ2aWV3U2VydmljZSwgSm9ic1NlcnZpY2UpIC0+XG4gICRzY29wZS5qb2JPYnNlcnZlciA9IC0+XG4gICAgJHNjb3BlLnJ1bm5pbmdKb2JzID0gSm9ic1NlcnZpY2UuZ2V0Sm9icygncnVubmluZycpXG4gICAgJHNjb3BlLmZpbmlzaGVkSm9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ2ZpbmlzaGVkJylcblxuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcbiAgJHNjb3BlLiRvbiAnJGRlc3Ryb3knLCAtPlxuICAgIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpXG5cbiAgJHNjb3BlLmpvYk9ic2VydmVyKClcbiIsImFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpLmNvbnRyb2xsZXIoJ092ZXJ2aWV3Q29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgT3ZlcnZpZXdTZXJ2aWNlLCBKb2JzU2VydmljZSkge1xuICAkc2NvcGUuam9iT2JzZXJ2ZXIgPSBmdW5jdGlvbigpIHtcbiAgICAkc2NvcGUucnVubmluZ0pvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdydW5uaW5nJyk7XG4gICAgcmV0dXJuICRzY29wZS5maW5pc2hlZEpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdmaW5pc2hlZCcpO1xuICB9O1xuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5qb2JPYnNlcnZlcigpO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLnNlcnZpY2UgJ092ZXJ2aWV3U2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRsb2cpIC0+XG4gIHNlcnZlclN0YXR1cyA9IHt9XG5cbiAgQGxvYWRTZXJ2ZXJTdGF0dXMgPSAtPlxuICAgICRodHRwLmdldChmbGlua0NvbmZpZy5qb2JTZXJ2ZXIgKyBcIi9tb25pdG9yL3N0YXR1c1wiKVxuICAgIC5zdWNjZXNzIChkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykgLT5cbiAgICAgICRsb2cgZGF0YVxuXG4gICAgLmVycm9yIChkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykgLT5cbiAgICAgIHJldHVyblxuXG4gICAgc2VydmVyU3RhdHVzXG5cbiAgQFxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuc2VydmljZSgnT3ZlcnZpZXdTZXJ2aWNlJywgZnVuY3Rpb24oJGh0dHAsIGZsaW5rQ29uZmlnLCAkbG9nKSB7XG4gIHZhciBzZXJ2ZXJTdGF0dXM7XG4gIHNlcnZlclN0YXR1cyA9IHt9O1xuICB0aGlzLmxvYWRTZXJ2ZXJTdGF0dXMgPSBmdW5jdGlvbigpIHtcbiAgICAkaHR0cC5nZXQoZmxpbmtDb25maWcuam9iU2VydmVyICsgXCIvbW9uaXRvci9zdGF0dXNcIikuc3VjY2VzcyhmdW5jdGlvbihkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykge1xuICAgICAgcmV0dXJuICRsb2coZGF0YSk7XG4gICAgfSkuZXJyb3IoZnVuY3Rpb24oZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIHt9KTtcbiAgICByZXR1cm4gc2VydmVyU3RhdHVzO1xuICB9O1xuICByZXR1cm4gdGhpcztcbn0pO1xuIl0sInNvdXJjZVJvb3QiOiIvc291cmNlLyJ9 \ No newline at end of file diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html new file mode 100644 index 0000000000000..a7a5d9db79692 --- /dev/null +++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html @@ -0,0 +1,57 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + +
Execution configuration
Execution mode{{ job['execution-config']['execution-mode'] }}
Max. number of execution retries{{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }}
Job parallelism{{ job['execution-config']['job-parallelism'] === -1 ? 'auto' : job['execution-config']['job-parallelism'] }}
Object reuse mode{{ job['execution-config']['object-reuse-mode'] }}
+ + + + + + + + + + + + +
User configuration
{{property.name}}
\ No newline at end of file diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html index c72695e39b821..1995e6df09aca 100644 --- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html +++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html @@ -41,6 +41,7 @@
  • Job Accumulators / Statistics
  • Timeline
  • Exceptions
  • +
  • Configuration
  • From d73cb7369f2259e0e3a07221c659ceffa67230d5 Mon Sep 17 00:00:00 2001 From: caofangkun Date: Thu, 30 Jul 2015 16:07:40 +0800 Subject: [PATCH 055/175] [FLINK-2434] [pom] Fix pattern for exclusion of org.apache.hadoop:hadoop-yarn-common:jersey-test-framework-grizzly2 This closes #955 --- flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml | 2 +- flink-shaded-hadoop/flink-shaded-include-yarn/pom.xml | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml index 649fbbd94151f..83422f96b35aa 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml @@ -118,7 +118,7 @@ under the License. com.sun.jersey.jersey-test-framework - jersey-test-framework-grizzly2+ + jersey-test-framework-grizzly2 com.sun.jersey.jersey-test-framework diff --git a/flink-shaded-hadoop/flink-shaded-include-yarn/pom.xml b/flink-shaded-hadoop/flink-shaded-include-yarn/pom.xml index 6e2c6b97e4e28..03a27ed8d0622 100644 --- a/flink-shaded-hadoop/flink-shaded-include-yarn/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-include-yarn/pom.xml @@ -250,7 +250,7 @@ under the License. com.sun.jersey.jersey-test-framework - jersey-test-framework-grizzly2+ + jersey-test-framework-grizzly2 com.sun.jersey.jersey-test-framework @@ -394,7 +394,7 @@ under the License. com.sun.jersey.jersey-test-framework - jersey-test-framework-grizzly2+ + jersey-test-framework-grizzly2 com.sun.jersey.jersey-test-framework @@ -538,7 +538,7 @@ under the License. com.sun.jersey.jersey-test-framework - jersey-test-framework-grizzly2+ + jersey-test-framework-grizzly2 com.sun.jersey.jersey-test-framework @@ -682,7 +682,7 @@ under the License. com.sun.jersey.jersey-test-framework - jersey-test-framework-grizzly2+ + jersey-test-framework-grizzly2 com.sun.jersey.jersey-test-framework From 83102f0ea052e7f7c43c9ba6aaff0dc1c24791c9 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Thu, 30 Jul 2015 14:07:42 +0200 Subject: [PATCH 056/175] [FLINK-2436] [streaming] Make ByteStreamStateHandles more robust Closes #958 --- .../runtime/state/ByteStreamStateHandle.java | 33 ++++- .../state/ByteStreamStateHandleTest.java | 125 ++++++++++++++++++ .../flink/tachyon/FileStateHandleTest.java | 15 ++- 3 files changed, 165 insertions(+), 8 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java index 257784a5dd63e..bf2dca876f8d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java @@ -35,9 +35,14 @@ public abstract class ByteStreamStateHandle implements StateHandle private static final long serialVersionUID = -962025800339325828L; private transient Serializable state; + private boolean isWritten = false; public ByteStreamStateHandle(Serializable state) { - this.state = state; + if (state != null) { + this.state = state; + } else { + throw new RuntimeException("State cannot be null"); + } } /** @@ -54,16 +59,25 @@ public ByteStreamStateHandle(Serializable state) { public Serializable getState() throws Exception { if (!stateFetched()) { ObjectInputStream stream = new ObjectInputStream(getInputStream()); - state = (Serializable) stream.readObject(); - stream.close(); + try { + state = (Serializable) stream.readObject(); + } finally { + stream.close(); + } } return state; } private void writeObject(ObjectOutputStream oos) throws Exception { - ObjectOutputStream stream = new ObjectOutputStream(getOutputStream()); - stream.writeObject(state); - stream.close(); + if (!isWritten) { + ObjectOutputStream stream = new ObjectOutputStream(getOutputStream()); + try { + stream.writeObject(state); + isWritten = true; + } finally { + stream.close(); + } + } oos.defaultWriteObject(); } @@ -74,4 +88,11 @@ private void writeObject(ObjectOutputStream oos) throws Exception { public boolean stateFetched() { return state != null; } + + /** + * Checks whether the state has already been written to the external store + */ + public boolean isWritten() { + return isWritten; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java new file mode 100644 index 0000000000000..a7378b96589a0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java @@ -0,0 +1,125 @@ +/* + * 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.runtime.state; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +import org.apache.flink.util.InstantiationUtil; +import org.junit.Test; + +public class ByteStreamStateHandleTest { + + @Test + public void testHandle() throws Exception { + MockHandle handle; + + try { + handle = new MockHandle(null); + fail(); + } catch (RuntimeException e) { + // expected behaviour + } + + handle = new MockHandle(1); + + assertEquals(1, handle.getState()); + assertTrue(handle.stateFetched()); + assertFalse(handle.isWritten()); + assertFalse(handle.discarded); + + MockHandle handleDs = serializeDeserialize(handle); + + assertEquals(1, handle.getState()); + assertTrue(handle.stateFetched()); + assertTrue(handle.isWritten()); + assertTrue(handle.generatedOutput); + assertFalse(handle.discarded); + + assertFalse(handleDs.stateFetched()); + assertTrue(handleDs.isWritten()); + assertFalse(handleDs.generatedOutput); + assertFalse(handle.discarded); + + try { + handleDs.getState(); + fail(); + } catch (UnsupportedOperationException e) { + // good + } + + MockHandle handleDs2 = serializeDeserialize(handleDs); + + assertFalse(handleDs2.stateFetched()); + assertTrue(handleDs2.isWritten()); + assertFalse(handleDs.generatedOutput); + assertFalse(handleDs2.generatedOutput); + assertFalse(handleDs2.discarded); + + handleDs2.discardState(); + assertTrue(handleDs2.discarded); + + } + + @SuppressWarnings("unchecked") + private > X serializeDeserialize(X handle) throws IOException, + ClassNotFoundException { + byte[] serialized = InstantiationUtil.serializeObject(handle); + return (X) InstantiationUtil.deserializeObject(serialized, Thread.currentThread() + .getContextClassLoader()); + } + + private static class MockHandle extends ByteStreamStateHandle { + + private static final long serialVersionUID = 1L; + + public MockHandle(Serializable state) { + super(state); + } + + boolean discarded = false; + transient boolean generatedOutput = false; + + @Override + public void discardState() throws Exception { + discarded = true; + } + + @Override + protected OutputStream getOutputStream() throws Exception { + generatedOutput = true; + return new ByteArrayOutputStream(); + } + + @Override + protected InputStream getInputStream() throws Exception { + throw new UnsupportedOperationException(); + } + + } + +} diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java index 82b5d35ca39a3..2873c789cf92d 100644 --- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java +++ b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; @@ -90,13 +91,23 @@ public void testFileStateHandle() throws Exception { + hdPath); FileStateHandle handle = (FileStateHandle) handleProvider.createStateHandle(state); + + try { + handleProvider.createStateHandle(null); + fail(); + } catch (RuntimeException e) { + // good + } assertTrue(handle.stateFetched()); + assertFalse(handle.isWritten()); // Serialize the handle so it writes the value to hdfs SerializedValue> serializedHandle = new SerializedValue>( handle); - + + assertTrue(handle.isWritten()); + // Deserialize the handle and verify that the state is not fetched yet FileStateHandle deserializedHandle = (FileStateHandle) serializedHandle .deserializeValue(Thread.currentThread().getContextClassLoader()); @@ -107,7 +118,7 @@ public void testFileStateHandle() throws Exception { // Test whether discard removes the checkpoint file properly assertTrue(hdfs.listFiles(hdPath, true).hasNext()); - handle.discardState(); + deserializedHandle.discardState(); assertFalse(hdfs.listFiles(hdPath, true).hasNext()); } From 7ee0275804232056b144fca8e9bb41bab37cef68 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 2 Aug 2015 20:44:50 +0200 Subject: [PATCH 057/175] [FLINK-2464][tests] Add logging to BufferSpillerTest to help debug spurious failures. --- .../streaming/runtime/io/BufferSpillerTest.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java index 05ce4be84d08f..ca6df16aa90f1 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java @@ -30,6 +30,8 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Random; @@ -40,6 +42,8 @@ import static org.junit.Assert.*; public class BufferSpillerTest { + + private static final Logger LOG = LoggerFactory.getLogger(BufferSpillerTest.class); private static final int PAGE_SIZE = 4096; @@ -179,6 +183,8 @@ public void testSpillAndRollOverSimple() { @Test public void testSpillWhileReading() { + LOG.debug("Starting SpillWhileReading test"); + try { final int sequences = 10; @@ -287,7 +293,11 @@ private static void validateBuffer(BufferOrEvent boe, int expectedSize, int expe MemorySegment seg = buf.getMemorySegment(); for (int i = 0; i < expectedSize; i++) { - assertEquals("wrong buffer contents", (byte) i, seg.get(i)); + byte expected = (byte) i; + if (expected != seg.get(i)) { + fail(String.format( + "wrong buffer contents at position %s : expected=%d , found=%d", i, expected, seg.get(i))); + } } } @@ -349,6 +359,8 @@ public void run() { int numBuffersAndEvents = nextSequence.numBuffersAndEvents; int numChannels = nextSequence.numChannels; + LOG.debug("Reading sequence {}", consumedSequences); + // consume sequence seq.open(); From 416ff589e8f7db0d478abdbc602cf089d6603cf3 Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Thu, 30 Jul 2015 09:48:09 +0530 Subject: [PATCH 058/175] [FLINK-2433] [docs] Add script to build local documentation on windows This closes #954 --- docs/build_docs.bat | 57 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 docs/build_docs.bat diff --git a/docs/build_docs.bat b/docs/build_docs.bat new file mode 100644 index 0000000000000..024b513d9031c --- /dev/null +++ b/docs/build_docs.bat @@ -0,0 +1,57 @@ +::############################################################################### +:: 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. +::############################################################################### + +@echo off +:start +call jekyll -version >nul 2>&1 +if "%errorlevel%"=="0" goto check_redcarpet +echo ERROR: Could not find jekyll. +echo Please install with 'gem install jekyll' (see http://jekyllrb.com). +exit /b 1 + +:check_redcarpet +call redcarpet -version >nul 2>&1 +if "%errorlevel%"=="0" goto check_pygments +echo WARN: Could not find redcarpet. +echo Please install with 'gem install redcarpet' (see https://github.com/vmg/redcarpet). +echo Redcarpet is needed for Markdown parsing and table of contents generation. +goto check_pygments + +:check_pygments +call python -c "import pygments" >nul 2>&1 +if "%errorlevel%"=="0" goto execute +echo WARN: Could not find pygments. +echo Please install with 'sudo easy_install Pygments' (requires Python; see http://pygments.org). +echo Pygments is needed for syntax highlighting of the code examples. +goto execute + +:execute +SET "DOCS_SRC=%cd%" +SET "DOCS_DST=%DOCS_SRC%\target" + +::default jekyll command is to just build site +::if flag p is set, start the webserver too. +IF "%1"=="" GOTO :build +IF "%1"=="-p" GOTO :serve +GOTO :build + +:build +jekyll build --source %DOCS_SRC% --destination %DOCS_DST% + +:serve +jekyll serve --baseurl "" --watch --source %DOCS_SRC% --destination %DOCS_DST% From fab61a1954ff1554448e826e1d273689ed520fc3 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 29 Jul 2015 18:03:52 +0200 Subject: [PATCH 059/175] [FLINK-2409] [webserver] Replaces ActorRefs with ActorGateways in the web server to automatically decorate messages with a leader session ID. Refactored MiniCluster to also store a reference to the web server to stop it. Adds support for the new web interface for yarn Fix web server start condition This closes #959. --- .../webmonitor/ExecutionGraphHolder.java | 14 +-- .../runtime/webmonitor/WebRuntimeMonitor.java | 5 +- .../handlers/RequestJobIdsHandler.java | 14 +-- .../handlers/RequestOverviewHandler.java | 14 +-- .../legacy/JobManagerInfoHandler.java | 55 +++++------ .../jobmanager/web/JobManagerInfoServlet.java | 52 +++++----- .../jobmanager/web/SetupInfoServlet.java | 18 ++-- .../runtime/jobmanager/web/WebInfoServer.java | 8 +- .../flink/runtime/jobmanager/JobManager.scala | 97 +++++++++--------- .../minicluster/FlinkMiniCluster.scala | 50 +++++++++- .../minicluster/LocalFlinkMiniCluster.scala | 45 ++++----- .../runtime/taskmanager/TaskManager.scala | 2 +- .../runtime/testingUtils/TestingCluster.scala | 21 ++-- .../test/util/ForkableFlinkMiniCluster.scala | 41 ++++---- .../apache/flink/yarn/ApplicationMaster.scala | 98 ++++++++++++------- 15 files changed, 293 insertions(+), 241 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java index 18a548c6c2047..a017f3a38f783 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java @@ -18,12 +18,9 @@ package org.apache.flink.runtime.webmonitor; -import akka.actor.ActorRef; -import akka.pattern.Patterns; -import akka.util.Timeout; - import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.messages.JobManagerMessages; import scala.concurrent.Await; @@ -42,18 +39,18 @@ */ public class ExecutionGraphHolder { - private final ActorRef source; + private final ActorGateway source; private final FiniteDuration timeout; private final WeakHashMap cache = new WeakHashMap(); - public ExecutionGraphHolder(ActorRef source) { + public ExecutionGraphHolder(ActorGateway source) { this(source, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT); } - public ExecutionGraphHolder(ActorRef source, FiniteDuration timeout) { + public ExecutionGraphHolder(ActorGateway source, FiniteDuration timeout) { if (source == null || timeout == null) { throw new NullPointerException(); } @@ -69,8 +66,7 @@ public ExecutionGraph getExecutionGraph(JobID jid) { } try { - Timeout to = new Timeout(timeout); - Future future = Patterns.ask(source, new JobManagerMessages.RequestJob(jid), to); + Future future = source.ask(new JobManagerMessages.RequestJob(jid), timeout); Object result = Await.result(future, timeout); if (result instanceof JobManagerMessages.JobNotFound) { return null; diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java index a2095d4852dd3..006d18d2a3d91 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.webmonitor; -import akka.actor.ActorRef; - import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelInitializer; @@ -35,6 +33,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler; import org.apache.flink.runtime.webmonitor.handlers.ExecutionPlanHandler; import org.apache.flink.runtime.webmonitor.handlers.JobConfigHandler; @@ -88,7 +87,7 @@ public class WebRuntimeMonitor implements WebMonitor { private Channel serverChannel; - public WebRuntimeMonitor(Configuration config, ActorRef jobManager, ActorRef archive) throws IOException { + public WebRuntimeMonitor(Configuration config, ActorGateway jobManager, ActorGateway archive) throws IOException { // figure out where our static contents is final String configuredWebRoot = config.getString(ConfigConstants.JOB_MANAGER_WEB_DOC_ROOT_KEY, null); final String flinkRoot = config.getString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, null); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java index 1f28a01eb7c94..aa1a39f288406 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java @@ -18,10 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; -import akka.actor.ActorRef; -import akka.pattern.Patterns; -import akka.util.Timeout; - +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview; import org.apache.flink.runtime.messages.webmonitor.RequestJobsWithIDsOverview; import org.apache.flink.runtime.webmonitor.JsonFactory; @@ -40,15 +37,15 @@ */ public class RequestJobIdsHandler implements RequestHandler, RequestHandler.JsonResponse { - private final ActorRef target; + private final ActorGateway target; private final FiniteDuration timeout; - public RequestJobIdsHandler(ActorRef target) { + public RequestJobIdsHandler(ActorGateway target) { this(target, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT); } - public RequestJobIdsHandler(ActorRef target, FiniteDuration timeout) { + public RequestJobIdsHandler(ActorGateway target, FiniteDuration timeout) { if (target == null || timeout == null) { throw new NullPointerException(); } @@ -60,8 +57,7 @@ public RequestJobIdsHandler(ActorRef target, FiniteDuration timeout) { public String handleRequest(Map params) throws Exception { // we need no parameters, get all requests try { - Timeout to = new Timeout(timeout); - Future future = Patterns.ask(target, RequestJobsWithIDsOverview.getInstance(), to); + Future future = target.ask(RequestJobsWithIDsOverview.getInstance(), timeout); JobsWithIDsOverview result = (JobsWithIDsOverview) Await.result(future, timeout); return JsonFactory.generateJobsOverviewJSON(result); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java index e51a4d1b09c53..c2c00c7952328 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java @@ -18,10 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; -import akka.actor.ActorRef; -import akka.pattern.Patterns; -import akka.util.Timeout; - +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.messages.webmonitor.RequestStatusWithJobIDsOverview; import org.apache.flink.runtime.messages.webmonitor.StatusWithJobIDsOverview; import org.apache.flink.runtime.webmonitor.JsonFactory; @@ -39,16 +36,16 @@ */ public class RequestOverviewHandler implements RequestHandler, RequestHandler.JsonResponse { - private final ActorRef jobManager; + private final ActorGateway jobManager; private final FiniteDuration timeout; - public RequestOverviewHandler(ActorRef jobManager) { + public RequestOverviewHandler(ActorGateway jobManager) { this(jobManager, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT); } - public RequestOverviewHandler(ActorRef jobManager, FiniteDuration timeout) { + public RequestOverviewHandler(ActorGateway jobManager, FiniteDuration timeout) { if (jobManager == null || timeout == null) { throw new NullPointerException(); } @@ -59,8 +56,7 @@ public RequestOverviewHandler(ActorRef jobManager, FiniteDuration timeout) { @Override public String handleRequest(Map params) throws Exception { try { - Timeout to = new Timeout(timeout); - Future future = Patterns.ask(jobManager, RequestStatusWithJobIDsOverview.getInstance(), to); + Future future = jobManager.ask(RequestStatusWithJobIDsOverview.getInstance(), timeout); StatusWithJobIDsOverview result = (StatusWithJobIDsOverview) Await.result(future, timeout); return JsonFactory.generateOverviewWithJobIDsJSON(result); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java index 0a1e08c237d95..9b52736264b9c 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java @@ -18,10 +18,6 @@ package org.apache.flink.runtime.webmonitor.legacy; -import akka.actor.ActorRef; - -import akka.pattern.Patterns; -import akka.util.Timeout; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; @@ -41,6 +37,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -78,12 +75,12 @@ public class JobManagerInfoHandler extends SimpleChannelInboundHandler { private static final Charset ENCODING = Charset.forName("UTF-8"); /** Underlying JobManager */ - private final ActorRef jobmanager; - private final ActorRef archive; + private final ActorGateway jobmanager; + private final ActorGateway archive; private final FiniteDuration timeout; - public JobManagerInfoHandler(ActorRef jobmanager, ActorRef archive, FiniteDuration timeout) { + public JobManagerInfoHandler(ActorGateway jobmanager, ActorGateway archive, FiniteDuration timeout) { this.jobmanager = jobmanager; this.archive = archive; this.timeout = timeout; @@ -118,8 +115,7 @@ protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exc @SuppressWarnings("unchecked") private String handleRequest(Routed routed) throws Exception { if ("archive".equals(routed.queryParam("get"))) { - Future response = Patterns.ask(archive, ArchiveMessages.getRequestArchivedJobs(), - new Timeout(timeout)); + Future response = archive.ask(ArchiveMessages.getRequestArchivedJobs(), timeout); Object result = Await.result(response, timeout); @@ -135,8 +131,7 @@ private String handleRequest(Routed routed) throws Exception { } } else if ("jobcounts".equals(routed.queryParam("get"))) { - Future response = Patterns.ask(archive, ArchiveMessages.getRequestJobCounts(), - new Timeout(timeout)); + Future response = archive.ask(ArchiveMessages.getRequestJobCounts(), timeout); Object result = Await.result(response, timeout); @@ -152,8 +147,8 @@ else if ("jobcounts".equals(routed.queryParam("get"))) { else if ("job".equals(routed.queryParam("get"))) { String jobId = routed.queryParam("job"); - Future response = Patterns.ask(archive, new JobManagerMessages.RequestJob(JobID.fromHexString(jobId)), - new Timeout(timeout)); + Future response = archive.ask(new JobManagerMessages.RequestJob(JobID.fromHexString(jobId)), + timeout); Object result = Await.result(response, timeout); @@ -182,8 +177,8 @@ else if ("groupvertex".equals(routed.queryParam("get"))) { throw new Exception("Found null groupVertexId"); } - Future response = Patterns.ask(archive, new JobManagerMessages.RequestJob(JobID.fromHexString(jobId)), - new Timeout(timeout)); + Future response = archive.ask(new JobManagerMessages.RequestJob(JobID.fromHexString(jobId)), + timeout); Object result = Await.result(response, timeout); @@ -205,9 +200,9 @@ else if ("groupvertex".equals(routed.queryParam("get"))) { } } else if ("taskmanagers".equals(routed.queryParam("get"))) { - Future response = Patterns.ask(jobmanager, + Future response = jobmanager.ask( JobManagerMessages.getRequestNumberRegisteredTaskManager(), - new Timeout(timeout)); + timeout); Object result = Await.result(response, timeout); @@ -219,9 +214,9 @@ else if ("taskmanagers".equals(routed.queryParam("get"))) { else { final int numberOfTaskManagers = (Integer)result; - final Future responseRegisteredSlots = Patterns.ask(jobmanager, + final Future responseRegisteredSlots = jobmanager.ask( JobManagerMessages.getRequestTotalNumberOfSlots(), - new Timeout(timeout)); + timeout); final Object resultRegisteredSlots = Await.result(responseRegisteredSlots, timeout); @@ -242,8 +237,8 @@ else if ("taskmanagers".equals(routed.queryParam("get"))) { else if ("cancel".equals(routed.queryParam("get"))) { String jobId = routed.queryParam("job"); - Future response = Patterns.ask(jobmanager, new JobManagerMessages.CancelJob(JobID.fromHexString(jobId)), - new Timeout(timeout)); + Future response = jobmanager.ask(new JobManagerMessages.CancelJob(JobID.fromHexString(jobId)), + timeout); Await.ready(response, timeout); return "{}"; @@ -256,8 +251,8 @@ else if ("version".equals(routed.queryParam("get"))) { return writeJsonForVersion(); } else{ - Future response = Patterns.ask(jobmanager, JobManagerMessages.getRequestRunningJobs(), - new Timeout(timeout)); + Future response = jobmanager.ask(JobManagerMessages.getRequestRunningJobs(), + timeout); Object result = Await.result(response, timeout); @@ -454,8 +449,9 @@ private String writeJsonForArchivedJob(ExecutionGraph graph) { } // write accumulators - final Future response = Patterns.ask(jobmanager, - new RequestAccumulatorResultsStringified(graph.getJobID()), new Timeout(timeout)); + final Future response = jobmanager.ask( + new RequestAccumulatorResultsStringified(graph.getJobID()), + timeout); Object result; try { @@ -549,9 +545,9 @@ else if (result instanceof AccumulatorResultsErroneous) { private String writeJsonUpdatesForJob(JobID jobId) { - final Future responseArchivedJobs = Patterns.ask(jobmanager, + final Future responseArchivedJobs = jobmanager.ask( JobManagerMessages.getRequestRunningJobs(), - new Timeout(timeout)); + timeout); Object resultArchivedJobs; try{ @@ -591,8 +587,9 @@ private String writeJsonUpdatesForJob(JobID jobId) { } bld.append("],"); - final Future responseJob = Patterns.ask(jobmanager, new JobManagerMessages.RequestJob(jobId), - new Timeout(timeout)); + final Future responseJob = jobmanager.ask( + new JobManagerMessages.RequestJob(jobId), + timeout); Object resultJob; try{ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java index 82ab63e92edc3..ce57714ca5611 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java @@ -32,12 +32,9 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import akka.actor.ActorRef; - -import akka.pattern.Patterns; -import akka.util.Timeout; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.messages.ArchiveMessages.ArchivedJobs; import org.apache.flink.runtime.messages.ArchiveMessages; @@ -78,12 +75,12 @@ public class JobManagerInfoServlet extends HttpServlet { private static final Logger LOG = LoggerFactory.getLogger(JobManagerInfoServlet.class); /** Underlying JobManager */ - private final ActorRef jobmanager; - private final ActorRef archive; + private final ActorGateway jobmanager; + private final ActorGateway archive; private final FiniteDuration timeout; - public JobManagerInfoServlet(ActorRef jobmanager, ActorRef archive, FiniteDuration timeout) { + public JobManagerInfoServlet(ActorGateway jobmanager, ActorGateway archive, FiniteDuration timeout) { this.jobmanager = jobmanager; this.archive = archive; this.timeout = timeout; @@ -102,8 +99,7 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws Se try { if("archive".equals(req.getParameter("get"))) { - response = Patterns.ask(archive, ArchiveMessages.getRequestArchivedJobs(), - new Timeout(timeout)); + response = archive.ask(ArchiveMessages.getRequestArchivedJobs(), timeout); result = Await.result(response, timeout); @@ -119,8 +115,7 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws Se } } else if("jobcounts".equals(req.getParameter("get"))) { - response = Patterns.ask(archive, ArchiveMessages.getRequestJobCounts(), - new Timeout(timeout)); + response = archive.ask(ArchiveMessages.getRequestJobCounts(), timeout); result = Await.result(response, timeout); @@ -135,8 +130,7 @@ else if("jobcounts".equals(req.getParameter("get"))) { else if("job".equals(req.getParameter("get"))) { String jobId = req.getParameter("job"); - response = Patterns.ask(archive, new RequestJob(JobID.fromHexString(jobId)), - new Timeout(timeout)); + response = archive.ask(new RequestJob(JobID.fromHexString(jobId)), timeout); result = Await.result(response, timeout); @@ -163,8 +157,7 @@ else if("groupvertex".equals(req.getParameter("get"))) { return; } - response = Patterns.ask(archive, new RequestJob(JobID.fromHexString(jobId)), - new Timeout(timeout)); + response = archive.ask(new RequestJob(JobID.fromHexString(jobId)), timeout); result = Await.result(response, timeout); @@ -186,9 +179,9 @@ else if("groupvertex".equals(req.getParameter("get"))) { } else if("taskmanagers".equals(req.getParameter("get"))) { - response = Patterns.ask(jobmanager, + response = jobmanager.ask( JobManagerMessages.getRequestNumberRegisteredTaskManager(), - new Timeout(timeout)); + timeout); result = Await.result(response, timeout); @@ -199,9 +192,9 @@ else if("taskmanagers".equals(req.getParameter("get"))) { } else { final int numberOfTaskManagers = (Integer)result; - final Future responseRegisteredSlots = Patterns.ask(jobmanager, + final Future responseRegisteredSlots = jobmanager.ask( JobManagerMessages.getRequestTotalNumberOfSlots(), - new Timeout(timeout)); + timeout); final Object resultRegisteredSlots = Await.result(responseRegisteredSlots, timeout); @@ -221,8 +214,9 @@ else if("taskmanagers".equals(req.getParameter("get"))) { else if("cancel".equals(req.getParameter("get"))) { String jobId = req.getParameter("job"); - response = Patterns.ask(jobmanager, new CancelJob(JobID.fromHexString(jobId)), - new Timeout(timeout)); + response = jobmanager.ask( + new CancelJob(JobID.fromHexString(jobId)), + timeout); Await.ready(response, timeout); } @@ -233,8 +227,9 @@ else if("updates".equals(req.getParameter("get"))) { writeJsonForVersion(resp.getWriter()); } else{ - response = Patterns.ask(jobmanager, JobManagerMessages.getRequestRunningJobs(), - new Timeout(timeout)); + response = jobmanager.ask( + JobManagerMessages.getRequestRunningJobs(), + timeout); result = Await.result(response, timeout); @@ -471,8 +466,8 @@ private void writeJsonForArchivedJob(PrintWriter wrt, ExecutionGraph graph) { } // write accumulators - final Future response = Patterns.ask(jobmanager, - new RequestAccumulatorResultsStringified(graph.getJobID()), new Timeout(timeout)); + final Future response = jobmanager.ask( + new RequestAccumulatorResultsStringified(graph.getJobID()), timeout); Object result; try { @@ -575,9 +570,9 @@ else if (result instanceof AccumulatorResultsErroneous) { private void writeJsonUpdatesForJob(PrintWriter wrt, JobID jobId) { try { - final Future responseArchivedJobs = Patterns.ask(jobmanager, + final Future responseArchivedJobs = jobmanager.ask( JobManagerMessages.getRequestRunningJobs(), - new Timeout(timeout)); + timeout); Object resultArchivedJobs = null; @@ -615,8 +610,7 @@ private void writeJsonUpdatesForJob(PrintWriter wrt, JobID jobId) { wrt.write("],"); - final Future responseJob = Patterns.ask(jobmanager, new RequestJob(jobId), - new Timeout(timeout)); + final Future responseJob = jobmanager.ask(new RequestJob(jobId), timeout); Object resultJob = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java index c3df2538005ef..1f2bfe06b9049 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java @@ -32,10 +32,8 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import akka.actor.ActorRef; -import akka.pattern.Patterns; -import akka.util.Timeout; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceID; @@ -67,13 +65,13 @@ public class SetupInfoServlet extends HttpServlet { final private Configuration configuration; - final private ActorRef jobmanager; + final private ActorGateway jobmanager; final private FiniteDuration timeout; - public SetupInfoServlet(Configuration conf, ActorRef jm, FiniteDuration timeout) { + public SetupInfoServlet(Configuration conf, ActorGateway jobManager, FiniteDuration timeout) { configuration = conf; - this.jobmanager = jm; + this.jobmanager = jobManager; this.timeout = timeout; } @@ -114,9 +112,9 @@ private void writeGlobalConfiguration(HttpServletResponse resp) throws IOExcepti private void writeTaskmanagers(HttpServletResponse resp) throws IOException { - final Future response = Patterns.ask(jobmanager, + final Future response = jobmanager.ask( JobManagerMessages.getRequestRegisteredTaskManagers(), - new Timeout(timeout)); + timeout); Object obj = null; @@ -183,9 +181,9 @@ private void writeStackTraceOfTaskManager(String instanceIdStr, HttpServletRespo StackTrace message = null; Throwable exception = null; - final Future response = Patterns.ask(jobmanager, + final Future response = jobmanager.ask( new RequestStackTrace(instanceID), - new Timeout(timeout)); + timeout); try { message = (StackTrace) Await.result(response, timeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java index a414cf622652e..4383b65963e1f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java @@ -23,12 +23,12 @@ import java.io.IOException; import java.net.URL; -import akka.actor.ActorRef; - import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.webmonitor.WebMonitor; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.server.Server; @@ -45,7 +45,7 @@ * This class sets up a web-server that contains a web frontend to display information about running jobs. * It instantiates and configures an embedded jetty server. */ -public class WebInfoServer { +public class WebInfoServer implements WebMonitor { /** Web root dir in the jar */ private static final String WEB_ROOT_DIR = "web-docs-infoserver"; @@ -70,7 +70,7 @@ public class WebInfoServer { * @throws IOException * Thrown, if the server setup failed for an I/O related reason. */ - public WebInfoServer(Configuration config, ActorRef jobmanager, ActorRef archive) throws IOException { + public WebInfoServer(Configuration config, ActorGateway jobmanager, ActorGateway archive) throws IOException { if (config == null) { throw new IllegalArgumentException("No Configuration has been passed to the web server"); } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 7bf4447f6bc28..5c0f468c1b1b2 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -1128,17 +1128,27 @@ object JobManager { "TaskManager_Process_Reaper") } - // start the job manager web frontend - if (configuration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) { - LOG.info("Starting NEW JobManger web frontend") - - // start the new web frontend. we need to load this dynamically - // because it is not in the same project/dependencies - startWebRuntimeMonitor(configuration, jobManager, archiver) - } - else if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) != -1) { - LOG.info("Starting JobManger web frontend") - val webServer = new WebInfoServer(configuration, jobManager, archiver) + if(configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) { + val lookupTimeout = AkkaUtils.getLookupTimeout(configuration) + val jobManagerGateway = JobManager.getJobManagerGateway(jobManager, lookupTimeout) + val archiverGateway = new AkkaActorGateway(archiver, jobManagerGateway.leaderSessionID()) + + // start the job manager web frontend + val webServer = if ( + configuration.getBoolean( + ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, + false)) { + + LOG.info("Starting NEW JobManger web frontend") + // start the new web frontend. we need to load this dynamically + // because it is not in the same project/dependencies + startWebRuntimeMonitor(configuration, jobManagerGateway, archiverGateway) + } + else { + LOG.info("Starting JobManger web frontend") + new WebInfoServer(configuration, jobManagerGateway, archiverGateway) + } + webServer.start() } } @@ -1570,46 +1580,37 @@ object JobManager { * this method does not throw any exceptions, but only logs them. * * @param config The configuration for the runtime monitor. - * @param jobManager The JobManager actor. + * @param jobManager The JobManager actor gateway. * @param archiver The execution graph archive actor. */ - def startWebRuntimeMonitor(config: Configuration, - jobManager: ActorRef, - archiver: ActorRef): Unit = { + def startWebRuntimeMonitor( + config: Configuration, + jobManager: ActorGateway, + archiver: ActorGateway) + : WebMonitor = { // try to load and instantiate the class - val monitor: WebMonitor = - try { - val classname = "org.apache.flink.runtime.webmonitor.WebRuntimeMonitor" - val clazz: Class[_ <: WebMonitor] = Class.forName(classname) - .asSubclass(classOf[WebMonitor]) - - val ctor: Constructor[_ <: WebMonitor] = clazz.getConstructor(classOf[Configuration], - classOf[ActorRef], - classOf[ActorRef]) - ctor.newInstance(config, jobManager, archiver) - } - catch { - case e: ClassNotFoundException => - LOG.error("Could not load web runtime monitor. " + - "Probably reason: flink-runtime-web is not in the classpath") - LOG.debug("Caught exception", e) - null - case e: InvocationTargetException => - LOG.error("WebServer could not be created", e.getTargetException()) - null - case t: Throwable => - LOG.error("Failed to instantiate web runtime monitor.", t) - null - } - - if (monitor != null) { - try { - monitor.start() - } - catch { - case e: Exception => - LOG.error("Failed to start web runtime monitor", e) - } + try { + val classname = "org.apache.flink.runtime.webmonitor.WebRuntimeMonitor" + val clazz: Class[_ <: WebMonitor] = Class.forName(classname) + .asSubclass(classOf[WebMonitor]) + + val ctor: Constructor[_ <: WebMonitor] = clazz.getConstructor(classOf[Configuration], + classOf[ActorGateway], + classOf[ActorGateway]) + ctor.newInstance(config, jobManager, archiver) + } + catch { + case e: ClassNotFoundException => + LOG.error("Could not load web runtime monitor. " + + "Probably reason: flink-runtime-web is not in the classpath") + LOG.debug("Caught exception", e) + null + case e: InvocationTargetException => + LOG.error("WebServer could not be created", e.getTargetException()) + null + case t: Throwable => + LOG.error("Failed to instantiate web runtime monitor.", t) + null } } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index 6f810fc4e0fb3..7c5723382dddc 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -30,10 +30,12 @@ import org.apache.flink.runtime.StreamingMode import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.client.{JobExecutionException, JobClient, SerializedJobExecutionResult} -import org.apache.flink.runtime.instance.ActorGateway +import org.apache.flink.runtime.instance.{AkkaActorGateway, ActorGateway} import org.apache.flink.runtime.jobgraph.JobGraph import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtJobManager +import org.apache.flink.runtime.webmonitor.WebMonitor import org.slf4j.LoggerFactory import scala.concurrent.duration.FiniteDuration @@ -74,7 +76,7 @@ abstract class FlinkMiniCluster( val configuration = generateConfiguration(userConfiguration) var jobManagerActorSystem = startJobManagerActorSystem() - var jobManagerActor = startJobManager(jobManagerActorSystem) + var (jobManagerActor, webMonitor) = startJobManager(jobManagerActorSystem) val numTaskManagers = configuration.getInteger( ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1) @@ -99,7 +101,7 @@ abstract class FlinkMiniCluster( def generateConfiguration(userConfiguration: Configuration): Configuration - def startJobManager(system: ActorSystem): ActorRef + def startJobManager(system: ActorSystem): (ActorRef, Option[WebMonitor]) def startTaskManager(index: Int, system: ActorSystem): ActorRef @@ -156,6 +158,10 @@ abstract class FlinkMiniCluster( } def shutdown(): Unit = { + webMonitor foreach { + _.stop() + } + val futures = taskManagerActors map { gracefulStop(_, timeout) } @@ -183,6 +189,44 @@ abstract class FlinkMiniCluster( } } + def startWebServer( + config: Configuration, + jobManager: ActorRef, + archiver: ActorRef) + : Option[WebMonitor] = { + if( + config.getBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, false) && + config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) { + + val lookupTimeout = AkkaUtils.getLookupTimeout(config) + + val jobManagerGateway = JobManager.getJobManagerGateway(jobManager, lookupTimeout) + val archiverGateway = new AkkaActorGateway(archiver, jobManagerGateway.leaderSessionID()) + + // start the job manager web frontend + val webServer = if ( + config.getBoolean( + ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, + false)) { + + LOG.info("Starting NEW JobManger web frontend") + // start the new web frontend. we need to load this dynamically + // because it is not in the same project/dependencies + JobManager.startWebRuntimeMonitor(config, jobManagerGateway, archiverGateway) + } + else { + LOG.info("Starting JobManger web frontend") + new WebInfoServer(config, jobManagerGateway, archiverGateway) + } + + webServer.start() + + Option(webServer) + } else { + None + } + } + def waitForTaskManagersToBeRegistered(): Unit = { implicit val executionContext = ExecutionContext.global diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index c056b63037254..54c457e71a2a5 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -23,12 +23,15 @@ import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import org.apache.flink.api.common.io.FileOutputFormat import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.runtime.StreamingMode +import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.client.JobClient +import org.apache.flink.runtime.instance.AkkaActorGateway import org.apache.flink.runtime.io.network.netty.NettyConfig import org.apache.flink.runtime.jobmanager.JobManager import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.util.EnvironmentInformation +import org.apache.flink.runtime.webmonitor.WebMonitor import org.slf4j.LoggerFactory @@ -42,9 +45,10 @@ import org.slf4j.LoggerFactory * @param singleActorSystem true if all actors (JobManager and TaskManager) shall be run in the same * [[ActorSystem]], otherwise false */ -class LocalFlinkMiniCluster(userConfiguration: Configuration, - singleActorSystem: Boolean, - streamingMode: StreamingMode) +class LocalFlinkMiniCluster( + userConfiguration: Configuration, + singleActorSystem: Boolean, + streamingMode: StreamingMode) extends FlinkMiniCluster(userConfiguration, singleActorSystem, streamingMode) { @@ -74,23 +78,14 @@ class LocalFlinkMiniCluster(userConfiguration: Configuration, config } - override def startJobManager(system: ActorSystem): ActorRef = { + override def startJobManager(system: ActorSystem): (ActorRef, Option[WebMonitor]) = { val config = configuration.clone() val (jobManager, archiver) = JobManager.startJobManagerActors(config, system, streamingMode) - - if (config.getBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, false)) { - if (userConfiguration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) { - // new web frontend - JobManager.startWebRuntimeMonitor(userConfiguration, jobManager, archiver) - } - else { - // old web frontend - val webServer = new WebInfoServer(configuration, jobManager, archiver) - webServer.start() - } - } - jobManager + + val webMonitorOption = startWebServer(config, jobManager, archiver) + + (jobManager, webMonitorOption) } override def startTaskManager(index: Int, system: ActorSystem): ActorRef = { @@ -125,13 +120,15 @@ class LocalFlinkMiniCluster(userConfiguration: Configuration, None } - TaskManager.startTaskManagerComponentsAndActor(config, system, - hostname, // network interface to bind to - Some(taskManagerActorName), // actor name - jobManagerPath, // job manager akka URL - localExecution, // start network stack? - streamingMode, - classOf[TaskManager]) + TaskManager.startTaskManagerComponentsAndActor( + config, + system, + hostname, // network interface to bind to + Some(taskManagerActorName), // actor name + jobManagerPath, // job manager akka URL + localExecution, // start network stack? + streamingMode, + classOf[TaskManager]) } def getJobClientActorSystem: ActorSystem = jobClientActorSystem diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index f974946df7d2f..0ec1040537505 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -171,7 +171,7 @@ class TaskManager( protected var leaderSessionID: Option[UUID] = None - private var currentRegistrationSessionID: UUID = UUID.randomUUID() + private val currentRegistrationSessionID: UUID = UUID.randomUUID() // -------------------------------------------------------------------------- // Actor messages and life cycle diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index ce0ef8d746939..f5a506dc5fe08 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager} import org.apache.flink.runtime.minicluster.FlinkMiniCluster import org.apache.flink.runtime.net.NetUtils import org.apache.flink.runtime.taskmanager.TaskManager +import org.apache.flink.runtime.webmonitor.WebMonitor /** * Testing cluster which starts the [[JobManager]] and [[TaskManager]] actors with testing support @@ -67,7 +68,7 @@ class TestingCluster(userConfiguration: Configuration, cfg } - override def startJobManager(actorSystem: ActorSystem): ActorRef = { + override def startJobManager(actorSystem: ActorSystem): (ActorRef, Option[WebMonitor]) = { val (executionContext, instanceManager, @@ -103,7 +104,7 @@ class TestingCluster(userConfiguration: Configuration, jobManagerProps } - actorSystem.actorOf(dispatcherJobManagerProps, JobManager.JOB_MANAGER_NAME) + (actorSystem.actorOf(dispatcherJobManagerProps, JobManager.JOB_MANAGER_NAME), None) } override def startTaskManager(index: Int, system: ActorSystem) = { @@ -116,12 +117,14 @@ class TestingCluster(userConfiguration: Configuration, None } - TaskManager.startTaskManagerComponentsAndActor(configuration, system, - hostname, - Some(tmActorName), - jobManagerPath, - numTaskManagers == 1, - streamingMode, - classOf[TestingTaskManager]) + TaskManager.startTaskManagerComponentsAndActor( + configuration, + system, + hostname, + Some(tmActorName), + jobManagerPath, + numTaskManagers == 1, + streamingMode, + classOf[TestingTaskManager]) } } diff --git a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala index cdf3960c4ae5b..e83c7a67b481a 100644 --- a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala +++ b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala @@ -22,12 +22,15 @@ import akka.actor.{Props, ActorRef, ActorSystem} import akka.pattern.Patterns._ import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.runtime.StreamingMode +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.instance.AkkaActorGateway import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager} import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingJobManager, TestingMemoryArchivist, TestingTaskManager} +import org.apache.flink.runtime.webmonitor.WebMonitor import scala.concurrent.Await @@ -40,9 +43,10 @@ import scala.concurrent.Await * @param singleActorSystem true, if all actors (JobManager and TaskManager) shall be run in the * same [[ActorSystem]], otherwise false. */ -class ForkableFlinkMiniCluster(userConfiguration: Configuration, - singleActorSystem: Boolean, - streamingMode: StreamingMode) +class ForkableFlinkMiniCluster( + userConfiguration: Configuration, + singleActorSystem: Boolean, + streamingMode: StreamingMode) extends LocalFlinkMiniCluster(userConfiguration, singleActorSystem, streamingMode) { @@ -78,7 +82,7 @@ class ForkableFlinkMiniCluster(userConfiguration: Configuration, super.generateConfiguration(config) } - override def startJobManager(actorSystem: ActorSystem): ActorRef = { + override def startJobManager(actorSystem: ActorSystem): (ActorRef, Option[WebMonitor]) = { val (executionContext, instanceManager, @@ -95,7 +99,7 @@ class ForkableFlinkMiniCluster(userConfiguration: Configuration, archiveCount) with TestingMemoryArchivist) - val archive = actorSystem.actorOf(testArchiveProps, JobManager.ARCHIVE_NAME) + val archiver = actorSystem.actorOf(testArchiveProps, JobManager.ARCHIVE_NAME) val jobManagerProps = Props( new JobManager( @@ -104,7 +108,7 @@ class ForkableFlinkMiniCluster(userConfiguration: Configuration, instanceManager, scheduler, libraryCacheManager, - archive, + archiver, executionRetries, delayBetweenRetries, timeout, @@ -113,21 +117,9 @@ class ForkableFlinkMiniCluster(userConfiguration: Configuration, val jobManager = actorSystem.actorOf(jobManagerProps, JobManager.JOB_MANAGER_NAME) - if (userConfiguration.getBoolean( - ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, false)) - { - if (userConfiguration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) { - // new web frontend - JobManager.startWebRuntimeMonitor(userConfiguration, jobManager, archive) - } - else { - // old web frontend - val webServer = new WebInfoServer(configuration, jobManager, archive) - webServer.start() - } - } + val webMonitorOption = startWebServer(configuration, jobManager, archiver) - jobManager + (jobManager, webMonitorOption) } override def startTaskManager(index: Int, system: ActorSystem): ActorRef = { @@ -163,11 +155,18 @@ class ForkableFlinkMiniCluster(userConfiguration: Configuration, val stopped = gracefulStop(jobManagerActor, TestingUtils.TESTING_DURATION) Await.result(stopped, TestingUtils.TESTING_DURATION) + webMonitor foreach { + _.stop() + } + jobManagerActorSystem.shutdown() jobManagerActorSystem.awaitTermination() jobManagerActorSystem = startJobManagerActorSystem() - jobManagerActor = startJobManager(jobManagerActorSystem) + val (newJobManagerActor, newWebMonitor) = startJobManager(jobManagerActorSystem) + + jobManagerActor = newJobManagerActor + webMonitor = newWebMonitor } def restartTaskManager(index: Int): Unit = { diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala index c497a900f00ef..9e0c976f7bf8c 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala @@ -26,9 +26,11 @@ import org.apache.flink.client.CliFrontend import org.apache.flink.configuration.{GlobalConfiguration, Configuration, ConfigConstants} import org.apache.flink.runtime.StreamingMode import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.instance.AkkaActorGateway import org.apache.flink.runtime.jobmanager.JobManager import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.util.EnvironmentInformation +import org.apache.flink.runtime.webmonitor.WebMonitor import org.apache.flink.yarn.Messages.StartYarnSession import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -68,7 +70,7 @@ object ApplicationMaster { override def run(): Object = { var actorSystem: ActorSystem = null - var webserver: WebInfoServer = null + var webserver: WebMonitor = null try { val conf = new YarnConfiguration() @@ -99,25 +101,44 @@ object ApplicationMaster { val slots = env.get(FlinkYarnClient.ENV_SLOTS).toInt val dynamicPropertiesEncodedString = env.get(FlinkYarnClient.ENV_DYNAMIC_PROPERTIES) - val (config: Configuration, - system: ActorSystem, - jobManager: ActorRef, - archiver: ActorRef) = startJobManager(currDir, ownHostname, - dynamicPropertiesEncodedString, - streamingMode) + val config = createConfiguration(currDir, dynamicPropertiesEncodedString) + + val ( + system: ActorSystem, + jobManager: ActorRef, + archiver: ActorRef) = startJobManager( + config, + ownHostname, + streamingMode) + actorSystem = system val extActor = system.asInstanceOf[ExtendedActorSystem] val jobManagerPort = extActor.provider.getDefaultAddress.port.get - // start the web info server if (config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) != -1) { + // start the web info server + val lookupTimeout = AkkaUtils.getLookupTimeout(config) + val jobManagerGateway = JobManager.getJobManagerGateway(jobManager, lookupTimeout) + val archiverGateway = new AkkaActorGateway( + archiver, + jobManagerGateway.leaderSessionID()) + LOG.info("Starting Job Manger web frontend.") config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logDirs) config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0); // set port to 0. // set JobManager host/port for web interface. config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, ownHostname) config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort) - webserver = new WebInfoServer(config, jobManager, archiver) + + webserver = if( + config.getBoolean( + ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, + false)) { + JobManager.startWebRuntimeMonitor(config, jobManagerGateway, archiverGateway) + } else { + new WebInfoServer(config, jobManagerGateway, archiverGateway) + } + webserver.start() } @@ -160,11 +181,17 @@ object ApplicationMaster { } - def generateConfigurationFile(fileName: String, currDir: String, ownHostname: String, - jobManagerPort: Int, - jobManagerWebPort: Int, logDirs: String, slots: Int, - taskManagerCount: Int, dynamicPropertiesEncodedString: String) - : Unit = { + def generateConfigurationFile( + fileName: String, + currDir: String, + ownHostname: String, + jobManagerPort: Int, + jobManagerWebPort: Int, + logDirs: String, + slots: Int, + taskManagerCount: Int, + dynamicPropertiesEncodedString: String) + : Unit = { LOG.info("Generate configuration file for application master.") val output = new PrintWriter(new BufferedWriter( new FileWriter(fileName)) @@ -208,26 +235,13 @@ object ApplicationMaster { * * @return (Configuration, JobManager ActorSystem, JobManager ActorRef, Archiver ActorRef) */ - def startJobManager(currDir: String, - hostname: String, - dynamicPropertiesEncodedString: String, - streamingMode: StreamingMode): - (Configuration, ActorSystem, ActorRef, ActorRef) = { + def startJobManager( + configuration: Configuration, + hostname: String, + streamingMode: StreamingMode) + : (ActorSystem, ActorRef, ActorRef) = { LOG.info("Starting JobManager for YARN") - LOG.info(s"Loading config from: $currDir.") - - GlobalConfiguration.loadConfiguration(currDir) - val configuration = GlobalConfiguration.getConfiguration() - - configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, currDir) - - // add dynamic properties to JobManager configuration. - val dynamicProperties = CliFrontend.getDynamicProperties(dynamicPropertiesEncodedString) - import scala.collection.JavaConverters._ - for(property <- dynamicProperties.asScala){ - configuration.setString(property.f0, property.f1) - } // set port to 0 to let Akka automatically determine the port. LOG.debug("Starting JobManager actor system") @@ -265,7 +279,25 @@ object ApplicationMaster { LOG.debug("Starting JobManager actor") val jobManager = JobManager.startActor(jobManagerProps, jobManagerSystem) - (configuration, jobManagerSystem, jobManager, archiver) + (jobManagerSystem, jobManager, archiver) + } + + def createConfiguration(curDir: String, dynamicPropertiesEncodedString: String): Configuration = { + LOG.info(s"Loading config from: $curDir.") + + GlobalConfiguration.loadConfiguration(curDir) + val configuration = GlobalConfiguration.getConfiguration() + + configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, curDir) + + // add dynamic properties to JobManager configuration. + val dynamicProperties = CliFrontend.getDynamicProperties(dynamicPropertiesEncodedString) + import scala.collection.JavaConverters._ + for(property <- dynamicProperties.asScala){ + configuration.setString(property.f0, property.f1) + } + + configuration } From 0693c92bdda655e1fbce232038909a7c2a385a22 Mon Sep 17 00:00:00 2001 From: tedyu Date: Tue, 21 Jul 2015 11:13:17 -0700 Subject: [PATCH 060/175] [FLINK-2322] [streaming] Close file streams to release resources early. This closes #928 --- .../flink/api/java/sca/UdfAnalyzerUtils.java | 19 ++++++++++++++++--- .../flink/api/java/utils/ParameterTool.java | 4 +++- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java index df1e421bf2cb4..dbfd29efb5f85 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java @@ -32,6 +32,7 @@ import org.objectweb.asm.tree.analysis.Value; import java.io.IOException; +import java.io.InputStream; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Collections; @@ -114,12 +115,14 @@ public static Object[] findMethodNode(String internalClassName, Method method) { */ @SuppressWarnings("unchecked") public static Object[] findMethodNode(String internalClassName, String name, String desc) { + InputStream stream = null; try { // iterate through hierarchy and search for method node / // class that really implements the method while (internalClassName != null) { - ClassReader cr = new ClassReader(Thread.currentThread().getContextClassLoader() - .getResourceAsStream(internalClassName.replace('.', '/') + ".class")); + stream = Thread.currentThread().getContextClassLoader() + .getResourceAsStream(internalClassName.replace('.', '/') + ".class"); + ClassReader cr = new ClassReader(stream); final ClassNode cn = new ClassNode(); cr.accept(cn, 0); for (MethodNode mn : (List) cn.methods) { @@ -129,9 +132,19 @@ public static Object[] findMethodNode(String internalClassName, String name, Str } internalClassName = cr.getSuperName(); } - } catch (IOException e) { + } + catch (IOException e) { throw new IllegalStateException("Method '" + name + "' could not be found", e); } + finally { + if (stream != null) { + try { + stream.close(); + } catch (IOException e) { + // best effort cleanup + } + } + } throw new IllegalStateException("Method '" + name + "' could not be found"); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java index 317dce40a97db..b60a559f66538 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java @@ -130,7 +130,9 @@ public static ParameterTool fromPropertiesFile(String path) throws IOException { throw new FileNotFoundException("Properties file "+path+" does not exist"); } Properties props = new Properties(); - props.load(new FileInputStream(propertiesFile)); + FileInputStream fis = new FileInputStream(propertiesFile); + props.load(fis); + fis.close(); return fromMap((Map)props); } From b08e30ad9a39ed9885f4daa05a08d13eb22ebb8c Mon Sep 17 00:00:00 2001 From: ffbin <869218239@qq.com> Date: Mon, 3 Aug 2015 15:50:26 +0800 Subject: [PATCH 061/175] [FLINK-2465] [streaming] SocketClientSink closes connection early This closes #972 --- .../flink/streaming/api/functions/sink/SocketClientSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java index da8fd7f01fb8f..adffe5e1877dd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java @@ -88,7 +88,7 @@ public void invoke(IN value) { */ private void closeConnection(){ try { - dataOutputStream.flush(); + dataOutputStream.close(); client.close(); } catch (IOException e) { throw new RuntimeException("Error while closing connection with socket server at " From 24dee42e0a68a38e87a5ec53e683c086a47698b3 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 3 Aug 2015 12:33:47 +0200 Subject: [PATCH 062/175] [hotfix] Remove unused (and broken) TraversableOnceIterable. --- .../flink/util/TraversableOnceIterable.java | 42 ------------------- 1 file changed, 42 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/util/TraversableOnceIterable.java diff --git a/flink-core/src/main/java/org/apache/flink/util/TraversableOnceIterable.java b/flink-core/src/main/java/org/apache/flink/util/TraversableOnceIterable.java deleted file mode 100644 index 73e3cd6e61cc2..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/util/TraversableOnceIterable.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.util; - -import java.util.Iterator; - -public class TraversableOnceIterable implements Iterable { - - private final Iterator iterator; - - public TraversableOnceIterable(Iterator iterator) { - if (iterator == null) { - throw new NullPointerException("The iterator must not be null."); - } - this.iterator = iterator; - } - - @Override - public Iterator iterator() { - if (iterator != null) { - return iterator; - } else { - throw new TraversableOnceException(); - } - } -} From 8eb9cbf88111b0375860a3965e093a736455db79 Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Wed, 29 Jul 2015 18:51:58 +0530 Subject: [PATCH 063/175] [FLINK-2425] [FLINK-2426] [runtime] Add an unmodifiable config and provide access to task manager configuration and hostname inside RuntimeEnvironment --- .../flink/configuration/Configuration.java | 2 +- .../UnmodifiableConfiguration.java | 100 ++++++++++++++++++ .../UnmodifiableConfigurationTest.java | 46 ++++++++ .../flink/runtime/execution/Environment.java | 10 ++ .../taskmanager/RuntimeEnvironment.java | 22 +++- .../flink/runtime/taskmanager/Task.java | 9 +- .../taskmanager/RuntimeConfiguration.scala | 23 ++++ .../runtime/taskmanager/TaskManager.scala | 7 +- .../operators/testutils/MockEnvironment.java | 11 ++ .../taskmanager/TaskAsyncCallTest.java | 10 +- .../flink/runtime/taskmanager/TaskTest.java | 6 +- .../runtime/tasks/StreamMockEnvironment.java | 11 ++ 12 files changed, 246 insertions(+), 11 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java create mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java index c095b5f7e29ec..e9d7621afefcb 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java @@ -472,7 +472,7 @@ public Map toMap() { // -------------------------------------------------------------------------------------------- - private void setValueInternal(String key, T value) { + void setValueInternal(String key, T value) { if (key == null) { throw new NullPointerException("Key must not be null."); } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java new file mode 100644 index 0000000000000..b436a531f3445 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java @@ -0,0 +1,100 @@ +/* + * 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.configuration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Unmodifiable version of the Configuration class + */ +public class UnmodifiableConfiguration extends Configuration { + + /** The log object used for debugging. */ + private static final Logger LOG = LoggerFactory.getLogger(UnmodifiableConfiguration.class); + + public UnmodifiableConfiguration(Configuration config) { + super(); + super.addAll(config); + } + + // -------------------------------------------------------------------------------------------- + // All setter methods must fail. + // -------------------------------------------------------------------------------------------- + + @Override + public final void setClass(String key, Class klazz) { + error(); + } + + @Override + public final void setString(String key, String value) { + error(); + } + + @Override + public final void setInteger(String key, int value) { + error(); + } + + @Override + public final void setLong(String key, long value) { + error(); + } + + @Override + public final void setBoolean(String key, boolean value) { + error(); + } + + @Override + public final void setFloat(String key, float value) { + error(); + } + + @Override + public final void setDouble(String key, double value) { + error(); + } + + @Override + public final void setBytes(String key, byte[] bytes) { + error(); + } + + @Override + public final void addAll(Configuration other) { + error(); + } + + @Override + public final void addAll(Configuration other, String prefix) { + error(); + } + + @Override + void setValueInternal(String key, T value){ + error(); + } + + private final void error(){ + throw new UnsupportedOperationException("The unmodifiable configuration object doesn't allow set methods."); + } + +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java new file mode 100644 index 0000000000000..302b72b7c6d78 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java @@ -0,0 +1,46 @@ +/* + * 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.configuration; + + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +import java.lang.reflect.Method; + +/** + * This class verifies that the Unmodifiable Configuration class overrides all setter methods in + * Configuration. + */ +public class UnmodifiableConfigurationTest { + + private static Configuration pc = new Configuration(); + private static UnmodifiableConfiguration unConf = new UnmodifiableConfiguration(pc); + private static Class clazz = unConf.getClass(); + + @Test + public void testOverride() throws Exception{ + for(Method m : clazz.getMethods()){ + if(m.getName().indexOf("set") == 0 || m.getName().indexOf("add") == 0 ) { + assertEquals(clazz, m.getDeclaringClass()); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index c5618695cfbd6..af295600e82c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -71,6 +71,16 @@ public interface Environment { */ Configuration getTaskConfiguration(); + /** + * @return The task manager configuration + */ + Configuration getTaskManagerConfiguration(); + + /** + * @return Hostname of the task manager + */ + String getHostname(); + /** * Returns the job-wide configuration object that was attached to the JobGraph. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 5e276bfd43efa..c0dfee62c148e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -75,6 +75,10 @@ public class RuntimeEnvironment implements Environment { private final AccumulatorRegistry accumulatorRegistry; + private Configuration taskManagerConfiguration; + + private String hostname; + // ------------------------------------------------------------------------ public RuntimeEnvironment( @@ -96,7 +100,8 @@ public RuntimeEnvironment( Map> distCacheEntries, ResultPartitionWriter[] writers, InputGate[] inputGates, - ActorGateway jobManager) { + ActorGateway jobManager, + RuntimeConfiguration taskManagerConfig) { checkArgument(parallelism > 0 && subtaskIndex >= 0 && subtaskIndex < parallelism); @@ -119,9 +124,10 @@ public RuntimeEnvironment( this.writers = checkNotNull(writers); this.inputGates = checkNotNull(inputGates); this.jobManager = checkNotNull(jobManager); + this.taskManagerConfiguration = checkNotNull(taskManagerConfig).configuration(); + this.hostname = taskManagerConfig.hostname(); } - // ------------------------------------------------------------------------ @Override @@ -168,7 +174,17 @@ public Configuration getJobConfiguration() { public Configuration getTaskConfiguration() { return taskConfiguration; } - + + @Override + public Configuration getTaskManagerConfiguration(){ + return taskManagerConfiguration; + } + + @Override + public String getHostname(){ + return hostname; + } + @Override public ClassLoader getUserClassLoader() { return userCodeClassLoader; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index c4f62fbcb4c21..878a69abe67fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -214,6 +214,9 @@ public class Task implements Runnable { * initialization, to be memory friendly */ private volatile SerializedValue> operatorState; + /** Access to task manager configuration and host names*/ + private RuntimeConfiguration taskManagerConfig; + /** *

    IMPORTANT: This constructor may not start any work that would need to * be undone in the case of a failing task deployment.

    @@ -227,7 +230,8 @@ public Task(TaskDeploymentDescriptor tdd, ActorGateway jobManagerActor, FiniteDuration actorAskTimeout, LibraryCacheManager libraryCache, - FileCache fileCache) + FileCache fileCache, + RuntimeConfiguration taskManagerConfig) { checkArgument(tdd.getNumberOfSubtasks() > 0); checkArgument(tdd.getIndexInSubtaskGroup() >= 0); @@ -258,6 +262,7 @@ public Task(TaskDeploymentDescriptor tdd, this.libraryCache = checkNotNull(libraryCache); this.fileCache = checkNotNull(fileCache); this.network = checkNotNull(networkEnvironment); + this.taskManagerConfig = checkNotNull(taskManagerConfig); this.executionListenerActors = new CopyOnWriteArrayList(); @@ -510,7 +515,7 @@ else if (current == ExecutionState.CANCELING) { userCodeClassLoader, memoryManager, ioManager, broadcastVariableManager, accumulatorRegistry, splitProvider, distributedCacheEntries, - writers, inputGates, jobManager); + writers, inputGates, jobManager, taskManagerConfig); // let the task code create its readers and writers invokable.setEnvironment(env); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala new file mode 100644 index 0000000000000..ef0e705273feb --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala @@ -0,0 +1,23 @@ +/* + * 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.runtime.taskmanager + +import org.apache.flink.configuration.UnmodifiableConfiguration + +case class RuntimeConfiguration(hostname: String, configuration: UnmodifiableConfiguration) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 0ec1040537505..3ab271a84c690 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -36,7 +36,7 @@ import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger -import org.apache.flink.configuration.{Configuration, ConfigConstants, GlobalConfiguration, IllegalConfigurationException} +import org.apache.flink.configuration._ import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage} @@ -892,7 +892,10 @@ class TaskManager( jobManagerGateway, config.timeout, libCache, - fileCache) + fileCache, + new RuntimeConfiguration( + self.path.toSerializationFormat, + new UnmodifiableConfiguration(config.configuration))) log.info(s"Received task ${task.getTaskNameWithSubtasks}") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index b9cb4166a3149..b71b01e49aa8c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; @@ -191,6 +192,16 @@ public Configuration getJobConfiguration() { return this.jobConfiguration; } + @Override + public Configuration getTaskManagerConfiguration(){ + return new UnmodifiableConfiguration(new Configuration()); + } + + @Override + public String getHostname(){ + return "localhost"; + } + @Override public int getNumberOfSubtasks() { return 1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 10a33c3a8f423..08f0094b449ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -29,6 +30,7 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; @@ -148,16 +150,20 @@ private static Task createTask() { Collections.emptyList(), 0); + ActorGateway taskManagerGateway = DummyActorGateway.INSTANCE; return new Task(tdd, mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, mock(BroadcastVariableManager.class), - DummyActorGateway.INSTANCE, + taskManagerGateway, DummyActorGateway.INSTANCE, new FiniteDuration(60, TimeUnit.SECONDS), libCache, - mock(FileCache.class)); + mock(FileCache.class), + new RuntimeConfiguration( + taskManagerGateway.path(), + new UnmodifiableConfiguration(new Configuration()))); } public static class CheckpointsInOrderInvokable extends AbstractInvokable diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index c6a8cb813b805..6d9df6d99929c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -20,6 +20,7 @@ import com.google.common.collect.Maps; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -725,7 +726,10 @@ private Task createTask(Class invokable, jobManagerGateway, new FiniteDuration(60, TimeUnit.SECONDS), libCache, - mock(FileCache.class)); + mock(FileCache.class), + new RuntimeConfiguration( + taskManagerGateway.path(), + new UnmodifiableConfiguration(new Configuration()))); } private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index bbc64f1e9ffde..44013ef7e8d4e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; @@ -290,5 +291,15 @@ public void acknowledgeCheckpoint(long checkpointId) { @Override public void acknowledgeCheckpoint(long checkpointId, StateHandle state) { } + + @Override + public Configuration getTaskManagerConfiguration(){ + return new UnmodifiableConfiguration(new Configuration()); + } + + @Override + public String getHostname(){ + return "localhost"; + } } From 5bf2197b148f2b9857d9fcdb5859f37ee8997100 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 3 Aug 2015 13:52:12 +0200 Subject: [PATCH 064/175] [FLINK-2426] [core] Cleanup/improve unmodifiable configuration. --- .../flink/configuration/Configuration.java | 27 +++++-- .../UnmodifiableConfiguration.java | 70 ++++--------------- .../configuration/ConfigurationTest.java | 21 +++++- .../UnmodifiableConfigurationTest.java | 67 +++++++++++++++--- 4 files changed, 113 insertions(+), 72 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java index e9d7621afefcb..da42958d40603 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java @@ -34,10 +34,10 @@ import org.slf4j.LoggerFactory; /** - * Lightweight configuration object which can store key/value pairs. + * Lightweight configuration object which stores key/value pairs. */ -@SuppressWarnings("EqualsBetweenInconvertibleTypes") -public class Configuration extends ExecutionConfig.GlobalJobParameters implements IOReadableWritable, java.io.Serializable, Cloneable { +public class Configuration extends ExecutionConfig.GlobalJobParameters + implements IOReadableWritable, java.io.Serializable, Cloneable { private static final long serialVersionUID = 1L; @@ -54,11 +54,25 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters implement /** Stores the concrete key/value pairs of this configuration object. */ - private final Map confData = new HashMap(); + private final HashMap confData; // -------------------------------------------------------------------------------------------- - - public Configuration() {} + + /** + * Creates a new empty configuration. + */ + public Configuration() { + this.confData = new HashMap(); + } + + /** + * Creates a new configuration with the copy of the given configuration. + * + * @param other The configuration to copy the entries from. + */ + public Configuration(Configuration other) { + this.confData = new HashMap(other.confData); + } // -------------------------------------------------------------------------------------------- @@ -362,6 +376,7 @@ public void setDouble(String key, double value) { * The default value which is returned in case there is no value associated with the given key. * @return the (default) value associated with the given key. */ + @SuppressWarnings("EqualsBetweenInconvertibleTypes") public byte[] getBytes(String key, byte[] defaultValue) { Object o = getRawValue(key); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java index b436a531f3445..5d92cf0f62a0f 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java @@ -18,66 +18,27 @@ package org.apache.flink.configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** - * Unmodifiable version of the Configuration class + * Unmodifiable version of the Configuration class. */ public class UnmodifiableConfiguration extends Configuration { - - /** The log object used for debugging. */ - private static final Logger LOG = LoggerFactory.getLogger(UnmodifiableConfiguration.class); - + + private static final long serialVersionUID = -8151292629158972280L; + + /** + * Creates a new UnmodifiableConfiguration, which holds a copy of the given configuration + * that cannot be altered. + * + * @param config The configuration with the original contents. + */ public UnmodifiableConfiguration(Configuration config) { - super(); - super.addAll(config); + super(config); } // -------------------------------------------------------------------------------------------- - // All setter methods must fail. + // All mutating methods must fail // -------------------------------------------------------------------------------------------- - @Override - public final void setClass(String key, Class klazz) { - error(); - } - - @Override - public final void setString(String key, String value) { - error(); - } - - @Override - public final void setInteger(String key, int value) { - error(); - } - - @Override - public final void setLong(String key, long value) { - error(); - } - - @Override - public final void setBoolean(String key, boolean value) { - error(); - } - - @Override - public final void setFloat(String key, float value) { - error(); - } - - @Override - public final void setDouble(String key, double value) { - error(); - } - - @Override - public final void setBytes(String key, byte[] bytes) { - error(); - } - @Override public final void addAll(Configuration other) { error(); @@ -89,12 +50,11 @@ public final void addAll(Configuration other, String prefix) { } @Override - void setValueInternal(String key, T value){ + final void setValueInternal(String key, T value){ error(); } - private final void error(){ - throw new UnsupportedOperationException("The unmodifiable configuration object doesn't allow set methods."); + private void error(){ + throw new UnsupportedOperationException("The configuration is unmodifiable; its contents cannot be changed."); } - } diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java index e1318927d0262..33dde3d72c4c1 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java @@ -23,8 +23,8 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CommonTestUtils; + import org.junit.Test; /** @@ -175,4 +175,23 @@ public void testConversions() { fail(e.getMessage()); } } + + @Test + public void testCopyConstructor() { + try { + final String key = "theKey"; + + Configuration cfg1 = new Configuration(); + cfg1.setString(key, "value"); + + Configuration cfg2 = new Configuration(cfg1); + cfg2.setString(key, "another value"); + + assertEquals("value", cfg1.getString(key, "")); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } } diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java index 302b72b7c6d78..3ea52b89ec4ea 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java @@ -18,29 +18,76 @@ package org.apache.flink.configuration; - import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import org.junit.Test; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; /** * This class verifies that the Unmodifiable Configuration class overrides all setter methods in * Configuration. */ public class UnmodifiableConfigurationTest { - - private static Configuration pc = new Configuration(); - private static UnmodifiableConfiguration unConf = new UnmodifiableConfiguration(pc); - private static Class clazz = unConf.getClass(); - + @Test - public void testOverride() throws Exception{ - for(Method m : clazz.getMethods()){ - if(m.getName().indexOf("set") == 0 || m.getName().indexOf("add") == 0 ) { - assertEquals(clazz, m.getDeclaringClass()); + public void testOverrideAddMethods() { + try { + Class clazz = UnmodifiableConfiguration.class; + for (Method m : clazz.getMethods()) { + if (m.getName().startsWith("add")) { + assertEquals(clazz, m.getDeclaringClass()); + } } } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testExceptionOnSet() { + try { + Map, Object> parameters = new HashMap, Object>(); + parameters.put(byte[].class, new byte[0]); + parameters.put(Class.class, Object.class); + parameters.put(int.class, 0); + parameters.put(long.class, 0L); + parameters.put(float.class, 0.0f); + parameters.put(double.class, 0.0); + parameters.put(String.class, ""); + parameters.put(boolean.class, false); + + Class clazz = UnmodifiableConfiguration.class; + UnmodifiableConfiguration config = new UnmodifiableConfiguration(new Configuration()); + + for (Method m : clazz.getMethods()) { + if (m.getName().startsWith("set")) { + + Class parameterClass = m.getParameterTypes()[1]; + Object parameter = parameters.get(parameterClass); + assertNotNull("method " + m + " not covered by test", parameter); + + try { + m.invoke(config, "key", parameter); + fail("should fail with an exception"); + } + catch (InvocationTargetException e) { + assertTrue(e.getTargetException() instanceof UnsupportedOperationException); + } + } + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } } From c3ef61de934a9c447ec442449c527ce719ee46c6 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 3 Aug 2015 17:41:57 +0200 Subject: [PATCH 065/175] [FLINK-2425] [runtime] Cleanup code for forwarding config and hostname into TaskManager's RuntimeEnvironment --- .../taskmanager/RuntimeEnvironment.java | 12 ++-- .../flink/runtime/taskmanager/Task.java | 8 +-- .../taskmanager/TaskManagerRuntimeInfo.java | 61 +++++++++++++++++++ .../taskmanager/RuntimeConfiguration.scala | 23 ------- .../runtime/taskmanager/TaskManager.scala | 8 ++- .../taskmanager/TaskAsyncCallTest.java | 6 +- .../flink/runtime/taskmanager/TaskTest.java | 8 +-- 7 files changed, 80 insertions(+), 46 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java delete mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index c0dfee62c148e..cd6dbd633f76b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -75,9 +75,9 @@ public class RuntimeEnvironment implements Environment { private final AccumulatorRegistry accumulatorRegistry; - private Configuration taskManagerConfiguration; + private final Configuration taskManagerConfiguration; - private String hostname; + private final String hostname; // ------------------------------------------------------------------------ @@ -95,13 +95,13 @@ public RuntimeEnvironment( MemoryManager memManager, IOManager ioManager, BroadcastVariableManager bcVarManager, - AccumulatorRegistry accumulatorRegistry, + AccumulatorRegistry accumulatorRegistry, InputSplitProvider splitProvider, Map> distCacheEntries, ResultPartitionWriter[] writers, InputGate[] inputGates, ActorGateway jobManager, - RuntimeConfiguration taskManagerConfig) { + TaskManagerRuntimeInfo taskManagerInfo) { checkArgument(parallelism > 0 && subtaskIndex >= 0 && subtaskIndex < parallelism); @@ -124,8 +124,8 @@ public RuntimeEnvironment( this.writers = checkNotNull(writers); this.inputGates = checkNotNull(inputGates); this.jobManager = checkNotNull(jobManager); - this.taskManagerConfiguration = checkNotNull(taskManagerConfig).configuration(); - this.hostname = taskManagerConfig.hostname(); + this.taskManagerConfiguration = checkNotNull(taskManagerInfo).getConfiguration(); + this.hostname = taskManagerInfo.getHostname(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 878a69abe67fd..36de90a5f8b9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -145,6 +145,9 @@ public class Task implements Runnable { /** The name of the class that holds the invokable code */ private final String nameOfInvokableClass; + /** Access to task manager configuration and host names*/ + private final TaskManagerRuntimeInfo taskManagerConfig; + /** The memory manager to be used by this task */ private final MemoryManager memoryManager; @@ -214,9 +217,6 @@ public class Task implements Runnable { * initialization, to be memory friendly */ private volatile SerializedValue> operatorState; - /** Access to task manager configuration and host names*/ - private RuntimeConfiguration taskManagerConfig; - /** *

    IMPORTANT: This constructor may not start any work that would need to * be undone in the case of a failing task deployment.

    @@ -231,7 +231,7 @@ public Task(TaskDeploymentDescriptor tdd, FiniteDuration actorAskTimeout, LibraryCacheManager libraryCache, FileCache fileCache, - RuntimeConfiguration taskManagerConfig) + TaskManagerRuntimeInfo taskManagerConfig) { checkArgument(tdd.getNumberOfSubtasks() > 0); checkArgument(tdd.getIndexInSubtaskGroup() >= 0); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java new file mode 100644 index 0000000000000..8d06f1006c63c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java @@ -0,0 +1,61 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.configuration.Configuration; + +/** + * Encapsulation of TaskManager runtime information, like hostname and configuration. + */ +public class TaskManagerRuntimeInfo implements java.io.Serializable { + + private static final long serialVersionUID = 5598219619760274072L; + + /** host name of the interface that the TaskManager uses to communicate */ + private final String hostname; + + /** configuration that the TaskManager was started with */ + private final Configuration configuration; + + /** + * Creates a runtime info. + * @param hostname The host name of the interface that the TaskManager uses to communicate. + * @param configuration The configuration that the TaskManager was started with. + */ + public TaskManagerRuntimeInfo(String hostname, Configuration configuration) { + this.hostname = hostname; + this.configuration = configuration; + } + + /** + * Gets host name of the interface that the TaskManager uses to communicate. + * @return The host name of the interface that the TaskManager uses to communicate. + */ + public String getHostname() { + return hostname; + } + + /** + * Gets the configuration that the TaskManager was started with. + * @return The configuration that the TaskManager was started with. + */ + public Configuration getConfiguration() { + return configuration; + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala deleted file mode 100644 index ef0e705273feb..0000000000000 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/RuntimeConfiguration.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.runtime.taskmanager - -import org.apache.flink.configuration.UnmodifiableConfiguration - -case class RuntimeConfiguration(hostname: String, configuration: UnmodifiableConfiguration) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 3ab271a84c690..cc8b8ba320f34 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -173,6 +173,10 @@ class TaskManager( private val currentRegistrationSessionID: UUID = UUID.randomUUID() + private val runtimeInfo = new TaskManagerRuntimeInfo( + connectionInfo.getHostname(), + new UnmodifiableConfiguration(config.configuration)) + // -------------------------------------------------------------------------- // Actor messages and life cycle // -------------------------------------------------------------------------- @@ -893,9 +897,7 @@ class TaskManager( config.timeout, libCache, fileCache, - new RuntimeConfiguration( - self.path.toSerializationFormat, - new UnmodifiableConfiguration(config.configuration))) + runtimeInfo) log.info(s"Received task ${task.getTaskNameWithSubtasks}") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 08f0094b449ef..a7d8d8d689ca4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -129,7 +128,6 @@ public void testMixedAsyncCallsInOrder() { } private static Task createTask() { - LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(ClassLoader.getSystemClassLoader()); @@ -161,9 +159,7 @@ private static Task createTask() { new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new RuntimeConfiguration( - taskManagerGateway.path(), - new UnmodifiableConfiguration(new Configuration()))); + new TaskManagerRuntimeInfo("localhost", new Configuration())); } public static class CheckpointsInOrderInvokable extends AbstractInvokable diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 6d9df6d99929c..0cba533eecc00 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -19,9 +19,8 @@ package org.apache.flink.runtime.taskmanager; import com.google.common.collect.Maps; + import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.UnmodifiableConfiguration; -import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -49,6 +48,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; + import scala.concurrent.duration.FiniteDuration; import java.lang.reflect.Field; @@ -727,9 +727,7 @@ private Task createTask(Class invokable, new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new RuntimeConfiguration( - taskManagerGateway.path(), - new UnmodifiableConfiguration(new Configuration()))); + new TaskManagerRuntimeInfo("localhost", new Configuration())); } private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { From 2cff89ed93bbacf8d054399514c1ca4fb8b24730 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 3 Aug 2015 16:28:34 +0200 Subject: [PATCH 066/175] [FLINK-2473] [core] Add a timeout to akka actorsystem shutdown. This works around a bug in akka where the "awaitTermination()" call freezes indefinitely. --- .../main/java/org/apache/flink/client/program/Client.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index d2022bbf78649..78c82f6dc835c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -26,6 +26,7 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; @@ -441,7 +442,9 @@ public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramIn } finally { actorSystem.shutdown(); - actorSystem.awaitTermination(); + + // wait at most for 30 seconds, to work around an occasional akka problem + actorSystem.awaitTermination(new FiniteDuration(30, TimeUnit.SECONDS)); } } From d0cd1c7d428a2bde6b955599b247b209240f2a34 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 3 Aug 2015 19:23:57 +0200 Subject: [PATCH 067/175] [FLINK-2464] [tests] Change log level of BufferSpillerTest to "info" to let statements occur in CI logs. --- .../apache/flink/streaming/runtime/io/BufferSpillerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java index ca6df16aa90f1..fbc19ec8515f3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java @@ -183,7 +183,7 @@ public void testSpillAndRollOverSimple() { @Test public void testSpillWhileReading() { - LOG.debug("Starting SpillWhileReading test"); + LOG.info("Starting SpillWhileReading test"); try { final int sequences = 10; @@ -359,7 +359,7 @@ public void run() { int numBuffersAndEvents = nextSequence.numBuffersAndEvents; int numChannels = nextSequence.numChannels; - LOG.debug("Reading sequence {}", consumedSequences); + LOG.info("Reading sequence {}", consumedSequences); // consume sequence seq.open(); From 949ff8dae6deb58229367bdcf199336c92399c27 Mon Sep 17 00:00:00 2001 From: ffbin <869218239@qq.com> Date: Sat, 1 Aug 2015 16:32:48 +0800 Subject: [PATCH 068/175] [FLINK-2456] [hbase] Add a repository ID for hadoop2 in flink-hbase module This closes #969 --- flink-staging/flink-hbase/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/flink-staging/flink-hbase/pom.xml b/flink-staging/flink-hbase/pom.xml index 9638c58e91c67..5dc80d92e3c3b 100644 --- a/flink-staging/flink-hbase/pom.xml +++ b/flink-staging/flink-hbase/pom.xml @@ -171,6 +171,18 @@ under the License. hadoop-2 + + + hadoop-2-repo2 + https://repo.maven.apache.org/maven2 + + true + + + false + + + From 2859f59f84aae46791b51493f8777679de496a86 Mon Sep 17 00:00:00 2001 From: mjsax Date: Mon, 3 Aug 2015 22:04:35 +0200 Subject: [PATCH 069/175] [FLINK-2095][docs] Added missing screenshots to webclient documentation This closes #976 --- docs/apis/web_client.md | 55 +++++++++++++++++++++++++++-------------- 1 file changed, 37 insertions(+), 18 deletions(-) diff --git a/docs/apis/web_client.md b/docs/apis/web_client.md index 8e3c84a724fea..4749d90497171 100644 --- a/docs/apis/web_client.md +++ b/docs/apis/web_client.md @@ -46,16 +46,27 @@ The web interface provides two views: ### Job View -The interface starts serving the job view. - -You can **upload** a Flink program as a jar file. To **execute** an uploaded program: - -* select it from the job list on the left, -* enter (optional) execution options in the *"Flink Options"* field (bottom left), -* enter (optional) program arguments in the *"Program Arguments"* field (bottom left), and -* click on the *"Run Job"* button (bottom right). - -If the *“Show optimizer plan”* option is enabled (default), the *plan view* is display next, otherwise the job is directly submitted to the JobManager for execution. +
    +
    + The interface starts serving the job view.
    +
    + You can upload a Flink program as a jar file.
    +
    + To execute an uploaded program: +
      +
    • select it from the job list on the left,
    • +
    • enter (optional) execution options in the "Flink Options" field (bottom left),
    • +
    • enter (optional) program arguments in the "Program Arguments" field (bottom left), and
    • +
    • click on the "Run Job" button (bottom right).
    • +
    + If the “Show optimizer plan” option is enabled (default), the plan view is display next, otherwise the job is directly submitted to the JobManager for execution. +
    +
    + +
    +
    + +
    The web interface can also handle multiple Flink jobs within a single jar file. To use this feature, package all required class files of all jobs into a single jar and specify the entry classes for each job as comma-separated-values in *program-class* argument within the jar's manifest file. The job view displays each entry class and you can pick any of them to preview the plan and/or submit the job to the JobManager. In case the jar's manifest file does not specify any entry class, you can specify it in the options field as: @@ -73,11 +84,19 @@ Furthermore, for each entry class implementing ```ProgramDescription``` interfac ### Plan View -The plan view shows the optimized execution plan of the submitted program in the upper half of the page. The bottom part of the page displays detailed information about the currently selected plan operator including: - -* the chosen shipping strategies (local forward, hash partition, range partition, broadcast, ...), -* the chosen local strategy (sort, hash join, merge join, ...), -* inferred data properties (partitioning, grouping, sorting), and -* used optimizer estimates (data size, I/O and network costs, ...). - -To submit the job for execution, click again on the *"Run Job"* button in the bottom right. +
    +
    + The plan view shows the optimized execution plan of the submitted program in the upper half of the page. The bottom part of the page displays detailed information about the currently selected plan operator including: +
      +
    • the chosen shipping strategies (local forward, hash partition, range partition, broadcast, ...),
    • +
    • the chosen local strategy (sort, hash join, merge join, ...),
    • +
    • inferred data properties (partitioning, grouping, sorting), and
    • +
    • used optimizer estimates (data size, I/O and network costs, ...).
    • +
    + + To submit the job for execution, click again on the "Run Job" button in the bottom right. +
    +
    + +
    +
    From bb5e61d242ad272a16b487a0cf461774b73e648b Mon Sep 17 00:00:00 2001 From: buzdin Date: Tue, 28 Jul 2015 16:45:25 +0100 Subject: [PATCH 070/175] [FLINK-2422] [web client] Added explicit link in case browser is not redirecting properly This closes #946 --- flink-clients/src/main/resources/web-docs/index.html | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-clients/src/main/resources/web-docs/index.html b/flink-clients/src/main/resources/web-docs/index.html index bfba666486ea6..e4275d8b2800f 100644 --- a/flink-clients/src/main/resources/web-docs/index.html +++ b/flink-clients/src/main/resources/web-docs/index.html @@ -22,5 +22,7 @@ +

    You are being redirected...

    +

    Click here if nothing happens.

    From 77b7471580ce9cada86e32c2b6919086ed2eb730 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 4 Aug 2015 09:41:33 +0200 Subject: [PATCH 071/175] [docs] Fixes broken link in FlinkML docs --- docs/libs/ml/index.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/libs/ml/index.md b/docs/libs/ml/index.md index 63cdf43c70660..88866828ae4fd 100644 --- a/docs/libs/ml/index.md +++ b/docs/libs/ml/index.md @@ -69,7 +69,8 @@ Next, you have to add the FlinkML dependency to the `pom.xml` of your project. {% endhighlight %} -Note that FlinkML is currently not part of the binary distribution. See linking with it for cluster execution [here](../apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution). +Note that FlinkML is currently not part of the binary distribution. +See linking with it for cluster execution [here]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution). Now you can start solving your analysis task. The following code snippet shows how easy it is to train a multiple linear regression model. From d570d078ae8e566586bf82bd7d3adc7775e30e2f Mon Sep 17 00:00:00 2001 From: ffbin <869218239@qq.com> Date: Tue, 4 Aug 2015 11:30:58 +0800 Subject: [PATCH 072/175] Remove unwanted check null of input1 in ConnectedDataStream Closes #978 --- .../flink/streaming/api/datastream/ConnectedDataStream.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java index b28f80b596d43..079f1363d1d6f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java @@ -60,9 +60,8 @@ public class ConnectedDataStream { protected ConnectedDataStream(DataStream input1, DataStream input2) { this.jobGraphBuilder = input1.streamGraph; this.environment = input1.environment; - if (input1 != null) { - this.dataStream1 = input1.copy(); - } + this.dataStream1 = input1.copy(); + if (input2 != null) { this.dataStream2 = input2.copy(); } From 06b37bf550315bd1d5be7dc3ed6638fd21768e1a Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 4 Aug 2015 12:45:22 +0200 Subject: [PATCH 073/175] [FLINK-2005] Remove Record API from jdbc module This closes #982 --- flink-staging/flink-jdbc/pom.xml | 12 - .../java/record/io/jdbc/JDBCInputFormat.java | 389 ------------------ .../java/record/io/jdbc/JDBCOutputFormat.java | 359 ---------------- .../record/io/jdbc/example/JDBCExample.java | 136 ------ .../java/record/io/jdbc/DevNullLogStream.java | 30 -- .../record/io/jdbc/JDBCInputFormatTest.java | 214 ---------- .../record/io/jdbc/JDBCOutputFormatTest.java | 225 ---------- 7 files changed, 1365 deletions(-) delete mode 100644 flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormat.java delete mode 100644 flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormat.java delete mode 100644 flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/example/JDBCExample.java delete mode 100644 flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/DevNullLogStream.java delete mode 100644 flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormatTest.java delete mode 100644 flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormatTest.java diff --git a/flink-staging/flink-jdbc/pom.xml b/flink-staging/flink-jdbc/pom.xml index 7b499a7f763a0..a3976c10aaf28 100644 --- a/flink-staging/flink-jdbc/pom.xml +++ b/flink-staging/flink-jdbc/pom.xml @@ -41,18 +41,6 @@ under the License. flink-java ${project.version} - - - org.apache.flink - flink-core - ${project.version} - - - - org.apache.flink - flink-clients - ${project.version} - org.apache.derby diff --git a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormat.java b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormat.java deleted file mode 100644 index 3cd295bfb2b8e..0000000000000 --- a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormat.java +++ /dev/null @@ -1,389 +0,0 @@ -/* - * 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.api.java.record.io.jdbc; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.SQLException; -import java.sql.Statement; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.api.common.io.NonParallelInput; -import org.apache.flink.api.java.record.io.GenericInputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.BooleanValue; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.NullValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.ShortValue; -import org.apache.flink.types.StringValue; - -/** - * InputFormat to read data from a database and generate PactReords. - * The InputFormat has to be configured with the query, and either all - * connection parameters or a complete database URL.{@link Configuration} The position of a value inside a Record is - * determined by the table - * returned. - * - * @see Configuration - * @see Record - * @see DriverManager - */ -public class JDBCInputFormat extends GenericInputFormat implements NonParallelInput { - - private static final long serialVersionUID = 1L; - - @SuppressWarnings("unused") - private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class); - - - public final String DRIVER_KEY = "driver"; - public final String USERNAME_KEY = "username"; - public final String PASSWORD_KEY = "password"; - public final String URL_KEY = "url"; - public final String QUERY_KEY = "query"; - - - private String username; - private String password; - private String driverName; - private String dbURL; - private String query; - - - private transient Connection dbConn; - private transient Statement statement; - private transient ResultSet resultSet; - - - /** - * Creates a non-configured JDBCInputFormat. This format has to be - * configured using configure(configuration). - */ - public JDBCInputFormat() {} - - /** - * Creates a JDBCInputFormat and configures it. - * - * @param driverName - * JDBC-Drivename - * @param dbURL - * Formatted URL containing all connection parameters. - * @param username - * @param password - * @param query - * Query to execute. - */ - public JDBCInputFormat(String driverName, String dbURL, String username, String password, String query) { - this.driverName = driverName; - this.query = query; - this.dbURL = dbURL; - this.username = username; - this.password = password; - } - - /** - * Creates a JDBCInputFormat and configures it. - * - * @param driverName - * JDBC-Drivername - * @param dbURL - * Formatted URL containing all connection parameters. - * @param query - * Query to execute. - */ - public JDBCInputFormat(String driverName, String dbURL, String query) { - this(driverName, dbURL, "", "", query); - } - - /** - * Creates a JDBCInputFormat and configures it. - * - * @param parameters - * Configuration with all connection parameters. - * @param query - * Query to execute. - */ - public JDBCInputFormat(Configuration parameters, String query) { - this.driverName = parameters.getString(DRIVER_KEY, ""); - this.username = parameters.getString(USERNAME_KEY, ""); - this.password = parameters.getString(PASSWORD_KEY, ""); - this.dbURL = parameters.getString(URL_KEY, ""); - this.query = query; - } - - - /** - * Configures this JDBCInputFormat. This includes setting the connection - * parameters (if necessary), establishing the connection and executing the - * query. - * - * @param parameters - * Configuration containing all or no parameters. - */ - @Override - public void configure(Configuration parameters) { - boolean needConfigure = isFieldNullOrEmpty(this.query) || isFieldNullOrEmpty(this.dbURL); - if (needConfigure) { - this.driverName = parameters.getString(DRIVER_KEY, null); - this.username = parameters.getString(USERNAME_KEY, null); - this.password = parameters.getString(PASSWORD_KEY, null); - this.query = parameters.getString(QUERY_KEY, null); - this.dbURL = parameters.getString(URL_KEY, null); - } - - try { - prepareQueryExecution(); - } catch (SQLException e) { - throw new IllegalArgumentException("Configure failed:\t!", e); - } - } - - /** - * Enters data value from the current resultSet into a Record. - * - * @param pos - * Record position to be set. - * @param type - * SQL type of the resultSet value. - * @param record - * Target Record. - */ - private void retrieveTypeAndFillRecord(int pos, int type, Record record) throws SQLException, - NotTransformableSQLFieldException { - switch (type) { - case java.sql.Types.NULL: - record.setField(pos, NullValue.getInstance()); - break; - case java.sql.Types.BOOLEAN: - record.setField(pos, new BooleanValue(resultSet.getBoolean(pos + 1))); - break; - case java.sql.Types.BIT: - record.setField(pos, new BooleanValue(resultSet.getBoolean(pos + 1))); - break; - case java.sql.Types.CHAR: - record.setField(pos, new StringValue(resultSet.getString(pos + 1))); - break; - case java.sql.Types.NCHAR: - record.setField(pos, new StringValue(resultSet.getString(pos + 1))); - break; - case java.sql.Types.VARCHAR: - record.setField(pos, new StringValue(resultSet.getString(pos + 1))); - break; - case java.sql.Types.LONGVARCHAR: - record.setField(pos, new StringValue(resultSet.getString(pos + 1))); - break; - case java.sql.Types.LONGNVARCHAR: - record.setField(pos, new StringValue(resultSet.getString(pos + 1))); - break; - case java.sql.Types.TINYINT: - record.setField(pos, new ShortValue(resultSet.getShort(pos + 1))); - break; - case java.sql.Types.SMALLINT: - record.setField(pos, new ShortValue(resultSet.getShort(pos + 1))); - break; - case java.sql.Types.BIGINT: - record.setField(pos, new LongValue(resultSet.getLong(pos + 1))); - break; - case java.sql.Types.INTEGER: - record.setField(pos, new IntValue(resultSet.getInt(pos + 1))); - break; - case java.sql.Types.FLOAT: - record.setField(pos, new DoubleValue(resultSet.getDouble(pos + 1))); - break; - case java.sql.Types.REAL: - record.setField(pos, new FloatValue(resultSet.getFloat(pos + 1))); - break; - case java.sql.Types.DOUBLE: - record.setField(pos, new DoubleValue(resultSet.getDouble(pos + 1))); - break; - case java.sql.Types.DECIMAL: - record.setField(pos, new DoubleValue(resultSet.getBigDecimal(pos + 1).doubleValue())); - break; - case java.sql.Types.NUMERIC: - record.setField(pos, new DoubleValue(resultSet.getBigDecimal(pos + 1).doubleValue())); - break; - case java.sql.Types.DATE: - record.setField(pos, new StringValue(resultSet.getDate(pos + 1).toString())); - break; - case java.sql.Types.TIME: - record.setField(pos, new LongValue(resultSet.getTime(pos + 1).getTime())); - break; - case java.sql.Types.TIMESTAMP: - record.setField(pos, new StringValue(resultSet.getTimestamp(pos + 1).toString())); - break; - case java.sql.Types.SQLXML: - record.setField(pos, new StringValue(resultSet.getSQLXML(pos + 1).toString())); - break; - default: - throw new NotTransformableSQLFieldException("Unknown sql-type [" + type + "]on column [" + pos + "]"); - - // case java.sql.Types.BINARY: - // case java.sql.Types.VARBINARY: - // case java.sql.Types.LONGVARBINARY: - // case java.sql.Types.ARRAY: - // case java.sql.Types.JAVA_OBJECT: - // case java.sql.Types.BLOB: - // case java.sql.Types.CLOB: - // case java.sql.Types.NCLOB: - // case java.sql.Types.DATALINK: - // case java.sql.Types.DISTINCT: - // case java.sql.Types.OTHER: - // case java.sql.Types.REF: - // case java.sql.Types.ROWID: - // case java.sql.Types.STRUCT: - } - } - - private boolean isFieldNullOrEmpty(String field) { - return (field == null || field.length() == 0); - } - - private void prepareQueryExecution() throws SQLException { - setClassForDBType(); - prepareCredentialsAndExecute(); - } - - /** - * Loads appropriate JDBC driver. - * - * @param dbType - * Type of the database. - * @return boolean value, indication whether an appropriate driver could be - * found. - */ - private void setClassForDBType() { - try { - Class.forName(driverName); - } catch (ClassNotFoundException cnfe) { - throw new IllegalArgumentException("JDBC-Class not found:\t" + cnfe.getLocalizedMessage()); - } - } - - private void prepareCredentialsAndExecute() throws SQLException { - if (isFieldNullOrEmpty(username)) { - prepareConnection(dbURL); - } else { - prepareConnection(); - } - executeQuery(); - } - - /** - * Establishes a connection to a database. - * - * @param dbURL - * Assembled URL containing all connection parameters. - * @return boolean value, indicating whether a connection could be - * established - */ - private void prepareConnection(String dbURL) throws SQLException { - dbConn = DriverManager.getConnection(dbURL); - } - - /** - * Assembles the Database URL and establishes a connection. - * - * @param dbType - * Type of the database. - * @param username - * Login username. - * @param password - * Login password. - * @return boolean value, indicating whether a connection could be - * established - */ - private void prepareConnection() throws SQLException { - dbConn = DriverManager.getConnection(dbURL, username, password); - } - - private void executeQuery() throws SQLException { - statement = dbConn.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY); - resultSet = statement.executeQuery(this.query); - } - - /** - * Checks whether all data has been read. - * - * @return boolean value indication whether all data has been read. - */ - @Override - public boolean reachedEnd() { - try { - if (resultSet.isLast()) { - resultSet.close(); - statement.close(); - dbConn.close(); - return true; - } else { - return false; - } - } catch (SQLException e) { - throw new IllegalArgumentException("Couldn't evaluate reachedEnd():\t" + e.getMessage()); - } catch (NullPointerException e) { - throw new IllegalArgumentException("Couldn't access resultSet:\t" + e.getMessage()); - } - } - - /** - * Stores the next resultSet row in a Record - * - * @param record - * target Record - * @return boolean value indicating that the operation was successful - */ - @Override - public Record nextRecord(Record record) { - try { - resultSet.next(); - ResultSetMetaData rsmd = resultSet.getMetaData(); - int column_count = rsmd.getColumnCount(); - record.setNumFields(column_count); - - for (int pos = 0; pos < column_count; pos++) { - int type = rsmd.getColumnType(pos + 1); - retrieveTypeAndFillRecord(pos, type, record); - } - return record; - } catch (SQLException e) { - throw new IllegalArgumentException("Couldn't read data:\t" + e.getMessage()); - } catch (NotTransformableSQLFieldException e) { - throw new IllegalArgumentException("Couldn't read data because of unknown column sql-type:\t" - + e.getMessage()); - } catch (NullPointerException e) { - throw new IllegalArgumentException("Couldn't access resultSet:\t" + e.getMessage()); - } - } - - public static class NotTransformableSQLFieldException extends Exception { - - private static final long serialVersionUID = 1L; - - public NotTransformableSQLFieldException(String message) { - super(message); - } - } - -} diff --git a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormat.java b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormat.java deleted file mode 100644 index 780001a873ba7..0000000000000 --- a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormat.java +++ /dev/null @@ -1,359 +0,0 @@ -/* - * 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.api.java.record.io.jdbc; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.SQLException; - -import org.apache.flink.api.common.io.FileOutputFormat; -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.java.record.operators.GenericDataSink; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.BooleanValue; -import org.apache.flink.types.ByteValue; -import org.apache.flink.types.CharValue; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.ShortValue; -import org.apache.flink.types.StringValue; -import org.apache.flink.types.Value; - -public class JDBCOutputFormat implements OutputFormat { - private static final long serialVersionUID = 1L; - - private static final int DEFAULT_BATCH_INTERVERAL = 5000; - - public static final String DRIVER_KEY = "driver"; - public static final String USERNAME_KEY = "username"; - public static final String PASSWORD_KEY = "password"; - public static final String URL_KEY = "url"; - public static final String QUERY_KEY = "query"; - public static final String FIELD_COUNT_KEY = "fields"; - public static final String FIELD_TYPE_KEY = "type"; - public static final String BATCH_INTERVAL = "batchInt"; - - private Connection dbConn; - private PreparedStatement upload; - - private String username; - private String password; - private String driverName; - private String dbURL; - - private String query; - private int fieldCount; - private Class[] fieldClasses; - - /** - * Variable indicating the current number of insert sets in a batch. - */ - private int batchCount = 0; - - /** - * Commit interval of batches. - * High batch interval: faster inserts, more memory required (reduce if OutOfMemoryExceptions occur) - * low batch interval: slower inserts, less memory. - */ - private int batchInterval = DEFAULT_BATCH_INTERVERAL; - - - /** - * Configures this JDBCOutputFormat. - * - * @param parameters - * Configuration containing all parameters. - */ - @Override - public void configure(Configuration parameters) { - this.driverName = parameters.getString(DRIVER_KEY, null); - this.username = parameters.getString(USERNAME_KEY, null); - this.password = parameters.getString(PASSWORD_KEY, null); - this.dbURL = parameters.getString(URL_KEY, null); - this.query = parameters.getString(QUERY_KEY, null); - this.fieldCount = parameters.getInteger(FIELD_COUNT_KEY, 0); - this.batchInterval = parameters.getInteger(BATCH_INTERVAL, DEFAULT_BATCH_INTERVERAL); - - @SuppressWarnings("unchecked") - Class[] classes = new Class[this.fieldCount]; - this.fieldClasses = classes; - - ClassLoader cl = getClass().getClassLoader(); - - try { - for (int i = 0; i < this.fieldCount; i++) { - Class clazz = parameters.getClass(FIELD_TYPE_KEY + i, null, cl); - if (clazz == null) { - throw new IllegalArgumentException("Invalid configuration for JDBCOutputFormat: " - + "No type class for parameter " + i); - } - this.fieldClasses[i] = clazz; - } - } - catch (ClassNotFoundException e) { - throw new RuntimeException("Could not load data type classes.", e); - } - } - - /** - * Connects to the target database and initializes the prepared statement. - * - * @param taskNumber The number of the parallel instance. - * @throws IOException Thrown, if the output could not be opened due to an - * I/O problem. - */ - @Override - public void open(int taskNumber, int numTasks) throws IOException { - try { - establishConnection(); - upload = dbConn.prepareStatement(query); - } catch (SQLException sqe) { - throw new IllegalArgumentException("open() failed:\t!", sqe); - } catch (ClassNotFoundException cnfe) { - throw new IllegalArgumentException("JDBC-Class not found:\t", cnfe); - } - } - - private void establishConnection() throws SQLException, ClassNotFoundException { - Class.forName(driverName); - if (username == null) { - dbConn = DriverManager.getConnection(dbURL); - } else { - dbConn = DriverManager.getConnection(dbURL, username, password); - } - } - - /** - * Adds a record to the prepared statement. - *

    - * When this method is called, the output format is guaranteed to be opened. - * - * @param record The records to add to the output. - * @throws IOException Thrown, if the records could not be added due to an - * I/O problem. - */ - - @Override - public void writeRecord(Record record) throws IOException { - try { - for (int x = 0; x < record.getNumFields(); x++) { - Value temp = record.getField(x, fieldClasses[x]); - addValue(x + 1, temp); - } - upload.addBatch(); - batchCount++; - if(batchCount >= batchInterval) { - upload.executeBatch(); - batchCount = 0; - } - } catch (SQLException sqe) { - throw new IllegalArgumentException("writeRecord() failed:\t", sqe); - } catch (IllegalArgumentException iae) { - throw new IllegalArgumentException("writeRecord() failed:\t", iae); - } - } - - private enum pactType { - BooleanValue, - ByteValue, - CharValue, - DoubleValue, - FloatValue, - IntValue, - LongValue, - ShortValue, - StringValue - } - - private void addValue(int index, Value value) throws SQLException { - pactType type; - try { - type = pactType.valueOf(value.getClass().getSimpleName()); - } catch (IllegalArgumentException iae) { - throw new IllegalArgumentException("PactType not supported:\t", iae); - } - switch (type) { - case BooleanValue: - upload.setBoolean(index, ((BooleanValue) value).getValue()); - break; - case ByteValue: - upload.setByte(index, ((ByteValue) value).getValue()); - break; - case CharValue: - upload.setString(index, String.valueOf(((CharValue) value).getValue())); - break; - case DoubleValue: - upload.setDouble(index, ((DoubleValue) value).getValue()); - break; - case FloatValue: - upload.setFloat(index, ((FloatValue) value).getValue()); - break; - case IntValue: - upload.setInt(index, ((IntValue) value).getValue()); - break; - case LongValue: - upload.setLong(index, ((LongValue) value).getValue()); - break; - case ShortValue: - upload.setShort(index, ((ShortValue) value).getValue()); - break; - case StringValue: - upload.setString(index, ((StringValue) value).getValue()); - break; - } - } - - /** - * Executes prepared statement and closes all resources of this instance. - * - * @throws IOException Thrown, if the input could not be closed properly. - */ - @Override - public void close() throws IOException { - try { - upload.executeBatch(); - batchCount = 0; - upload.close(); - dbConn.close(); - } catch (SQLException sqe) { - throw new IllegalArgumentException("close() failed:\t", sqe); - } - } - - /** - * Creates a configuration builder that can be used to set the - * output format's parameters to the config in a fluent fashion. - * - * @return A config builder for setting parameters. - */ - public static ConfigBuilder configureOutputFormat(GenericDataSink target) { - return new ConfigBuilder(target.getParameters()); - } - - /** - * Abstract builder used to set parameters to the output format's - * configuration in a fluent way. - */ - protected static abstract class AbstractConfigBuilder - extends FileOutputFormat.AbstractConfigBuilder { - - /** - * Creates a new builder for the given configuration. - * - * @param config The configuration into which the parameters will be written. - */ - protected AbstractConfigBuilder(Configuration config) { - super(config); - } - - /** - * Sets the query field. - * @param value value to be set. - * @return The builder itself. - */ - public T setQuery(String value) { - this.config.setString(QUERY_KEY, value); - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - - /** - * Sets the url field. - * @param value value to be set. - * @return The builder itself. - */ - public T setUrl(String value) { - this.config.setString(URL_KEY, value); - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - - /** - * Sets the username field. - * @param value value to be set. - * @return The builder itself. - */ - public T setUsername(String value) { - this.config.setString(USERNAME_KEY, value); - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - - /** - * Sets the password field. - * @param value value to be set. - * @return The builder itself. - */ - public T setPassword(String value) { - this.config.setString(PASSWORD_KEY, value); - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - - /** - * Sets the driver field. - * @param value value to be set. - * @return The builder itself. - */ - public T setDriver(String value) { - this.config.setString(DRIVER_KEY, value); - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - - /** - * Sets the type of a column. - * Types are applied in the order they were set. - * @param type PactType to apply. - * @return The builder itself. - */ - public T setClass(Class type) { - final int numYet = this.config.getInteger(FIELD_COUNT_KEY, 0); - this.config.setClass(FIELD_TYPE_KEY + numYet, type); - this.config.setInteger(FIELD_COUNT_KEY, numYet + 1); - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - } - - /** - * A builder used to set parameters to the output format's configuration in a fluent way. - */ - public static final class ConfigBuilder extends AbstractConfigBuilder { - /** - * Creates a new builder for the given configuration. - * - * @param targetConfig The configuration into which the parameters will be written. - */ - protected ConfigBuilder(Configuration targetConfig) { - super(targetConfig); - } - } -} diff --git a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/example/JDBCExample.java b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/example/JDBCExample.java deleted file mode 100644 index 213fd6a714738..0000000000000 --- a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/record/io/jdbc/example/JDBCExample.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.api.java.record.io.jdbc.example; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.Statement; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.io.jdbc.JDBCInputFormat; -import org.apache.flink.api.java.record.io.jdbc.JDBCOutputFormat; -import org.apache.flink.api.java.record.operators.GenericDataSink; -import org.apache.flink.api.java.record.operators.GenericDataSource; -import org.apache.flink.client.LocalExecutor; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.StringValue; - -/** - * Stand-alone example for the JDBC connector. - * - * NOTE: To run this example, you need the apache derby code in your classpath. - * See the Maven file (pom.xml) for a reference to the derby dependency. You can - * simply Change the scope of the Maven dependency from test to compile. - */ -public class JDBCExample implements Program, ProgramDescription { - - @Override - public Plan getPlan(String[] args) { - /* - * In this example we use the constructor where the url contains all the settings that are needed. - * You could also use the default constructor and deliver a Configuration with all the needed settings. - * You also could set the settings to the source-instance. - */ - GenericDataSource source = new GenericDataSource( - new JDBCInputFormat( - "org.apache.derby.jdbc.EmbeddedDriver", - "jdbc:derby:memory:ebookshop", - "select * from books"), - "Data Source"); - - GenericDataSink sink = new GenericDataSink(new JDBCOutputFormat(), "Data Output"); - JDBCOutputFormat.configureOutputFormat(sink) - .setDriver("org.apache.derby.jdbc.EmbeddedDriver") - .setUrl("jdbc:derby:memory:ebookshop") - .setQuery("insert into newbooks (id,title,author,price,qty) values (?,?,?,?,?)") - .setClass(IntValue.class) - .setClass(StringValue.class) - .setClass(StringValue.class) - .setClass(FloatValue.class) - .setClass(IntValue.class); - - sink.addInput(source); - return new Plan(sink, "JDBC Example Job"); - } - - @Override - public String getDescription() { - return "Parameter:"; - } - - /* - * To run this example, you need the apache derby code in your classpath! - */ - public static void main(String[] args) throws Exception { - - prepareTestDb(); - JDBCExample tut = new JDBCExample(); - JobExecutionResult res = LocalExecutor.execute(tut, args); - System.out.println("runtime: " + res.getNetRuntime() + " ms"); - - System.exit(0); - } - - private static void prepareTestDb() throws Exception { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - Connection conn = DriverManager.getConnection(dbURL); - - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE books ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - - sqlQueryBuilder = new StringBuilder("CREATE TABLE newbooks ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - - sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', 'Tan Ah Teck', 11.11, 11),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', 'Tan Ah Teck', 22.22, 22),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', 'Mohammad Ali', 33.33, 33),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', 'Kumar', 44.44, 44),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', 'Kevin Jones', 55.55, 55)"); - - stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - - conn.close(); - } -} diff --git a/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/DevNullLogStream.java b/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/DevNullLogStream.java deleted file mode 100644 index 172f58573493d..0000000000000 --- a/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/DevNullLogStream.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.api.java.record.io.jdbc; - -import java.io.OutputStream; - -public class DevNullLogStream { - - public static final OutputStream DEV_NULL = new OutputStream() { - public void write(int b) {} - }; - -} diff --git a/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormatTest.java b/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormatTest.java deleted file mode 100644 index 8e0a2c5636993..0000000000000 --- a/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCInputFormatTest.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * 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.api.java.record.io.jdbc; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; - -import org.junit.Assert; - -import org.apache.flink.api.java.record.io.jdbc.JDBCInputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.types.Value; -import org.junit.After; -import org.junit.BeforeClass; -import org.junit.Test; - -public class JDBCInputFormatTest { - JDBCInputFormat jdbcInputFormat; - Configuration config; - static Connection conn; - static final Value[][] dbData = { - {new IntValue(1001), new StringValue("Java for dummies"), new StringValue("Tan Ah Teck"), new DoubleValue(11.11), new IntValue(11)}, - {new IntValue(1002), new StringValue("More Java for dummies"), new StringValue("Tan Ah Teck"), new DoubleValue(22.22), new IntValue(22)}, - {new IntValue(1003), new StringValue("More Java for more dummies"), new StringValue("Mohammad Ali"), new DoubleValue(33.33), new IntValue(33)}, - {new IntValue(1004), new StringValue("A Cup of Java"), new StringValue("Kumar"), new DoubleValue(44.44), new IntValue(44)}, - {new IntValue(1005), new StringValue("A Teaspoon of Java"), new StringValue("Kevin Jones"), new DoubleValue(55.55), new IntValue(55)}}; - - @BeforeClass - public static void setUpClass() { - try { - prepareDerbyDatabase(); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void prepareDerbyDatabase() throws ClassNotFoundException { - System.setProperty("derby.stream.error.field","org.apache.flink.api.java.record.io.jdbc.DevNullLogStream.DEV_NULL"); - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - createConnection(dbURL); - } - - private static void cleanUpDerbyDatabases() { - try { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - Statement stat = conn.createStatement(); - stat.executeUpdate("DROP TABLE books"); - stat.close(); - conn.close(); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(); - } - } - - /* - Loads JDBC derby driver ; creates(if necessary) and populates database. - */ - private static void createConnection(String dbURL) { - try { - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - createTable(); - insertDataToSQLTables(); - conn.close(); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void createTable() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE books ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - - sqlQueryBuilder = new StringBuilder("CREATE TABLE bookscontent ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("content BLOB(10K) DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - } - - private static void insertDataToSQLTables() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', 'Tan Ah Teck', 11.11, 11),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', 'Tan Ah Teck', 22.22, 22),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', 'Mohammad Ali', 33.33, 33),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', 'Kumar', 44.44, 44),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', 'Kevin Jones', 55.55, 55)"); - - Statement stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - - sqlQueryBuilder = new StringBuilder("INSERT INTO bookscontent (id, title, content) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', CAST(X'7f454c4602' AS BLOB)),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', CAST(X'7f454c4602' AS BLOB)),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', CAST(X'7f454c4602' AS BLOB)),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', CAST(X'7f454c4602' AS BLOB)),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', CAST(X'7f454c4602' AS BLOB))"); - - stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - } - - - @After - public void tearDown() { - jdbcInputFormat = null; - } - - @Test(expected = IllegalArgumentException.class) - public void testInvalidConnection() { - jdbcInputFormat = new JDBCInputFormat("org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:idontexist", "select * from books;"); - jdbcInputFormat.configure(null); - } - - @Test(expected = IllegalArgumentException.class) - public void testInvalidQuery() { - jdbcInputFormat = new JDBCInputFormat("org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:ebookshop", "abc"); - jdbcInputFormat.configure(null); - } - - @Test(expected = IllegalArgumentException.class) - public void testInvalidDBType() { - jdbcInputFormat = new JDBCInputFormat("idontexist.Driver", "jdbc:derby:memory:ebookshop", "select * from books;"); - jdbcInputFormat.configure(null); - } - - @Test(expected = IllegalArgumentException.class) - public void testUnsupportedSQLType() { - jdbcInputFormat = new JDBCInputFormat("org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:ebookshop", "select * from bookscontent"); - jdbcInputFormat.configure(null); - jdbcInputFormat.nextRecord(new Record()); - } - - @Test(expected = IllegalArgumentException.class) - public void testNotConfiguredFormatNext() { - jdbcInputFormat = new JDBCInputFormat("org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:ebookshop", "select * from books"); - jdbcInputFormat.nextRecord(new Record()); - } - - @Test(expected = IllegalArgumentException.class) - public void testNotConfiguredFormatEnd() { - jdbcInputFormat = new JDBCInputFormat("org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:ebookshop", "select * from books"); - jdbcInputFormat.reachedEnd(); - } - - @Test - public void testJDBCInputFormat() throws IOException { - jdbcInputFormat = new JDBCInputFormat("org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:ebookshop", "select * from books"); - jdbcInputFormat.configure(null); - Record record = new Record(); - int recordCount = 0; - while (!jdbcInputFormat.reachedEnd()) { - jdbcInputFormat.nextRecord(record); - Assert.assertEquals(5, record.getNumFields()); - Assert.assertEquals("Field 0 should be int", IntValue.class, record.getField(0, IntValue.class).getClass()); - Assert.assertEquals("Field 1 should be String", StringValue.class, record.getField(1, StringValue.class).getClass()); - Assert.assertEquals("Field 2 should be String", StringValue.class, record.getField(2, StringValue.class).getClass()); - Assert.assertEquals("Field 3 should be float", DoubleValue.class, record.getField(3, DoubleValue.class).getClass()); - Assert.assertEquals("Field 4 should be int", IntValue.class, record.getField(4, IntValue.class).getClass()); - - int[] pos = {0, 1, 2, 3, 4}; - Value[] values = {new IntValue(), new StringValue(), new StringValue(), new DoubleValue(), new IntValue()}; - Assert.assertTrue(record.equalsFields(pos, dbData[recordCount], values)); - - recordCount++; - } - Assert.assertEquals(5, recordCount); - - cleanUpDerbyDatabases(); - } - -} diff --git a/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormatTest.java b/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormatTest.java deleted file mode 100644 index c824ea1a61703..0000000000000 --- a/flink-staging/flink-jdbc/src/test/java/org/apache/flink/api/java/record/io/jdbc/JDBCOutputFormatTest.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * 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.api.java.record.io.jdbc; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; - -import org.junit.Assert; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.types.Value; -import org.junit.After; -import org.junit.BeforeClass; -import org.junit.Test; - -public class JDBCOutputFormatTest { - private JDBCInputFormat jdbcInputFormat; - private JDBCOutputFormat jdbcOutputFormat; - - private static Connection conn; - - static final Value[][] dbData = { - {new IntValue(1001), new StringValue("Java for dummies"), new StringValue("Tan Ah Teck"), new DoubleValue(11.11), new IntValue(11)}, - {new IntValue(1002), new StringValue("More Java for dummies"), new StringValue("Tan Ah Teck"), new DoubleValue(22.22), new IntValue(22)}, - {new IntValue(1003), new StringValue("More Java for more dummies"), new StringValue("Mohammad Ali"), new DoubleValue(33.33), new IntValue(33)}, - {new IntValue(1004), new StringValue("A Cup of Java"), new StringValue("Kumar"), new DoubleValue(44.44), new IntValue(44)}, - {new IntValue(1005), new StringValue("A Teaspoon of Java"), new StringValue("Kevin Jones"), new DoubleValue(55.55), new IntValue(55)}}; - - @BeforeClass - public static void setUpClass() { - try { - System.setProperty("derby.stream.error.field", "org.apache.flink.api.java.record.io.jdbc.DevNullLogStream.DEV_NULL"); - prepareDerbyInputDatabase(); - prepareDerbyOutputDatabase(); - } catch (ClassNotFoundException e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void cleanUpDerbyDatabases() { - try { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - Statement stat = conn.createStatement(); - stat.executeUpdate("DROP TABLE books"); - stat.executeUpdate("DROP TABLE newbooks"); - stat.close(); - conn.close(); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void prepareDerbyInputDatabase() throws ClassNotFoundException { - try { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - createTableBooks(); - insertDataToSQLTables(); - conn.close(); - } catch (ClassNotFoundException e) { - e.printStackTrace(); - Assert.fail(); - } catch (SQLException e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void prepareDerbyOutputDatabase() throws ClassNotFoundException { - try { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - createTableNewBooks(); - conn.close(); - } catch (ClassNotFoundException e) { - e.printStackTrace(); - Assert.fail(); - } catch (SQLException e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void createTableBooks() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE books ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - } - - private static void createTableNewBooks() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE newbooks ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - } - - private static void insertDataToSQLTables() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', 'Tan Ah Teck', 11.11, 11),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', 'Tan Ah Teck', 22.22, 22),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', 'Mohammad Ali', 33.33, 33),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', 'Kumar', 44.44, 44),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', 'Kevin Jones', 55.55, 55)"); - - Statement stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - } - - - @After - public void tearDown() { - jdbcOutputFormat = null; - cleanUpDerbyDatabases(); - } - - @Test - public void testJDBCOutputFormat() throws IOException { - String sourceTable = "books"; - String targetTable = "newbooks"; - String driverPath = "org.apache.derby.jdbc.EmbeddedDriver"; - String dbUrl = "jdbc:derby:memory:ebookshop"; - - Configuration cfg = new Configuration(); - cfg.setString("driver", driverPath); - cfg.setString("url", dbUrl); - cfg.setString("query", "insert into " + targetTable + " (id, title, author, price, qty) values (?,?,?,?,?)"); - cfg.setInteger("fields", 5); - cfg.setClass("type0", IntValue.class); - cfg.setClass("type1", StringValue.class); - cfg.setClass("type2", StringValue.class); - cfg.setClass("type3", FloatValue.class); - cfg.setClass("type4", IntValue.class); - - jdbcOutputFormat = new JDBCOutputFormat(); - jdbcOutputFormat.configure(cfg); - jdbcOutputFormat.open(0,1); - - jdbcInputFormat = new JDBCInputFormat( - driverPath, - dbUrl, - "select * from " + sourceTable); - jdbcInputFormat.configure(null); - - Record record = new Record(); - while (!jdbcInputFormat.reachedEnd()) { - jdbcInputFormat.nextRecord(record); - jdbcOutputFormat.writeRecord(record); - } - - jdbcOutputFormat.close(); - jdbcInputFormat.close(); - - jdbcInputFormat = new JDBCInputFormat( - driverPath, - dbUrl, - "select * from " + targetTable); - jdbcInputFormat.configure(null); - - int recordCount = 0; - while (!jdbcInputFormat.reachedEnd()) { - jdbcInputFormat.nextRecord(record); - Assert.assertEquals(5, record.getNumFields()); - Assert.assertEquals("Field 0 should be int", IntValue.class, record.getField(0, IntValue.class).getClass()); - Assert.assertEquals("Field 1 should be String", StringValue.class, record.getField(1, StringValue.class).getClass()); - Assert.assertEquals("Field 2 should be String", StringValue.class, record.getField(2, StringValue.class).getClass()); - Assert.assertEquals("Field 3 should be float", DoubleValue.class, record.getField(3, DoubleValue.class).getClass()); - Assert.assertEquals("Field 4 should be int", IntValue.class, record.getField(4, IntValue.class).getClass()); - - int[] pos = {0, 1, 2, 3, 4}; - Value[] values = {new IntValue(), new StringValue(), new StringValue(), new DoubleValue(), new IntValue()}; - Assert.assertTrue(record.equalsFields(pos, dbData[recordCount], values)); - - recordCount++; - } - Assert.assertEquals(5, recordCount); - - jdbcInputFormat.close(); - } -} From b2d8c40a06e0a36e90913d316ff2b003b701fee1 Mon Sep 17 00:00:00 2001 From: Enrique Bautista Date: Tue, 21 Jul 2015 18:52:05 +0200 Subject: [PATCH 074/175] [FLINK-2205] Fix confusing entries in JobManager WebUI JobConfig section. Default display for 'Number of execution retries' is now 'deactivated' and for 'Job parallelism' is 'auto', as suggested in JIRA. This closes #927 --- .../flink/runtime/jobmanager/web/JobManagerInfoServlet.java | 2 +- .../src/main/resources/web-docs-infoserver/js/analyzer.js | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java index ce57714ca5611..0ecc941467234 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java @@ -436,7 +436,7 @@ private void writeJsonForArchivedJob(PrintWriter wrt, ExecutionGraph graph) { if(ec != null) { wrt.write("\"executionConfig\": {"); wrt.write("\"Execution Mode\": \""+ec.getExecutionMode()+"\","); - wrt.write("\"Number of execution retries\": \""+ec.getNumberOfExecutionRetries()+"\","); + wrt.write("\"Max. number of execution retries\": \""+ec.getNumberOfExecutionRetries()+"\","); wrt.write("\"Job parallelism\": \""+ec.getParallelism()+"\","); wrt.write("\"Object reuse mode\": \""+ec.isObjectReuseEnabled()+"\""); ExecutionConfig.GlobalJobParameters uc = ec.getGlobalJobParameters(); diff --git a/flink-runtime/src/main/resources/web-docs-infoserver/js/analyzer.js b/flink-runtime/src/main/resources/web-docs-infoserver/js/analyzer.js index 3934017566815..4030f805c9daf 100644 --- a/flink-runtime/src/main/resources/web-docs-infoserver/js/analyzer.js +++ b/flink-runtime/src/main/resources/web-docs-infoserver/js/analyzer.js @@ -107,6 +107,10 @@ function analyzeTime(json, stacked) { $.each(job.executionConfig, function(key, value) { if(key == "userConfig") { return; + } else if(key == "Max. number of execution retries" && value == -1) { + value = "deactivated"; + } else if(key == "Job parallelism" && value == -1) { + value = "auto"; } configTable += ""+key+""+value+""; }); From 30761572b5040669b07d261ec9b109797debc549 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 30 Jul 2015 21:44:06 +0200 Subject: [PATCH 075/175] [FLINK-2442] [fix] FieldPositionKeys support Pojo fields This closes #963 --- .../apache/flink/api/java/operators/Keys.java | 50 +++++++++---------- .../api/java/typeutils/TupleTypeInfoBase.java | 20 -------- .../flink/api/java/operators/KeysTest.java | 27 ++++++++++ 3 files changed, 52 insertions(+), 45 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java index 69d306f891260..09874e5d5e10f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java @@ -223,43 +223,43 @@ public ExpressionKeys(int[] groupingFields, TypeInformation type, boolean all } else { groupingFields = rangeCheckFields(groupingFields, type.getArity() -1); } - CompositeType compositeType = (CompositeType) type; Preconditions.checkArgument(groupingFields.length > 0, "Grouping fields can not be empty at this point"); keyFields = new ArrayList(type.getTotalFields()); // for each key, find the field: for(int j = 0; j < groupingFields.length; j++) { + int keyPos = groupingFields[j]; + + int offset = 0; for(int i = 0; i < type.getArity(); i++) { - TypeInformation fieldType = compositeType.getTypeAt(i); - - if(groupingFields[j] == i) { // check if user set the key - int keyId = countNestedElementsBefore(compositeType, i) + i; - if(fieldType instanceof TupleTypeInfoBase) { - TupleTypeInfoBase tupleFieldType = (TupleTypeInfoBase) fieldType; - tupleFieldType.addAllFields(keyId, keyFields); - } else { - Preconditions.checkArgument(fieldType instanceof AtomicType, "Wrong field type"); - keyFields.add(new FlatFieldDescriptor(keyId, fieldType)); + + TypeInformation fieldType = ((CompositeType) type).getTypeAt(i); + if(i < keyPos) { + // not yet there, increment key offset + offset += fieldType.getTotalFields(); + } + else { + // arrived at key position + if(fieldType instanceof CompositeType) { + // add all nested fields of composite type + ((CompositeType) fieldType).getFlatFields("*", offset, keyFields); } - + else if(fieldType instanceof AtomicType) { + // add atomic type field + keyFields.add(new FlatFieldDescriptor(offset, fieldType)); + } + else { + // type should either be composite or atomic + throw new InvalidProgramException("Field type is neither CompositeType nor AtomicType: "+fieldType); + } + // go to next key + break; } } } keyFields = removeNullElementsFromList(keyFields); } - - private static int countNestedElementsBefore(CompositeType compositeType, int pos) { - if( pos == 0) { - return 0; - } - int ret = 0; - for (int i = 0; i < pos; i++) { - TypeInformation fieldType = compositeType.getTypeAt(i); - ret += fieldType.getTotalFields() -1; - } - return ret; - } - + public static List removeNullElementsFromList(List in) { List elements = new ArrayList(); for(R e: in) { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java index 3314ca9ae42a2..881e690d5ccd7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java @@ -23,7 +23,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.java.operators.Keys.ExpressionKeys; @@ -88,25 +87,6 @@ public Class getTypeClass() { return tupleType; } - /** - * Recursively add all fields in this tuple type. We need this in particular to get all - * the types. - * @param startKeyId - * @param keyFields - */ - public void addAllFields(int startKeyId, List keyFields) { - for(int i = 0; i < this.getArity(); i++) { - TypeInformation type = this.types[i]; - if(type instanceof AtomicType) { - keyFields.add(new FlatFieldDescriptor(startKeyId, type)); - } else if(type instanceof TupleTypeInfoBase) { - TupleTypeInfoBase ttb = (TupleTypeInfoBase) type; - ttb.addAllFields(startKeyId, keyFields); - } - startKeyId += type.getTotalFields(); - } - } - @Override public void getFlatFields(String fieldExpression, int offset, List result) { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/KeysTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/KeysTest.java index 67d02406baef1..cf8936d195b16 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/KeysTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/KeysTest.java @@ -28,6 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest.ComplexNestedClass; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.junit.Assert; @@ -254,4 +255,30 @@ public void testPojoKeys() { ek = new ExpressionKeys(new String[]{"i0"}, ti); Assert.assertArrayEquals(new int[] {0}, ek.computeLogicalKeyPositions()); } + + @Test + public void testTupleWithNestedPojo() { + + TypeInformation> ti = + new TupleTypeInfo>( + BasicTypeInfo.INT_TYPE_INFO, + TypeExtractor.getForClass(Pojo1.class), + TypeExtractor.getForClass(PojoWithMultiplePojos.class) + ); + + ExpressionKeys> ek; + + ek = new ExpressionKeys>(new int[]{0}, ti); + Assert.assertArrayEquals(new int[] {0}, ek.computeLogicalKeyPositions()); + + ek = new ExpressionKeys>(new int[]{1}, ti); + Assert.assertArrayEquals(new int[] {1,2}, ek.computeLogicalKeyPositions()); + + ek = new ExpressionKeys>(new int[]{2}, ti); + Assert.assertArrayEquals(new int[] {3,4,5,6,7}, ek.computeLogicalKeyPositions()); + + ek = new ExpressionKeys>(new int[]{}, ti, true); + Assert.assertArrayEquals(new int[] {0,1,2,3,4,5,6,7}, ek.computeLogicalKeyPositions()); + + } } From 0dc6849a594b61a6cad8ee582ca1758f0349a72b Mon Sep 17 00:00:00 2001 From: Johann Kovacs Date: Fri, 10 Jul 2015 17:21:58 +0200 Subject: [PATCH 076/175] [FLINK-2105] Extract abstract superclass, interface from MergeMatchIterators, KeyGroupedIterators --- .../operators/sort/AbstractMergeIterator.java | 356 ++++++++++++++++ .../sort/AbstractMergeMatchIterator.java | 107 +++++ .../sort/NonReusingMergeMatchIterator.java | 382 +---------------- .../sort/ReusingMergeMatchIterator.java | 389 +----------------- .../runtime/util/KeyGroupedIterator.java | 31 ++ .../util/NonReusingKeyGroupedIterator.java | 2 +- .../util/ReusingKeyGroupedIterator.java | 5 +- 7 files changed, 517 insertions(+), 755 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java new file mode 100644 index 0000000000000..9a61c140660c3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java @@ -0,0 +1,356 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator; +import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator; +import org.apache.flink.runtime.operators.util.JoinTaskIterator; +import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +public abstract class AbstractMergeIterator implements JoinTaskIterator { + + private final Logger LOG = LoggerFactory.getLogger(getClass()); + + protected TypePairComparator pairComparator; + + protected KeyGroupedIterator iterator1; + protected KeyGroupedIterator iterator2; + + protected final TypeSerializer serializer1; + protected final TypeSerializer serializer2; + + private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key + + private final IOManager ioManager; + private final MemoryManager memoryManager; + private final List memoryForSpillingIterator; + + // instances for object reuse + protected T1 copy1; + protected T1 spillHeadCopy; + protected T2 copy2; + protected T2 blockHeadCopy; + + public AbstractMergeIterator(MutableObjectIterator input1, MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) throws MemoryAllocationException { + if (numMemoryPages < 2) { + throw new IllegalArgumentException("Merger needs at least 2 memory pages."); + } + + this.pairComparator = pairComparator; + this.serializer1 = serializer1; + this.serializer2 = serializer2; + + this.memoryManager = memoryManager; + this.ioManager = ioManager; + + this.iterator1 = createKeyGroupedIterator(input1, serializer1, comparator1.duplicate()); + this.iterator2 = createKeyGroupedIterator(input2, serializer2, comparator2.duplicate()); + + final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; + this.blockIt = new NonReusingBlockResettableIterator(this.memoryManager, this.serializer2, + (numMemoryPages - numPagesForSpiller), parentTask); + this.memoryForSpillingIterator = memoryManager.allocatePages(parentTask, numPagesForSpiller); + } + + @Override + public void open() throws IOException { + } + + @Override + public void close() { + if (this.blockIt != null) { + try { + this.blockIt.close(); + } catch (Throwable t) { + LOG.error("Error closing block memory iterator: " + t.getMessage(), t); + } + } + + this.memoryManager.release(this.memoryForSpillingIterator); + } + + @Override + public void abort() { + close(); + } + + /** + * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come + * from different inputs. The output of the match() method is forwarded. + *

    + * This method first zig-zags between the two sorted inputs in order to find a common + * key, and then calls the match stub with the cross product of the values. + * + * @throws Exception Forwards all exceptions from the user code and the I/O system. + * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) + */ + @Override + public abstract boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) + throws Exception; + + protected void crossMatchingGroup(Iterator values1, Iterator values2, FlatJoinFunction matchFunction, Collector collector) throws Exception { + final T1 firstV1 = values1.next(); + final T2 firstV2 = values2.next(); + + final boolean v1HasNext = values1.hasNext(); + final boolean v2HasNext = values2.hasNext(); + + // check if one side is already empty + // this check could be omitted if we put this in MatchTask. + // then we can derive the local strategy (with build side). + + if (v1HasNext) { + if (v2HasNext) { + // both sides contain more than one value + // TODO: Decide which side to spill and which to block! + crossMwithNValues(firstV1, values1, firstV2, values2, matchFunction, collector); + } else { + crossSecond1withNValues(firstV2, firstV1, values1, matchFunction, collector); + } + } else { + if (v2HasNext) { + crossFirst1withNValues(firstV1, firstV2, values2, matchFunction, collector); + } else { + // both sides contain only one value + matchFunction.join(firstV1, firstV2, collector); + } + } + } + + /** + * Crosses a single value from the first input with N values, all sharing a common key. + * Effectively realizes a 1:N match (join). + * + * @param val1 The value form the 1 side. + * @param firstValN The first of the values from the N side. + * @param valsN Iterator over remaining N side values. + * @throws Exception Forwards all exceptions thrown by the stub. + */ + private void crossFirst1withNValues(final T1 val1, final T2 firstValN, + final Iterator valsN, final FlatJoinFunction matchFunction, final Collector collector) + throws Exception { + T1 copy1 = createCopy(serializer1, val1, this.copy1); + matchFunction.join(copy1, firstValN, collector); + + // set copy and match first element + boolean more = true; + do { + final T2 nRec = valsN.next(); + + if (valsN.hasNext()) { + copy1 = createCopy(serializer1, val1, this.copy1); + matchFunction.join(copy1, nRec, collector); + } else { + matchFunction.join(val1, nRec, collector); + more = false; + } + } + while (more); + } + + /** + * Crosses a single value from the second side with N values, all sharing a common key. + * Effectively realizes a N:1 match (join). + * + * @param val1 The value form the 1 side. + * @param firstValN The first of the values from the N side. + * @param valsN Iterator over remaining N side values. + * @throws Exception Forwards all exceptions thrown by the stub. + */ + private void crossSecond1withNValues(T2 val1, T1 firstValN, + Iterator valsN, FlatJoinFunction matchFunction, Collector collector) throws Exception { + T2 copy2 = createCopy(serializer2, val1, this.copy2); + matchFunction.join(firstValN, copy2, collector); + + // set copy and match first element + boolean more = true; + do { + final T1 nRec = valsN.next(); + + if (valsN.hasNext()) { + copy2 = createCopy(serializer2, val1, this.copy2); + matchFunction.join(nRec, copy2, collector); + } else { + matchFunction.join(nRec, val1, collector); + more = false; + } + } + while (more); + } + + private void crossMwithNValues(final T1 firstV1, Iterator spillVals, + final T2 firstV2, final Iterator blockVals, + final FlatJoinFunction matchFunction, final Collector collector) throws Exception { + // ================================================== + // We have one first (head) element from both inputs (firstV1 and firstV2) + // We have an iterator for both inputs. + // we make the V1 side the spilling side and the V2 side the blocking side. + // In order to get the full cross product without unnecessary spilling, we do the + // following: + // 1) cross the heads + // 2) cross the head of the spilling side against the first block of the blocking side + // 3) cross the iterator of the spilling side with the head of the block side + // 4) cross the iterator of the spilling side with the first block + // --------------------------------------------------- + // If the blocking side has more than one block, we really need to make the spilling side fully + // resettable. For each further block on the block side, we do: + // 5) cross the head of the spilling side with the next block + // 6) cross the spilling iterator with the next block. + + // match the first values first + T1 copy1 = this.createCopy(serializer1, firstV1, this.copy1); + T2 blockHeadCopy = this.createCopy(serializer2, firstV2, this.blockHeadCopy); + T1 spillHeadCopy = null; + + // --------------- 1) Cross the heads ------------------- + matchFunction.join(copy1, firstV2, collector); + + // for the remaining values, we do a block-nested-loops join + SpillingResettableIterator spillIt = null; + + try { + // create block iterator on the second input + this.blockIt.reopen(blockVals); + + // ------------- 2) cross the head of the spilling side with the first block ------------------ + while (this.blockIt.hasNext()) { + final T2 nextBlockRec = this.blockIt.next(); + copy1 = this.createCopy(serializer1, firstV1, this.copy1); + matchFunction.join(copy1, nextBlockRec, collector); + } + this.blockIt.reset(); + + // spilling is required if the blocked input has data beyond the current block. + // in that case, create the spilling iterator + final Iterator leftSideIter; + final boolean spillingRequired = this.blockIt.hasFurtherInput(); + if (spillingRequired) { + // more data than would fit into one block. we need to wrap the other side in a spilling iterator + // create spilling iterator on first input + spillIt = new SpillingResettableIterator(spillVals, this.serializer1, + this.memoryManager, this.ioManager, this.memoryForSpillingIterator); + leftSideIter = spillIt; + spillIt.open(); + + spillHeadCopy = this.createCopy(serializer1, firstV1, this.spillHeadCopy); + } else { + leftSideIter = spillVals; + } + + // cross the values in the v1 iterator against the current block + + while (leftSideIter.hasNext()) { + final T1 nextSpillVal = leftSideIter.next(); + copy1 = this.createCopy(serializer1, nextSpillVal, this.copy1); + + + // -------- 3) cross the iterator of the spilling side with the head of the block side -------- + T2 copy2 = this.createCopy(serializer2, blockHeadCopy, this.copy2); + matchFunction.join(copy1, copy2, collector); + + // -------- 4) cross the iterator of the spilling side with the first block -------- + while (this.blockIt.hasNext()) { + T2 nextBlockRec = this.blockIt.next(); + + // get instances of key and block value + copy1 = this.createCopy(serializer1, nextSpillVal, this.copy1); + matchFunction.join(copy1, nextBlockRec, collector); + } + // reset block iterator + this.blockIt.reset(); + } + + // if everything from the block-side fit into a single block, we are done. + // note that in this special case, we did not create a spilling iterator at all + if (!spillingRequired) { + return; + } + + // here we are, because we have more blocks on the block side + // loop as long as there are blocks from the blocked input + while (this.blockIt.nextBlock()) { + // rewind the spilling iterator + spillIt.reset(); + + // ------------- 5) cross the head of the spilling side with the next block ------------ + while (this.blockIt.hasNext()) { + copy1 = this.createCopy(serializer1, spillHeadCopy, this.copy1); + final T2 nextBlockVal = blockIt.next(); + matchFunction.join(copy1, nextBlockVal, collector); + } + this.blockIt.reset(); + + // -------- 6) cross the spilling iterator with the next block. ------------------ + while (spillIt.hasNext()) { + // get value from resettable iterator + final T1 nextSpillVal = spillIt.next(); + // cross value with block values + while (this.blockIt.hasNext()) { + // get instances of key and block value + final T2 nextBlockVal = this.blockIt.next(); + copy1 = this.createCopy(serializer1, nextSpillVal, this.copy1); + matchFunction.join(copy1, nextBlockVal, collector); + } + + // reset block iterator + this.blockIt.reset(); + } + // reset v1 iterator + spillIt.reset(); + } + } finally { + if (spillIt != null) { + this.memoryForSpillingIterator.addAll(spillIt.close()); + } + } + } + + + protected abstract KeyGroupedIterator createKeyGroupedIterator(MutableObjectIterator input, TypeSerializer serializer, TypeComparator comparator); + + /** + * Copies an instance of the given type, potentially reusing the object passed as the reuse parameter, which may be null. + */ + protected abstract T createCopy(TypeSerializer serializer, T value, T reuse); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java new file mode 100644 index 0000000000000..791494d720aee --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java @@ -0,0 +1,107 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; + +import java.util.Iterator; + +/** + * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the + * matching through a sort-merge join strategy. + */ +public abstract class AbstractMergeMatchIterator extends AbstractMergeIterator { + + public AbstractMergeMatchIterator(MutableObjectIterator input1, MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) + throws MemoryAllocationException { + super(input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + } + + /** + * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come + * from different inputs. The output of the match() method is forwarded. + *

    + * This method first zig-zags between the two sorted inputs in order to find a common + * key, and then calls the match stub with the cross product of the values. + * + * @throws Exception Forwards all exceptions from the user code and the I/O system. + * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) + */ + @Override + public boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) + throws Exception { + if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { + // consume all remaining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon) + while (this.iterator1.nextKey()) ; + while (this.iterator2.nextKey()) ; + + return false; + } + + final TypePairComparator comparator = this.pairComparator; + comparator.setReference(this.iterator1.getCurrent()); + T2 current2 = this.iterator2.getCurrent(); + + // zig zag + while (true) { + // determine the relation between the (possibly composite) keys + final int comp = comparator.compareToReference(current2); + + if (comp == 0) { + break; + } + + if (comp < 0) { + if (!this.iterator2.nextKey()) { + return false; + } + current2 = this.iterator2.getCurrent(); + } else { + if (!this.iterator1.nextKey()) { + return false; + } + comparator.setReference(this.iterator1.getCurrent()); + } + } + + // here, we have a common key! call the match function with the cross product of the + // values + final Iterator values1 = this.iterator1.getValues(); + final Iterator values2 = this.iterator2.getValues(); + + crossMatchingGroup(values1, values2, matchFunction, collector); + return true; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java index c89b5c5dbe8c9..9705778c55f82 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -18,60 +18,19 @@ package org.apache.flink.runtime.operators.sort; -import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator; -import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator; -import org.apache.flink.runtime.operators.util.JoinTaskIterator; +import org.apache.flink.runtime.util.KeyGroupedIterator; import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; -import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -/** - * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the - * matching through a sort-merge join strategy. - */ -public class NonReusingMergeMatchIterator implements JoinTaskIterator { - - /** - * The log used by this iterator to log messages. - */ - private static final Logger LOG = LoggerFactory.getLogger(NonReusingMergeMatchIterator.class); - - // -------------------------------------------------------------------------------------------- - - private TypePairComparator comp; - - private NonReusingKeyGroupedIterator iterator1; - - private NonReusingKeyGroupedIterator iterator2; - - private final TypeSerializer serializer1; - - private final TypeSerializer serializer2; - - private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key - - private final List memoryForSpillingIterator; - - private final MemoryManager memoryManager; - - private final IOManager ioManager; - - // -------------------------------------------------------------------------------------------- +public class NonReusingMergeMatchIterator extends AbstractMergeMatchIterator { public NonReusingMergeMatchIterator( MutableObjectIterator input1, @@ -83,341 +42,18 @@ public NonReusingMergeMatchIterator( IOManager ioManager, int numMemoryPages, AbstractInvokable parentTask) - throws MemoryAllocationException - { - if (numMemoryPages < 2) { - throw new IllegalArgumentException("Merger needs at least 2 memory pages."); - } - - this.comp = pairComparator; - this.serializer1 = serializer1; - this.serializer2 = serializer2; - - this.memoryManager = memoryManager; - this.ioManager = ioManager; - - this.iterator1 = new NonReusingKeyGroupedIterator(input1, comparator1.duplicate()); - this.iterator2 = new NonReusingKeyGroupedIterator(input2, comparator2.duplicate()); - - final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; - this.blockIt = new NonReusingBlockResettableIterator(this.memoryManager, this.serializer2, - (numMemoryPages - numPagesForSpiller), parentTask); - this.memoryForSpillingIterator = memoryManager.allocatePages(parentTask, numPagesForSpiller); + throws MemoryAllocationException { + super(input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); } - - @Override - public void open() throws IOException {} - - - @Override - public void close() { - if (this.blockIt != null) { - try { - this.blockIt.close(); - } - catch (Throwable t) { - LOG.error("Error closing block memory iterator: " + t.getMessage(), t); - } - } - - this.memoryManager.release(this.memoryForSpillingIterator); - } - - @Override - public void abort() { - close(); + protected KeyGroupedIterator createKeyGroupedIterator(MutableObjectIterator input, TypeSerializer serializer, TypeComparator comparator) { + return new NonReusingKeyGroupedIterator(input, comparator); } - /** - * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come - * from different inputs. The output of the match() method is forwarded. - *

    - * This method first zig-zags between the two sorted inputs in order to find a common - * key, and then calls the match stub with the cross product of the values. - * - * @throws Exception Forwards all exceptions from the user code and the I/O system. - * - * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) - */ @Override - public boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) - throws Exception - { - if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { - // consume all remaining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon) - while (this.iterator1.nextKey()); - while (this.iterator2.nextKey()); - - return false; - } - - final TypePairComparator comparator = this.comp; - comparator.setReference(this.iterator1.getCurrent()); - T2 current2 = this.iterator2.getCurrent(); - - // zig zag - while (true) { - // determine the relation between the (possibly composite) keys - final int comp = comparator.compareToReference(current2); - - if (comp == 0) { - break; - } - - if (comp < 0) { - if (!this.iterator2.nextKey()) { - return false; - } - current2 = this.iterator2.getCurrent(); - } - else { - if (!this.iterator1.nextKey()) { - return false; - } - comparator.setReference(this.iterator1.getCurrent()); - } - } - - // here, we have a common key! call the match function with the cross product of the - // values - final NonReusingKeyGroupedIterator.ValuesIterator values1 = this.iterator1.getValues(); - final NonReusingKeyGroupedIterator.ValuesIterator values2 = this.iterator2.getValues(); - - final T1 firstV1 = values1.next(); - final T2 firstV2 = values2.next(); - - final boolean v1HasNext = values1.hasNext(); - final boolean v2HasNext = values2.hasNext(); - - // check if one side is already empty - // this check could be omitted if we put this in MatchTask. - // then we can derive the local strategy (with build side). - - if (v1HasNext) { - if (v2HasNext) { - // both sides contain more than one value - // TODO: Decide which side to spill and which to block! - crossMwithNValues(firstV1, values1, firstV2, values2, matchFunction, collector); - } else { - crossSecond1withNValues(firstV2, firstV1, values1, matchFunction, collector); - } - } else { - if (v2HasNext) { - crossFirst1withNValues(firstV1, firstV2, values2, matchFunction, collector); - } else { - // both sides contain only one value - matchFunction.join(firstV1, firstV2, collector); - } - } - return true; - } - - /** - * Crosses a single value from the first input with N values, all sharing a common key. - * Effectively realizes a 1:N match (join). - * - * @param val1 The value form the 1 side. - * @param firstValN The first of the values from the N side. - * @param valsN Iterator over remaining N side values. - * - * @throws Exception Forwards all exceptions thrown by the stub. - */ - private void crossFirst1withNValues(final T1 val1, final T2 firstValN, - final Iterator valsN, final FlatJoinFunction matchFunction, final Collector collector) - throws Exception - { - T1 copy1 = this.serializer1.copy(val1); - matchFunction.join(copy1, firstValN, collector); - - // set copy and match first element - boolean more = true; - do { - final T2 nRec = valsN.next(); - - if (valsN.hasNext()) { - copy1 = this.serializer1.copy(val1); - matchFunction.join(copy1, nRec, collector); - } else { - matchFunction.join(val1, nRec, collector); - more = false; - } - } - while (more); - } - - /** - * Crosses a single value from the second side with N values, all sharing a common key. - * Effectively realizes a N:1 match (join). - * - * @param val1 The value form the 1 side. - * @param firstValN The first of the values from the N side. - * @param valsN Iterator over remaining N side values. - * - * @throws Exception Forwards all exceptions thrown by the stub. - */ - private void crossSecond1withNValues(T2 val1, T1 firstValN, - Iterator valsN, FlatJoinFunction matchFunction, Collector collector) - throws Exception - { - T2 copy2 = this.serializer2.copy(val1); - matchFunction.join(firstValN, copy2, collector); - - // set copy and match first element - boolean more = true; - do { - final T1 nRec = valsN.next(); - - if (valsN.hasNext()) { - copy2 = this.serializer2.copy(val1); - matchFunction.join(nRec, copy2, collector); - } else { - matchFunction.join(nRec, val1, collector); - more = false; - } - } - while (more); + protected T createCopy(TypeSerializer serializer, T value, T reuse) { + return serializer.copy(value); } - - /** - * @param firstV1 - * @param spillVals - * @param firstV2 - * @param blockVals - */ - private void crossMwithNValues(final T1 firstV1, Iterator spillVals, - final T2 firstV2, final Iterator blockVals, - final FlatJoinFunction matchFunction, final Collector collector) - throws Exception - { - // ================================================== - // We have one first (head) element from both inputs (firstV1 and firstV2) - // We have an iterator for both inputs. - // we make the V1 side the spilling side and the V2 side the blocking side. - // In order to get the full cross product without unnecessary spilling, we do the - // following: - // 1) cross the heads - // 2) cross the head of the spilling side against the first block of the blocking side - // 3) cross the iterator of the spilling side with the head of the block side - // 4) cross the iterator of the spilling side with the first block - // --------------------------------------------------- - // If the blocking side has more than one block, we really need to make the spilling side fully - // resettable. For each further block on the block side, we do: - // 5) cross the head of the spilling side with the next block - // 6) cross the spilling iterator with the next block. - - // match the first values first - T1 copy1 = this.serializer1.copy(firstV1); - T2 blockHeadCopy = this.serializer2.copy(firstV2); - T1 spillHeadCopy = null; - - // --------------- 1) Cross the heads ------------------- - matchFunction.join(copy1, firstV2, collector); - - // for the remaining values, we do a block-nested-loops join - SpillingResettableIterator spillIt = null; - - try { - // create block iterator on the second input - this.blockIt.reopen(blockVals); - - // ------------- 2) cross the head of the spilling side with the first block ------------------ - while (this.blockIt.hasNext()) { - final T2 nextBlockRec = this.blockIt.next(); - copy1 = this.serializer1.copy(firstV1); - matchFunction.join(copy1, nextBlockRec, collector); - } - this.blockIt.reset(); - - // spilling is required if the blocked input has data beyond the current block. - // in that case, create the spilling iterator - final Iterator leftSideIter; - final boolean spillingRequired = this.blockIt.hasFurtherInput(); - if (spillingRequired) - { - // more data than would fit into one block. we need to wrap the other side in a spilling iterator - // create spilling iterator on first input - spillIt = new SpillingResettableIterator(spillVals, this.serializer1, - this.memoryManager, this.ioManager, this.memoryForSpillingIterator); - leftSideIter = spillIt; - spillIt.open(); - - spillHeadCopy = this.serializer1.copy(firstV1); - } - else { - leftSideIter = spillVals; - } - - // cross the values in the v1 iterator against the current block - - while (leftSideIter.hasNext()) { - final T1 nextSpillVal = leftSideIter.next(); - copy1 = this.serializer1.copy(nextSpillVal); - - - // -------- 3) cross the iterator of the spilling side with the head of the block side -------- - T2 copy2 = this.serializer2.copy(blockHeadCopy); - matchFunction.join(copy1, copy2, collector); - - // -------- 4) cross the iterator of the spilling side with the first block -------- - while (this.blockIt.hasNext()) { - T2 nextBlockRec = this.blockIt.next(); - - // get instances of key and block value - copy1 = this.serializer1.copy(nextSpillVal); - matchFunction.join(copy1, nextBlockRec, collector); - } - // reset block iterator - this.blockIt.reset(); - } - - // if everything from the block-side fit into a single block, we are done. - // note that in this special case, we did not create a spilling iterator at all - if (!spillingRequired) { - return; - } - - // here we are, because we have more blocks on the block side - // loop as long as there are blocks from the blocked input - while (this.blockIt.nextBlock()) - { - // rewind the spilling iterator - spillIt.reset(); - - // ------------- 5) cross the head of the spilling side with the next block ------------ - while (this.blockIt.hasNext()) { - copy1 = this.serializer1.copy(spillHeadCopy); - final T2 nextBlockVal = blockIt.next(); - matchFunction.join(copy1, nextBlockVal, collector); - } - this.blockIt.reset(); - - // -------- 6) cross the spilling iterator with the next block. ------------------ - while (spillIt.hasNext()) - { - // get value from resettable iterator - final T1 nextSpillVal = spillIt.next(); - // cross value with block values - while (this.blockIt.hasNext()) { - // get instances of key and block value - final T2 nextBlockVal = this.blockIt.next(); - copy1 = this.serializer1.copy(nextSpillVal); - matchFunction.join(copy1, nextBlockVal, collector); - } - - // reset block iterator - this.blockIt.reset(); - } - // reset v1 iterator - spillIt.reset(); - } - } - finally { - if (spillIt != null) { - this.memoryForSpillingIterator.addAll(spillIt.close()); - } - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java index 66beee1f807fb..c9cf5a2b9626d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java @@ -18,70 +18,20 @@ package org.apache.flink.runtime.operators.sort; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator; -import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator; -import org.apache.flink.runtime.operators.util.JoinTaskIterator; +import org.apache.flink.runtime.util.KeyGroupedIterator; import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; -import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -/** - * An implementation of the {@link JoinTaskIterator} that realizes the - * matching through a sort-merge join strategy. - */ -public class ReusingMergeMatchIterator implements JoinTaskIterator { - - /** - * The log used by this iterator to log messages. - */ - private static final Logger LOG = LoggerFactory.getLogger(ReusingMergeMatchIterator.class); - - // -------------------------------------------------------------------------------------------- - - private TypePairComparator comp; - - private ReusingKeyGroupedIterator iterator1; - - private ReusingKeyGroupedIterator iterator2; - - private final TypeSerializer serializer1; - - private final TypeSerializer serializer2; - - private T1 copy1; - - private T1 spillHeadCopy; - - private T2 copy2; - - private T2 blockHeadCopy; - - private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key - - private final List memoryForSpillingIterator; - - private final MemoryManager memoryManager; +public class ReusingMergeMatchIterator extends AbstractMergeMatchIterator { - private final IOManager ioManager; - - // -------------------------------------------------------------------------------------------- - public ReusingMergeMatchIterator( MutableObjectIterator input1, MutableObjectIterator input2, @@ -92,344 +42,23 @@ public ReusingMergeMatchIterator( IOManager ioManager, int numMemoryPages, AbstractInvokable parentTask) - throws MemoryAllocationException - { - if (numMemoryPages < 2) { - throw new IllegalArgumentException("Merger needs at least 2 memory pages."); - } - - this.comp = pairComparator; - this.serializer1 = serializer1; - this.serializer2 = serializer2; - + throws MemoryAllocationException { + super(input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + this.copy1 = serializer1.createInstance(); this.spillHeadCopy = serializer1.createInstance(); this.copy2 = serializer2.createInstance(); this.blockHeadCopy = serializer2.createInstance(); - - this.memoryManager = memoryManager; - this.ioManager = ioManager; - - this.iterator1 = new ReusingKeyGroupedIterator(input1, this.serializer1, comparator1.duplicate()); - this.iterator2 = new ReusingKeyGroupedIterator(input2, this.serializer2, comparator2.duplicate()); - - final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; - this.blockIt = new NonReusingBlockResettableIterator(this.memoryManager, this.serializer2, - (numMemoryPages - numPagesForSpiller), parentTask); - this.memoryForSpillingIterator = memoryManager.allocatePages(parentTask, numPagesForSpiller); - } - - - @Override - public void open() throws IOException {} - - - @Override - public void close() { - if (this.blockIt != null) { - try { - this.blockIt.close(); - } - catch (Throwable t) { - LOG.error("Error closing block memory iterator: " + t.getMessage(), t); - } - } - - this.memoryManager.release(this.memoryForSpillingIterator); } - @Override - public void abort() { - close(); + protected KeyGroupedIterator createKeyGroupedIterator(MutableObjectIterator input, TypeSerializer serializer, TypeComparator comparator) { + return new ReusingKeyGroupedIterator(input, serializer, comparator); } - /** - * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come - * from different inputs. The output of the match() method is forwarded. - *

    - * This method first zig-zags between the two sorted inputs in order to find a common - * key, and then calls the match stub with the cross product of the values. - * - * @throws Exception Forwards all exceptions from the user code and the I/O system. - * - * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(FlatJoinFunction, Collector) - */ @Override - public boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) - throws Exception - { - if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { - // consume all remaining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon) - while (this.iterator1.nextKey()); - while (this.iterator2.nextKey()); - - return false; - } - - final TypePairComparator comparator = this.comp; - comparator.setReference(this.iterator1.getCurrent()); - T2 current2 = this.iterator2.getCurrent(); - - // zig zag - while (true) { - // determine the relation between the (possibly composite) keys - final int comp = comparator.compareToReference(current2); - - if (comp == 0) { - break; - } - - if (comp < 0) { - if (!this.iterator2.nextKey()) { - return false; - } - current2 = this.iterator2.getCurrent(); - } - else { - if (!this.iterator1.nextKey()) { - return false; - } - comparator.setReference(this.iterator1.getCurrent()); - } - } - - // here, we have a common key! call the match function with the cross product of the - // values - final ReusingKeyGroupedIterator.ValuesIterator values1 = this.iterator1.getValues(); - final ReusingKeyGroupedIterator.ValuesIterator values2 = this.iterator2.getValues(); - - final T1 firstV1 = values1.next(); - final T2 firstV2 = values2.next(); - - final boolean v1HasNext = values1.hasNext(); - final boolean v2HasNext = values2.hasNext(); - - // check if one side is already empty - // this check could be omitted if we put this in MatchTask. - // then we can derive the local strategy (with build side). - - if (v1HasNext) { - if (v2HasNext) { - // both sides contain more than one value - // TODO: Decide which side to spill and which to block! - crossMwithNValues(firstV1, values1, firstV2, values2, matchFunction, collector); - } else { - crossSecond1withNValues(firstV2, firstV1, values1, matchFunction, collector); - } - } else { - if (v2HasNext) { - crossFirst1withNValues(firstV1, firstV2, values2, matchFunction, collector); - } else { - // both sides contain only one value - matchFunction.join(firstV1, firstV2, collector); - } - } - return true; + protected T createCopy(TypeSerializer serializer, T value, T reuse) { + return serializer.copy(value, reuse); } - /** - * Crosses a single value from the first input with N values, all sharing a common key. - * Effectively realizes a 1:N match (join). - * - * @param val1 The value form the 1 side. - * @param firstValN The first of the values from the N side. - * @param valsN Iterator over remaining N side values. - * - * @throws Exception Forwards all exceptions thrown by the stub. - */ - private void crossFirst1withNValues(final T1 val1, final T2 firstValN, - final Iterator valsN, final FlatJoinFunction matchFunction, final Collector collector) - throws Exception - { - this.copy1 = this.serializer1.copy(val1, this.copy1); - matchFunction.join(this.copy1, firstValN, collector); - - // set copy and match first element - boolean more = true; - do { - final T2 nRec = valsN.next(); - - if (valsN.hasNext()) { - this.copy1 = this.serializer1.copy(val1, this.copy1); - matchFunction.join(this.copy1, nRec, collector); - } else { - matchFunction.join(val1, nRec, collector); - more = false; - } - } - while (more); - } - - /** - * Crosses a single value from the second side with N values, all sharing a common key. - * Effectively realizes a N:1 match (join). - * - * @param val1 The value form the 1 side. - * @param firstValN The first of the values from the N side. - * @param valsN Iterator over remaining N side values. - * - * @throws Exception Forwards all exceptions thrown by the stub. - */ - private void crossSecond1withNValues(T2 val1, T1 firstValN, - Iterator valsN, FlatJoinFunction matchFunction, Collector collector) - throws Exception - { - this.copy2 = this.serializer2.copy(val1, this.copy2); - matchFunction.join(firstValN, this.copy2, collector); - - // set copy and match first element - boolean more = true; - do { - final T1 nRec = valsN.next(); - - if (valsN.hasNext()) { - this.copy2 = this.serializer2.copy(val1, this.copy2); - matchFunction.join(nRec,this.copy2,collector); - } else { - matchFunction.join(nRec, val1, collector); - more = false; - } - } - while (more); - } - - /** - * @param firstV1 - * @param spillVals - * @param firstV2 - * @param blockVals - */ - private void crossMwithNValues(final T1 firstV1, Iterator spillVals, - final T2 firstV2, final Iterator blockVals, - final FlatJoinFunction matchFunction, final Collector collector) - throws Exception - { - // ================================================== - // We have one first (head) element from both inputs (firstV1 and firstV2) - // We have an iterator for both inputs. - // we make the V1 side the spilling side and the V2 side the blocking side. - // In order to get the full cross product without unnecessary spilling, we do the - // following: - // 1) cross the heads - // 2) cross the head of the spilling side against the first block of the blocking side - // 3) cross the iterator of the spilling side with the head of the block side - // 4) cross the iterator of the spilling side with the first block - // --------------------------------------------------- - // If the blocking side has more than one block, we really need to make the spilling side fully - // resettable. For each further block on the block side, we do: - // 5) cross the head of the spilling side with the next block - // 6) cross the spilling iterator with the next block. - - // match the first values first - this.copy1 = this.serializer1.copy(firstV1, this.copy1); - this.blockHeadCopy = this.serializer2.copy(firstV2, this.blockHeadCopy); - - // --------------- 1) Cross the heads ------------------- - matchFunction.join(this.copy1, firstV2, collector); - - // for the remaining values, we do a block-nested-loops join - SpillingResettableIterator spillIt = null; - - try { - // create block iterator on the second input - this.blockIt.reopen(blockVals); - - // ------------- 2) cross the head of the spilling side with the first block ------------------ - while (this.blockIt.hasNext()) { - final T2 nextBlockRec = this.blockIt.next(); - this.copy1 = this.serializer1.copy(firstV1, this.copy1); - matchFunction.join(this.copy1, nextBlockRec, collector); - } - this.blockIt.reset(); - - // spilling is required if the blocked input has data beyond the current block. - // in that case, create the spilling iterator - final Iterator leftSideIter; - final boolean spillingRequired = this.blockIt.hasFurtherInput(); - if (spillingRequired) - { - // more data than would fit into one block. we need to wrap the other side in a spilling iterator - // create spilling iterator on first input - spillIt = new SpillingResettableIterator(spillVals, this.serializer1, - this.memoryManager, this.ioManager, this.memoryForSpillingIterator); - leftSideIter = spillIt; - spillIt.open(); - - this.spillHeadCopy = this.serializer1.copy(firstV1, this.spillHeadCopy); - } - else { - leftSideIter = spillVals; - } - - // cross the values in the v1 iterator against the current block - - while (leftSideIter.hasNext()) { - final T1 nextSpillVal = leftSideIter.next(); - this.copy1 = this.serializer1.copy(nextSpillVal, this.copy1); - - - // -------- 3) cross the iterator of the spilling side with the head of the block side -------- - this.copy2 = this.serializer2.copy(this.blockHeadCopy, this.copy2); - matchFunction.join(this.copy1, this.copy2, collector); - - // -------- 4) cross the iterator of the spilling side with the first block -------- - while (this.blockIt.hasNext()) { - T2 nextBlockRec = this.blockIt.next(); - - // get instances of key and block value - this.copy1 = this.serializer1.copy(nextSpillVal, this.copy1); - matchFunction.join(this.copy1, nextBlockRec, collector); - } - // reset block iterator - this.blockIt.reset(); - } - - // if everything from the block-side fit into a single block, we are done. - // note that in this special case, we did not create a spilling iterator at all - if (!spillingRequired) { - return; - } - - // here we are, because we have more blocks on the block side - // loop as long as there are blocks from the blocked input - while (this.blockIt.nextBlock()) - { - // rewind the spilling iterator - spillIt.reset(); - - // ------------- 5) cross the head of the spilling side with the next block ------------ - while (this.blockIt.hasNext()) { - this.copy1 = this.serializer1.copy(this.spillHeadCopy, this.copy1); - final T2 nextBlockVal = blockIt.next(); - matchFunction.join(this.copy1, nextBlockVal, collector); - } - this.blockIt.reset(); - - // -------- 6) cross the spilling iterator with the next block. ------------------ - while (spillIt.hasNext()) - { - // get value from resettable iterator - final T1 nextSpillVal = spillIt.next(); - // cross value with block values - while (this.blockIt.hasNext()) { - // get instances of key and block value - final T2 nextBlockVal = this.blockIt.next(); - this.copy1 = this.serializer1.copy(nextSpillVal, this.copy1); - matchFunction.join(this.copy1, nextBlockVal, collector); - } - - // reset block iterator - this.blockIt.reset(); - } - // reset v1 iterator - spillIt.reset(); - } - } - finally { - if (spillIt != null) { - this.memoryForSpillingIterator.addAll(spillIt.close()); - } - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java new file mode 100644 index 0000000000000..64e8298b2a3b7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java @@ -0,0 +1,31 @@ +/* + * 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.runtime.util; + +import java.io.IOException; +import java.util.Iterator; + +public interface KeyGroupedIterator { + + boolean nextKey() throws IOException; + + E getCurrent(); + + Iterator getValues(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java index 3f28cfcfd876d..6f4448c9b33bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java @@ -29,7 +29,7 @@ /** * The key grouped iterator returns a key and all values that share the same key. */ -public final class NonReusingKeyGroupedIterator { +public final class NonReusingKeyGroupedIterator implements KeyGroupedIterator { private final MutableObjectIterator iterator; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java index 4dc9dd38ef487..1477f1020f35f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java @@ -31,7 +31,7 @@ * The KeyValueIterator returns a key and all values that belong to the key (share the same key). * */ -public final class ReusingKeyGroupedIterator { +public final class ReusingKeyGroupedIterator implements KeyGroupedIterator { private final MutableObjectIterator iterator; @@ -78,6 +78,7 @@ public ReusingKeyGroupedIterator(MutableObjectIterator iterator, TypeSerializ * * @return true if the input iterator has an other group of key-value pairs that share the same key. */ + @Override public boolean nextKey() throws IOException { // first element (or empty) @@ -139,6 +140,7 @@ public TypeComparator getComparatorWithCurrentReference() { return this.comparator; } + @Override public E getCurrent() { return this.current; } @@ -150,6 +152,7 @@ public E getCurrent() { * * @return Iterator over all values that belong to the current key. */ + @Override public ValuesIterator getValues() { return this.valuesIterator; } From db0b0087b02985f55bcc6e65571b11ca33b0886f Mon Sep 17 00:00:00 2001 From: Johann Kovacs Date: Fri, 10 Jul 2015 17:26:05 +0200 Subject: [PATCH 077/175] [FLINK-2105] Add support for sorted but sparse test data generation --- .../runtime/operators/testutils/TestData.java | 207 ++++++++++++++++++ 1 file changed, 207 insertions(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java index fd34a3ba3dfbb..8688d4eb7f00b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java @@ -21,6 +21,8 @@ import java.util.Comparator; import java.util.Random; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; @@ -279,6 +281,169 @@ public void reset() { this.counter = 0; } } + + /** + * Tuple2 generator. + */ + public static class TupleGenerator implements MutableObjectIterator> { + + public enum KeyMode { + SORTED, RANDOM, SORTED_SPARSE + }; + + public enum ValueMode { + FIX_LENGTH, RANDOM_LENGTH, CONSTANT + }; + + private static char[] alpha = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'a', 'b', 'c', + 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm' }; + + private final long seed; + + private final int keyMax; + + private final float keyDensity; + + private final int valueLength; + + private final KeyMode keyMode; + + private final ValueMode valueMode; + + private Random random; + + private int counter; + + private int key; + private String value; + + public TupleGenerator(long seed, int keyMax, int valueLength) { + this(seed, keyMax, valueLength, KeyMode.RANDOM, ValueMode.FIX_LENGTH); + } + + public TupleGenerator(long seed, int keyMax, int valueLength, KeyMode keyMode, ValueMode valueMode) { + this(seed, keyMax, valueLength, keyMode, valueMode, null); + } + + public TupleGenerator(long seed, int keyMax, int valueLength, KeyMode keyMode, ValueMode valueMode, String constant) { + this(seed, keyMax, 1.0f, valueLength, keyMode, valueMode, constant); + } + + public TupleGenerator(long seed, int keyMax, float keyDensity, int valueLength, KeyMode keyMode, ValueMode valueMode, String constant) { + this.seed = seed; + this.keyMax = keyMax; + this.keyDensity = keyDensity; + this.valueLength = valueLength; + this.keyMode = keyMode; + this.valueMode = valueMode; + + this.random = new Random(seed); + this.counter = 0; + + this.value = constant == null ? null : constant; + } + + public Tuple2 next(Tuple2 reuse) { + this.key = nextKey(); + if (this.valueMode != ValueMode.CONSTANT) { + this.value = randomString(); + } + reuse.setFields(this.key, this.value); + return reuse; + } + + public Tuple2 next() { + return next(new Tuple2()); + } + + public boolean next(org.apache.flink.types.Value[] target) { + this.key = nextKey(); + // TODO change this to something proper + ((IntValue)target[0]).setValue(this.key); + ((IntValue)target[1]).setValue(random.nextInt()); + return true; + } + + private int nextKey() { + if (keyMode == KeyMode.SORTED) { + return ++counter; + } else if (keyMode == KeyMode.SORTED_SPARSE) { + int max = (int) (1 / keyDensity); + counter += random.nextInt(max) + 1; + return counter; + } else { + return Math.abs(random.nextInt() % keyMax) + 1; + } + } + + public void reset() { + this.random = new Random(seed); + this.counter = 0; + } + + private String randomString() { + int length; + + if (valueMode == ValueMode.FIX_LENGTH) { + length = valueLength; + } else { + length = valueLength - random.nextInt(valueLength / 3); + } + + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < length; i++) { + sb.append(alpha[random.nextInt(alpha.length)]); + } + return sb.toString(); + } + + } + + + /** + * Record reader mock. + */ + public static class TupleGeneratorIterator implements MutableObjectIterator> { + + private final TupleGenerator generator; + + private final int numberOfRecords; + + private int counter; + + public TupleGeneratorIterator(TupleGenerator generator, int numberOfRecords) { + this.generator = generator; + this.generator.reset(); + this.numberOfRecords = numberOfRecords; + this.counter = 0; + } + + @Override + public Tuple2 next(Tuple2 target) { + if (counter < numberOfRecords) { + counter++; + return generator.next(target); + } + else { + return null; + } + } + + @Override + public Tuple2 next() { + if (counter < numberOfRecords) { + counter++; + return generator.next(); + } + else { + return null; + } + } + + public void reset() { + this.counter = 0; + } + } // -------------------------------------------------------------------------------------------- @@ -325,4 +490,46 @@ public void reset() { this.pos = 0; } } + + public static class TupleConstantValueIterator implements MutableObjectIterator> { + + private int key; + private String value; + + private final String valueValue; + + + private final int numPairs; + + private int pos; + + + public TupleConstantValueIterator(int keyValue, String valueValue, int numPairs) { + this.key = keyValue; + this.valueValue = valueValue; + this.numPairs = numPairs; + } + + @Override + public Tuple2 next(Tuple2 reuse) { + if (pos < this.numPairs) { + this.value = this.valueValue + ' ' + pos; + reuse.setFields(this.key, this.value); + pos++; + return reuse; + } + else { + return null; + } + } + + @Override + public Tuple2 next() { + return next(new Tuple2()); + } + + public void reset() { + this.pos = 0; + } + } } From df9f4819b9368600c7531dbf4d4ec42c1cddea8f Mon Sep 17 00:00:00 2001 From: r-pogalz Date: Mon, 3 Aug 2015 12:59:01 +0200 Subject: [PATCH 078/175] [FLINK-2105] [tests] Move duplicate utility classes to testutil package --- .../flink/runtime/operators/MatchDriver.java | 8 +- ...va => AbstractMergeInnerJoinIterator.java} | 35 +- ... => NonReusingMergeInnerJoinIterator.java} | 4 +- ...ava => ReusingMergeInnerJoinIterator.java} | 4 +- ...usingSortMergeInnerJoinIteratorITCase.java | 318 +++++++++++++++ ...onReusingSortMergeMatchIteratorITCase.java | 371 ------------------ ...usingSortMergeInnerJoinIteratorITCase.java | 318 +++++++++++++++ .../ReusingSortMergeMatchIteratorITCase.java | 371 ------------------ .../testutils/CollectionIterator.java | 61 +++ .../runtime/operators/testutils/Match.java | 63 +++ .../testutils/MatchRemovingMatcher.java | 58 +++ .../testutils/SimpleTupleJoinFunction.java | 41 ++ .../util/HashVsSortMiniBenchmark.java | 6 +- 13 files changed, 888 insertions(+), 770 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/{AbstractMergeMatchIterator.java => AbstractMergeInnerJoinIterator.java} (75%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/{NonReusingMergeMatchIterator.java => NonReusingMergeInnerJoinIterator.java} (94%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/{ReusingMergeMatchIterator.java => ReusingMergeInnerJoinIterator.java} (94%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/CollectionIterator.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/SimpleTupleJoinFunction.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java index 0381aaba8e8b9..e54fca57d8f70 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator; -import org.apache.flink.runtime.operators.sort.NonReusingMergeMatchIterator; +import org.apache.flink.runtime.operators.sort.NonReusingMergeInnerJoinIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.FlatJoinFunction; @@ -33,7 +33,7 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; -import org.apache.flink.runtime.operators.sort.ReusingMergeMatchIterator; +import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; @@ -126,7 +126,7 @@ public void prepare() throws Exception{ if (this.objectReuseEnabled) { switch (ls) { case MERGE: - this.matchIterator = new ReusingMergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.matchIterator = new ReusingMergeInnerJoinIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: @@ -141,7 +141,7 @@ public void prepare() throws Exception{ } else { switch (ls) { case MERGE: - this.matchIterator = new NonReusingMergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.matchIterator = new NonReusingMergeInnerJoinIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java similarity index 75% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java index 791494d720aee..e9ccf521260bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java @@ -33,34 +33,35 @@ /** * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the - * matching through a sort-merge join strategy. + * joining through a sort-merge join strategy. */ -public abstract class AbstractMergeMatchIterator extends AbstractMergeIterator { +public abstract class AbstractMergeInnerJoinIterator extends AbstractMergeIterator { - public AbstractMergeMatchIterator(MutableObjectIterator input1, MutableObjectIterator input2, - TypeSerializer serializer1, TypeComparator comparator1, - TypeSerializer serializer2, TypeComparator comparator2, - TypePairComparator pairComparator, - MemoryManager memoryManager, - IOManager ioManager, - int numMemoryPages, - AbstractInvokable parentTask) + public AbstractMergeInnerJoinIterator( + MutableObjectIterator input1, MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) throws MemoryAllocationException { super(input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); } /** - * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come - * from different inputs. The output of the match() method is forwarded. - *

    + * Calls the JoinFunction#join() method for all two key-value pairs that share the same key and come + * from different inputs. The output of the join() method is forwarded. + *

    * This method first zig-zags between the two sorted inputs in order to find a common - * key, and then calls the match stub with the cross product of the values. + * key, and then calls the join stub with the cross product of the values. * * @throws Exception Forwards all exceptions from the user code and the I/O system. * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) */ @Override - public boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) + public boolean callWithNextKey(final FlatJoinFunction joinFunction, final Collector collector) throws Exception { if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { // consume all remaining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon) @@ -96,12 +97,12 @@ public boolean callWithNextKey(final FlatJoinFunction matchFunction, } } - // here, we have a common key! call the match function with the cross product of the + // here, we have a common key! call the join function with the cross product of the // values final Iterator values1 = this.iterator1.getValues(); final Iterator values2 = this.iterator2.getValues(); - crossMatchingGroup(values1, values2, matchFunction, collector); + crossMatchingGroup(values1, values2, joinFunction, collector); return true; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java similarity index 94% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java index 9705778c55f82..644084cdc4520 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java @@ -30,9 +30,9 @@ import org.apache.flink.util.MutableObjectIterator; -public class NonReusingMergeMatchIterator extends AbstractMergeMatchIterator { +public class NonReusingMergeInnerJoinIterator extends AbstractMergeInnerJoinIterator { - public NonReusingMergeMatchIterator( + public NonReusingMergeInnerJoinIterator( MutableObjectIterator input1, MutableObjectIterator input2, TypeSerializer serializer1, TypeComparator comparator1, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java similarity index 94% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java index c9cf5a2b9626d..3a1a17a9205d4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java @@ -30,9 +30,9 @@ import org.apache.flink.util.MutableObjectIterator; -public class ReusingMergeMatchIterator extends AbstractMergeMatchIterator { +public class ReusingMergeInnerJoinIterator extends AbstractMergeInnerJoinIterator { - public ReusingMergeMatchIterator( + public ReusingMergeInnerJoinIterator( MutableObjectIterator input1, MutableObjectIterator input2, TypeSerializer serializer1, TypeComparator comparator1, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java new file mode 100644 index 0000000000000..7fc37349fb47a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java @@ -0,0 +1,318 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.GenericPairComparator; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntComparator; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.TupleComparator; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.*; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.*; +import java.util.Map.Entry; + +@SuppressWarnings("deprecation") +public class NonReusingSortMergeInnerJoinIteratorITCase { + + // total memory + private static final int MEMORY_SIZE = 1024 * 1024 * 16; + private static final int PAGES_FOR_BNLJN = 2; + + // the size of the left and right inputs + private static final int INPUT_1_SIZE = 20000; + + private static final int INPUT_2_SIZE = 1000; + + // random seeds for the left and right input data generators + private static final long SEED1 = 561349061987311L; + + private static final long SEED2 = 231434613412342L; + + // dummy abstract task + private final AbstractInvokable parentTask = new DummyInvokable(); + + private IOManager ioManager; + private MemoryManager memoryManager; + + private TypeSerializer> serializer1; + private TypeSerializer> serializer2; + private TypeComparator> comparator1; + private TypeComparator> comparator2; + private TypePairComparator, Tuple2> pairComparator; + + + @SuppressWarnings("unchecked") + @Before + public void beforeTest() { + serializer1 = new TupleSerializer>( + (Class>) (Class) Tuple2.class, + new TypeSerializer[] { IntSerializer.INSTANCE, StringSerializer.INSTANCE }); + serializer2 = new TupleSerializer>( + (Class>) (Class) Tuple2.class, + new TypeSerializer[] { IntSerializer.INSTANCE, StringSerializer.INSTANCE }); + comparator1 = new TupleComparator>( + new int[]{0}, + new TypeComparator[] { new IntComparator(true) }, + new TypeSerializer[] { IntSerializer.INSTANCE }); + comparator2 = new TupleComparator>( + new int[]{0}, + new TypeComparator[] { new IntComparator(true) }, + new TypeSerializer[] { IntSerializer.INSTANCE }); + pairComparator = new GenericPairComparator, Tuple2>(comparator1, comparator2); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.ioManager = new IOManagerAsync(); + } + + @After + public void afterTest() { + if (this.ioManager != null) { + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + Assert.fail("I/O manager failed to properly shut down."); + } + this.ioManager = null; + } + + if (this.memoryManager != null) { + Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", + this.memoryManager.verifyEmpty()); + this.memoryManager.shutdown(); + this.memoryManager = null; + } + } + + @Test + public void testMerge() { + try { + + final TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchValues( + collectData(input1), + collectData(input2)); + + final FlatJoinFunction, Tuple2, Tuple2> joinFunction = + new MatchRemovingMatcher(expectedMatchesMap); + + final Collector> collector = new DiscardingOutputCollector>(); + + // reset the generators + generator1.reset(); + generator2.reset(); + input1.reset(); + input2.reset(); + + // compare with iterator values + NonReusingMergeInnerJoinIterator, Tuple2, Tuple2> iterator = + new NonReusingMergeInnerJoinIterator, Tuple2, Tuple2>( + input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(joinFunction, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + Assert.assertTrue("Collection for key " + entry.getKey() + " is not empty", entry.getValue().isEmpty()); + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testMergeWithHighNumberOfCommonKeys() + { + // the size of the left and right inputs + final int INPUT_1_SIZE = 200; + final int INPUT_2_SIZE = 100; + + final int INPUT_1_DUPLICATES = 10; + final int INPUT_2_DUPLICATES = 4000; + final int DUPLICATE_KEY = 13; + + try { + final TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE); + + final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); + final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); + + final List>> inList1 = new ArrayList>>(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + final List>> inList2 = new ArrayList>>(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + MutableObjectIterator> input1 = new MergeIterator>(inList1, comparator1.duplicate()); + MutableObjectIterator> input2 = new MergeIterator>(inList2, comparator2.duplicate()); + + // collect expected data + final Map> expectedMatchesMap = matchValues( + collectData(input1), + collectData(input2)); + + // re-create the whole thing for actual processing + + // reset the generators and iterators + generator1.reset(); + generator2.reset(); + const1Iter.reset(); + const2Iter.reset(); + gen1Iter.reset(); + gen2Iter.reset(); + + inList1.clear(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + inList2.clear(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + input1 = new MergeIterator>(inList1, comparator1.duplicate()); + input2 = new MergeIterator>(inList2, comparator2.duplicate()); + + final FlatJoinFunction, Tuple2, Tuple2> joinFunction = new MatchRemovingMatcher(expectedMatchesMap); + + final Collector> collector = new DiscardingOutputCollector>(); + + + // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it + // needs to spill for the duplicate keys + NonReusingMergeInnerJoinIterator, Tuple2, Tuple2> iterator = + new NonReusingMergeInnerJoinIterator, Tuple2, Tuple2>( + input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(joinFunction, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private Map> matchValues( + Map> leftMap, + Map> rightMap) + { + Map> map = new HashMap>(); + + for (Integer key : leftMap.keySet()) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + + if (rightValues == null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection matchedValues = map.get(key); + + for (String leftValue : leftValues) { + for (String rightValue : rightValues) { + matchedValues.add(new Match(leftValue, rightValue)); + } + } + } + + return map; + } + + + private Map> collectData(MutableObjectIterator> iter) + throws Exception + { + Map> map = new HashMap>(); + Tuple2 pair = new Tuple2(); + + while ((pair = iter.next(pair)) != null) { + final Integer key = pair.getField(0); + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection values = map.get(key); + final String value = pair.getField(1); + values.add(value); + } + + return map; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java deleted file mode 100644 index 757b2e78890de..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java +++ /dev/null @@ -1,371 +0,0 @@ -/* - * 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.runtime.operators.sort; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypePairComparator; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.record.RecordComparator; -import org.apache.flink.api.common.typeutils.record.RecordPairComparator; -import org.apache.flink.api.common.typeutils.record.RecordSerializer; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; -import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; -import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.apache.flink.runtime.operators.testutils.TestData; -import org.apache.flink.runtime.operators.testutils.TestData.Generator; -import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; -import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; -import org.apache.flink.types.Record; -import org.apache.flink.types.Value; -import org.apache.flink.util.Collector; -import org.apache.flink.util.MutableObjectIterator; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -@SuppressWarnings("deprecation") -public class NonReusingSortMergeMatchIteratorITCase { - - // total memory - private static final int MEMORY_SIZE = 1024 * 1024 * 16; - private static final int PAGES_FOR_BNLJN = 2; - - // the size of the left and right inputs - private static final int INPUT_1_SIZE = 20000; - - private static final int INPUT_2_SIZE = 1000; - - // random seeds for the left and right input data generators - private static final long SEED1 = 561349061987311L; - - private static final long SEED2 = 231434613412342L; - - // dummy abstract task - private final AbstractInvokable parentTask = new DummyInvokable(); - - private IOManager ioManager; - private MemoryManager memoryManager; - - private TypeSerializer serializer1; - private TypeSerializer serializer2; - private TypeComparator comparator1; - private TypeComparator comparator2; - private TypePairComparator pairComparator; - - - @SuppressWarnings("unchecked") - @Before - public void beforeTest() { - this.serializer1 = RecordSerializer.get(); - this.serializer2 = RecordSerializer.get(); - this.comparator1 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); - this.comparator2 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); - this.pairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[]{TestData.Key.class}); - - this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); - this.ioManager = new IOManagerAsync(); - } - - @After - public void afterTest() { - if (this.ioManager != null) { - this.ioManager.shutdown(); - if (!this.ioManager.isProperlyShutDown()) { - Assert.fail("I/O manager failed to properly shut down."); - } - this.ioManager = null; - } - - if (this.memoryManager != null) { - Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", - this.memoryManager.verifyEmpty()); - this.memoryManager.shutdown(); - this.memoryManager = null; - } - } - - - - @Test - public void testMerge() { - try { - - final TestData.Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - final TestData.Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - - final TestData.GeneratorIterator input1 = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); - final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); - - // collect expected data - final Map> expectedMatchesMap = matchValues( - collectData(input1), - collectData(input2)); - - final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); - - final Collector collector = new DiscardingOutputCollector(); - - // reset the generators - generator1.reset(); - generator2.reset(); - input1.reset(); - input2.reset(); - - // compare with iterator values - NonReusingMergeMatchIterator iterator = - new NonReusingMergeMatchIterator( - input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, - this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); - - iterator.open(); - - while (iterator.callWithNextKey(matcher, collector)); - - iterator.close(); - - // assert that each expected match was seen - for (Entry> entry : expectedMatchesMap.entrySet()) { - Assert.assertTrue("Collection for key " + entry.getKey() + " is not empty", entry.getValue().isEmpty()); - } - } - catch (Exception e) { - e.printStackTrace(); - Assert.fail("An exception occurred during the test: " + e.getMessage()); - } - } - - @Test - public void testMergeWithHighNumberOfCommonKeys() - { - // the size of the left and right inputs - final int INPUT_1_SIZE = 200; - final int INPUT_2_SIZE = 100; - - final int INPUT_1_DUPLICATES = 10; - final int INPUT_2_DUPLICATES = 4000; - final int DUPLICATE_KEY = 13; - - try { - final TestData.Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - final TestData.Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - - final TestData.GeneratorIterator gen1Iter = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); - final TestData.GeneratorIterator gen2Iter = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); - - final TestData.ConstantValueIterator const1Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); - final TestData.ConstantValueIterator const2Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); - - final List> inList1 = new ArrayList>(); - inList1.add(gen1Iter); - inList1.add(const1Iter); - - final List> inList2 = new ArrayList>(); - inList2.add(gen2Iter); - inList2.add(const2Iter); - - MutableObjectIterator input1 = new MergeIterator(inList1, comparator1.duplicate()); - MutableObjectIterator input2 = new MergeIterator(inList2, comparator2.duplicate()); - - // collect expected data - final Map> expectedMatchesMap = matchValues( - collectData(input1), - collectData(input2)); - - // re-create the whole thing for actual processing - - // reset the generators and iterators - generator1.reset(); - generator2.reset(); - const1Iter.reset(); - const2Iter.reset(); - gen1Iter.reset(); - gen2Iter.reset(); - - inList1.clear(); - inList1.add(gen1Iter); - inList1.add(const1Iter); - - inList2.clear(); - inList2.add(gen2Iter); - inList2.add(const2Iter); - - input1 = new MergeIterator(inList1, comparator1.duplicate()); - input2 = new MergeIterator(inList2, comparator2.duplicate()); - - final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); - - final Collector collector = new DiscardingOutputCollector(); - - - // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it - // needs to spill for the duplicate keys - NonReusingMergeMatchIterator iterator = - new NonReusingMergeMatchIterator( - input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, - this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); - - iterator.open(); - - while (iterator.callWithNextKey(matcher, collector)); - - iterator.close(); - - // assert that each expected match was seen - for (Entry> entry : expectedMatchesMap.entrySet()) { - if (!entry.getValue().isEmpty()) { - Assert.fail("Collection for key " + entry.getKey() + " is not empty"); - } - } - } - catch (Exception e) { - e.printStackTrace(); - Assert.fail("An exception occurred during the test: " + e.getMessage()); - } - } - - - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private Map> matchValues( - Map> leftMap, - Map> rightMap) - { - Map> map = new HashMap>(); - - for (TestData.Key key : leftMap.keySet()) { - Collection leftValues = leftMap.get(key); - Collection rightValues = rightMap.get(key); - - if (rightValues == null) { - continue; - } - - if (!map.containsKey(key)) { - map.put(key, new ArrayList()); - } - - Collection matchedValues = map.get(key); - - for (TestData.Value leftValue : leftValues) { - for (TestData.Value rightValue : rightValues) { - matchedValues.add(new Match(leftValue, rightValue)); - } - } - } - - return map; - } - - - private Map> collectData(MutableObjectIterator iter) - throws Exception - { - Map> map = new HashMap>(); - Record pair = new Record(); - - while ((pair = iter.next(pair)) != null) { - TestData.Key key = pair.getField(0, TestData.Key.class); - - if (!map.containsKey(key)) { - map.put(new TestData.Key(key.getKey()), new ArrayList()); - } - - Collection values = map.get(key); - values.add(new TestData.Value(pair.getField(1, TestData.Value.class).getValue())); - } - - return map; - } - - /** - * Private class used for storage of the expected matches in a hashmap. - */ - private static class Match { - private final Value left; - - private final Value right; - - public Match(Value left, Value right) { - this.left = left; - this.right = right; - } - - @Override - public boolean equals(Object obj) { - Match o = (Match) obj; - return this.left.equals(o.left) && this.right.equals(o.right); - } - - @Override - public int hashCode() { - return this.left.hashCode() ^ this.right.hashCode(); - } - - @Override - public String toString() { - return left + ", " + right; - } - } - - private static final class MatchRemovingMatcher extends JoinFunction { - private static final long serialVersionUID = 1L; - - private final Map> toRemoveFrom; - - protected MatchRemovingMatcher(Map> map) { - this.toRemoveFrom = map; - } - - @Override - public void join(Record rec1, Record rec2, Collector out) throws Exception { - TestData.Key key = rec1.getField(0, TestData.Key.class); - TestData.Value value1 = rec1.getField(1, TestData.Value.class); - TestData.Value value2 = rec2.getField(1, TestData.Value.class); - - Collection matches = this.toRemoveFrom.get(key); - if (matches == null) { - Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected."); - } - - boolean contained = matches.remove(new Match(value1, value2)); - if (!contained) { - Assert.fail("Produced match was not contained: " + key + " - " + value1 + ":" + value2); - } - if (matches.isEmpty()) { - this.toRemoveFrom.remove(key); - } - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java new file mode 100644 index 0000000000000..e4eec86acba56 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java @@ -0,0 +1,318 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.GenericPairComparator; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntComparator; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.TupleComparator; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.*; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.*; +import java.util.Map.Entry; + +@SuppressWarnings("deprecation") +public class ReusingSortMergeInnerJoinIteratorITCase { + + // total memory + private static final int MEMORY_SIZE = 1024 * 1024 * 16; + private static final int PAGES_FOR_BNLJN = 2; + + // the size of the left and right inputs + private static final int INPUT_1_SIZE = 20000; + + private static final int INPUT_2_SIZE = 1000; + + // random seeds for the left and right input data generators + private static final long SEED1 = 561349061987311L; + + private static final long SEED2 = 231434613412342L; + + // dummy abstract task + private final AbstractInvokable parentTask = new DummyInvokable(); + + private IOManager ioManager; + private MemoryManager memoryManager; + + private TypeSerializer> serializer1; + private TypeSerializer> serializer2; + private TypeComparator> comparator1; + private TypeComparator> comparator2; + private TypePairComparator, Tuple2> pairComparator; + + + @SuppressWarnings("unchecked") + @Before + public void beforeTest() { + serializer1 = new TupleSerializer>( + (Class>) (Class) Tuple2.class, + new TypeSerializer[] { IntSerializer.INSTANCE, StringSerializer.INSTANCE }); + serializer2 = new TupleSerializer>( + (Class>) (Class) Tuple2.class, + new TypeSerializer[] { IntSerializer.INSTANCE, StringSerializer.INSTANCE }); + comparator1 = new TupleComparator>( + new int[]{0}, + new TypeComparator[] { new IntComparator(true) }, + new TypeSerializer[] { IntSerializer.INSTANCE }); + comparator2 = new TupleComparator>( + new int[]{0}, + new TypeComparator[] { new IntComparator(true) }, + new TypeSerializer[] { IntSerializer.INSTANCE }); + pairComparator = new GenericPairComparator, Tuple2>(comparator1, comparator2); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.ioManager = new IOManagerAsync(); + } + + @After + public void afterTest() { + if (this.ioManager != null) { + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + Assert.fail("I/O manager failed to properly shut down."); + } + this.ioManager = null; + } + + if (this.memoryManager != null) { + Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", + this.memoryManager.verifyEmpty()); + this.memoryManager.shutdown(); + this.memoryManager = null; + } + } + + @Test + public void testMerge() { + try { + + final TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchValues( + collectData(input1), + collectData(input2)); + + final FlatJoinFunction, Tuple2, Tuple2> joinFunction = + new MatchRemovingMatcher(expectedMatchesMap); + + final Collector> collector = new DiscardingOutputCollector>(); + + // reset the generators + generator1.reset(); + generator2.reset(); + input1.reset(); + input2.reset(); + + // compare with iterator values + ReusingMergeInnerJoinIterator, Tuple2, Tuple2> iterator = + new ReusingMergeInnerJoinIterator, Tuple2, Tuple2>( + input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(joinFunction, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + Assert.assertTrue("Collection for key " + entry.getKey() + " is not empty", entry.getValue().isEmpty()); + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testMergeWithHighNumberOfCommonKeys() + { + // the size of the left and right inputs + final int INPUT_1_SIZE = 200; + final int INPUT_2_SIZE = 100; + + final int INPUT_1_DUPLICATES = 10; + final int INPUT_2_DUPLICATES = 4000; + final int DUPLICATE_KEY = 13; + + try { + final TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE); + + final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); + final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); + + final List>> inList1 = new ArrayList>>(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + final List>> inList2 = new ArrayList>>(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + MutableObjectIterator> input1 = new MergeIterator>(inList1, comparator1.duplicate()); + MutableObjectIterator> input2 = new MergeIterator>(inList2, comparator2.duplicate()); + + // collect expected data + final Map> expectedMatchesMap = matchValues( + collectData(input1), + collectData(input2)); + + // re-create the whole thing for actual processing + + // reset the generators and iterators + generator1.reset(); + generator2.reset(); + const1Iter.reset(); + const2Iter.reset(); + gen1Iter.reset(); + gen2Iter.reset(); + + inList1.clear(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + inList2.clear(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + input1 = new MergeIterator>(inList1, comparator1.duplicate()); + input2 = new MergeIterator>(inList2, comparator2.duplicate()); + + final FlatJoinFunction, Tuple2, Tuple2> matcher = new MatchRemovingMatcher(expectedMatchesMap); + + final Collector> collector = new DiscardingOutputCollector>(); + + + // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it + // needs to spill for the duplicate keys + ReusingMergeInnerJoinIterator, Tuple2, Tuple2> iterator = + new ReusingMergeInnerJoinIterator, Tuple2, Tuple2>( + input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private Map> matchValues( + Map> leftMap, + Map> rightMap) + { + Map> map = new HashMap>(); + + for (Integer key : leftMap.keySet()) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + + if (rightValues == null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection matchedValues = map.get(key); + + for (String leftValue : leftValues) { + for (String rightValue : rightValues) { + matchedValues.add(new Match(leftValue, rightValue)); + } + } + } + + return map; + } + + + private Map> collectData(MutableObjectIterator> iter) + throws Exception + { + Map> map = new HashMap>(); + Tuple2 pair = new Tuple2(); + + while ((pair = iter.next(pair)) != null) { + final Integer key = pair.getField(0); + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection values = map.get(key); + final String value = pair.getField(1); + values.add(value); + } + + return map; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java deleted file mode 100644 index 474fa3cf63997..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java +++ /dev/null @@ -1,371 +0,0 @@ -/* - * 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.runtime.operators.sort; - -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypePairComparator; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.record.RecordComparator; -import org.apache.flink.api.common.typeutils.record.RecordPairComparator; -import org.apache.flink.api.common.typeutils.record.RecordSerializer; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; -import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; -import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.apache.flink.runtime.operators.testutils.TestData; -import org.apache.flink.runtime.operators.testutils.TestData.Generator; -import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; -import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; -import org.apache.flink.types.Record; -import org.apache.flink.types.Value; -import org.apache.flink.util.Collector; -import org.apache.flink.util.MutableObjectIterator; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -@SuppressWarnings("deprecation") -public class ReusingSortMergeMatchIteratorITCase { - - // total memory - private static final int MEMORY_SIZE = 1024 * 1024 * 16; - private static final int PAGES_FOR_BNLJN = 2; - - // the size of the left and right inputs - private static final int INPUT_1_SIZE = 20000; - - private static final int INPUT_2_SIZE = 1000; - - // random seeds for the left and right input data generators - private static final long SEED1 = 561349061987311L; - - private static final long SEED2 = 231434613412342L; - - // dummy abstract task - private final AbstractInvokable parentTask = new DummyInvokable(); - - private IOManager ioManager; - private MemoryManager memoryManager; - - private TypeSerializer serializer1; - private TypeSerializer serializer2; - private TypeComparator comparator1; - private TypeComparator comparator2; - private TypePairComparator pairComparator; - - - @SuppressWarnings("unchecked") - @Before - public void beforeTest() { - this.serializer1 = RecordSerializer.get(); - this.serializer2 = RecordSerializer.get(); - this.comparator1 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); - this.comparator2 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); - this.pairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[]{TestData.Key.class}); - - this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); - this.ioManager = new IOManagerAsync(); - } - - @After - public void afterTest() { - if (this.ioManager != null) { - this.ioManager.shutdown(); - if (!this.ioManager.isProperlyShutDown()) { - Assert.fail("I/O manager failed to properly shut down."); - } - this.ioManager = null; - } - - if (this.memoryManager != null) { - Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", - this.memoryManager.verifyEmpty()); - this.memoryManager.shutdown(); - this.memoryManager = null; - } - } - - - - @Test - public void testMerge() { - try { - - final Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - final Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - - final TestData.GeneratorIterator input1 = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); - final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); - - // collect expected data - final Map> expectedMatchesMap = matchValues( - collectData(input1), - collectData(input2)); - - final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); - - final Collector collector = new DiscardingOutputCollector(); - - // reset the generators - generator1.reset(); - generator2.reset(); - input1.reset(); - input2.reset(); - - // compare with iterator values - ReusingMergeMatchIterator iterator = - new ReusingMergeMatchIterator( - input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, - this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); - - iterator.open(); - - while (iterator.callWithNextKey(matcher, collector)); - - iterator.close(); - - // assert that each expected match was seen - for (Entry> entry : expectedMatchesMap.entrySet()) { - Assert.assertTrue("Collection for key " + entry.getKey() + " is not empty", entry.getValue().isEmpty()); - } - } - catch (Exception e) { - e.printStackTrace(); - Assert.fail("An exception occurred during the test: " + e.getMessage()); - } - } - - @Test - public void testMergeWithHighNumberOfCommonKeys() - { - // the size of the left and right inputs - final int INPUT_1_SIZE = 200; - final int INPUT_2_SIZE = 100; - - final int INPUT_1_DUPLICATES = 10; - final int INPUT_2_DUPLICATES = 4000; - final int DUPLICATE_KEY = 13; - - try { - final Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - final Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); - - final TestData.GeneratorIterator gen1Iter = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); - final TestData.GeneratorIterator gen2Iter = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); - - final TestData.ConstantValueIterator const1Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); - final TestData.ConstantValueIterator const2Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); - - final List> inList1 = new ArrayList>(); - inList1.add(gen1Iter); - inList1.add(const1Iter); - - final List> inList2 = new ArrayList>(); - inList2.add(gen2Iter); - inList2.add(const2Iter); - - MutableObjectIterator input1 = new MergeIterator(inList1, comparator1.duplicate()); - MutableObjectIterator input2 = new MergeIterator(inList2, comparator2.duplicate()); - - // collect expected data - final Map> expectedMatchesMap = matchValues( - collectData(input1), - collectData(input2)); - - // re-create the whole thing for actual processing - - // reset the generators and iterators - generator1.reset(); - generator2.reset(); - const1Iter.reset(); - const2Iter.reset(); - gen1Iter.reset(); - gen2Iter.reset(); - - inList1.clear(); - inList1.add(gen1Iter); - inList1.add(const1Iter); - - inList2.clear(); - inList2.add(gen2Iter); - inList2.add(const2Iter); - - input1 = new MergeIterator(inList1, comparator1.duplicate()); - input2 = new MergeIterator(inList2, comparator2.duplicate()); - - final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); - - final Collector collector = new DiscardingOutputCollector(); - - - // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it - // needs to spill for the duplicate keys - ReusingMergeMatchIterator iterator = - new ReusingMergeMatchIterator( - input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, - this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); - - iterator.open(); - - while (iterator.callWithNextKey(matcher, collector)); - - iterator.close(); - - // assert that each expected match was seen - for (Entry> entry : expectedMatchesMap.entrySet()) { - if (!entry.getValue().isEmpty()) { - Assert.fail("Collection for key " + entry.getKey() + " is not empty"); - } - } - } - catch (Exception e) { - e.printStackTrace(); - Assert.fail("An exception occurred during the test: " + e.getMessage()); - } - } - - - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private Map> matchValues( - Map> leftMap, - Map> rightMap) - { - Map> map = new HashMap>(); - - for (TestData.Key key : leftMap.keySet()) { - Collection leftValues = leftMap.get(key); - Collection rightValues = rightMap.get(key); - - if (rightValues == null) { - continue; - } - - if (!map.containsKey(key)) { - map.put(key, new ArrayList()); - } - - Collection matchedValues = map.get(key); - - for (TestData.Value leftValue : leftValues) { - for (TestData.Value rightValue : rightValues) { - matchedValues.add(new Match(leftValue, rightValue)); - } - } - } - - return map; - } - - - private Map> collectData(MutableObjectIterator iter) - throws Exception - { - Map> map = new HashMap>(); - Record pair = new Record(); - - while ((pair = iter.next(pair)) != null) { - TestData.Key key = pair.getField(0, TestData.Key.class); - - if (!map.containsKey(key)) { - map.put(new TestData.Key(key.getKey()), new ArrayList()); - } - - Collection values = map.get(key); - values.add(new TestData.Value(pair.getField(1, TestData.Value.class).getValue())); - } - - return map; - } - - /** - * Private class used for storage of the expected matches in a hashmap. - */ - private static class Match { - private final Value left; - - private final Value right; - - public Match(Value left, Value right) { - this.left = left; - this.right = right; - } - - @Override - public boolean equals(Object obj) { - Match o = (Match) obj; - return this.left.equals(o.left) && this.right.equals(o.right); - } - - @Override - public int hashCode() { - return this.left.hashCode() ^ this.right.hashCode(); - } - - @Override - public String toString() { - return left + ", " + right; - } - } - - private static final class MatchRemovingMatcher extends JoinFunction { - private static final long serialVersionUID = 1L; - - private final Map> toRemoveFrom; - - protected MatchRemovingMatcher(Map> map) { - this.toRemoveFrom = map; - } - - @Override - public void join(Record rec1, Record rec2, Collector out) throws Exception { - TestData.Key key = rec1.getField(0, TestData.Key.class); - TestData.Value value1 = rec1.getField(1, TestData.Value.class); - TestData.Value value2 = rec2.getField(1, TestData.Value.class); - - Collection matches = this.toRemoveFrom.get(key); - if (matches == null) { - Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected."); - } - - boolean contained = matches.remove(new Match(value1, value2)); - if (!contained) { - Assert.fail("Produced match was not contained: " + key + " - " + value1 + ":" + value2); - } - if (matches.isEmpty()) { - this.toRemoveFrom.remove(key); - } - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/CollectionIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/CollectionIterator.java new file mode 100644 index 0000000000000..7fd1b6c21776f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/CollectionIterator.java @@ -0,0 +1,61 @@ +/* + * 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.runtime.operators.testutils; + +import org.apache.flink.runtime.util.ResettableMutableObjectIterator; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; + + +public class CollectionIterator implements ResettableMutableObjectIterator { + + private final Collection collection; + private Iterator iterator; + + public CollectionIterator(Collection collection) { + this.collection = collection; + this.iterator = collection.iterator(); + } + + @Override + public T next(T reuse) throws IOException { + return next(); + } + + @Override + public T next() throws IOException { + if (!iterator.hasNext()) { + return null; + } else { + return iterator.next(); + } + } + + @Override + public void reset() throws IOException { + iterator = collection.iterator(); + } + + public static CollectionIterator of(T... values) { + return new CollectionIterator(Arrays.asList(values)); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java new file mode 100644 index 0000000000000..539d8648097ca --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java @@ -0,0 +1,63 @@ +/* + * 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.runtime.operators.testutils; + +/** + * Utility class for keeping track of matches in join operator tests. + * + * @see org.apache.flink.runtime.operators.testutils.MatchRemovingMatcher + */ +public class Match { + private final String left; + + private final String right; + + public Match(String left, String right) { + this.left = left; + this.right = right; + } + + @Override + public boolean equals(Object obj) { + Match o = (Match) obj; + if (left == null && o.left == null && right.equals(o.right)) { + return true; + } else if (right == null && o.right == null && left.equals(o.left)) { + return true; + } else { + return this.left.equals(o.left) && this.right.equals(o.right); + } + } + + @Override + public int hashCode() { + if (left == null) { + return right.hashCode(); + } else if (right == null) { + return left.hashCode(); + } else { + return this.left.hashCode() ^ this.right.hashCode(); + } + } + + @Override + public String toString() { + return left + ", " + right; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java new file mode 100644 index 0000000000000..f69b4d74f762c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java @@ -0,0 +1,58 @@ +/* + * 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.runtime.operators.testutils; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.Collector; +import org.junit.Assert; + +import java.util.Collection; +import java.util.Map; + + +public final class MatchRemovingMatcher implements FlatJoinFunction,Tuple2,Tuple2> { + private static final long serialVersionUID = 1L; + + private final Map> toRemoveFrom; + + public MatchRemovingMatcher(Map> map) { + this.toRemoveFrom = map; + } + + @Override + public void join(Tuple2 rec1, Tuple2 rec2, Collector> out) throws Exception { + final Integer key = rec1 != null ? (Integer) rec1.getField(0) : (Integer) rec2.getField(0); + final String value1 = rec1 != null ? (String) rec1.getField(1) : null; + final String value2 = rec2 != null ? (String) rec2.getField(1) : null; + + Collection matches = this.toRemoveFrom.get(key); + if (matches == null) { + Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected."); + } + + boolean contained = matches.remove(new Match(value1, value2)); + if (!contained) { + Assert.fail("Produced match was not contained: " + key + " - " + value1 + ":" + value2); + } + if (matches.isEmpty()) { + this.toRemoveFrom.remove(key); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/SimpleTupleJoinFunction.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/SimpleTupleJoinFunction.java new file mode 100644 index 0000000000000..06a62e56e3c17 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/SimpleTupleJoinFunction.java @@ -0,0 +1,41 @@ +/* + * 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.runtime.operators.testutils; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.util.Collector; + +/** + * Simple flat join function that joins two binary tuples and considers null cases. + */ +public class SimpleTupleJoinFunction implements FlatJoinFunction, Tuple2, Tuple4> { + + @Override + public void join(Tuple2 first, Tuple2 second, Collector> out) throws Exception { + if (first == null) { + out.collect(new Tuple4(null, null, second.f0, second.f1)); + } else if (second == null) { + out.collect(new Tuple4(first.f0, first.f1, null, null)); + } else { + out.collect(new Tuple4(first.f0, first.f1, second.f0, second.f1)); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java index 5d1ce7f4ac4e0..38d999285c572 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; -import org.apache.flink.runtime.operators.sort.ReusingMergeMatchIterator; +import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; @@ -143,8 +143,8 @@ public void testSortBothMerge() { final MutableObjectIterator sortedInput2 = sorter2.getIterator(); // compare with iterator values - ReusingMergeMatchIterator iterator = - new ReusingMergeMatchIterator(sortedInput1, sortedInput2, + ReusingMergeInnerJoinIterator iterator = + new ReusingMergeInnerJoinIterator(sortedInput1, sortedInput2, this.serializer1.getSerializer(), this.comparator1, this.serializer2.getSerializer(), this.comparator2, this.pairComparator11, this.memoryManager, this.ioManager, MEMORY_PAGES_FOR_MERGE, this.parentTask); From 941ac6dfd446d8e97e2fe2f589164978602adf94 Mon Sep 17 00:00:00 2001 From: r-pogalz Date: Mon, 3 Aug 2015 12:59:48 +0200 Subject: [PATCH 079/175] [FLINK-2105] Implement Sort-Merge Outer Join algorithm This closes #907 --- .../operators/sort/AbstractMergeIterator.java | 58 +-- .../sort/AbstractMergeOuterJoinIterator.java | 189 +++++++ .../NonReusingMergeOuterJoinIterator.java | 60 +++ .../sort/ReusingMergeOuterJoinIterator.java | 63 +++ ...tractSortMergeOuterJoinIteratorITCase.java | 462 ++++++++++++++++++ ...usingSortMergeInnerJoinIteratorITCase.java | 4 +- ...usingSortMergeOuterJoinIteratorITCase.java | 82 ++++ ...usingSortMergeInnerJoinIteratorITCase.java | 4 +- ...usingSortMergeOuterJoinIteratorITCase.java | 82 ++++ .../runtime/operators/testutils/Match.java | 2 +- ...gMatcher.java => MatchRemovingJoiner.java} | 4 +- 11 files changed, 974 insertions(+), 36 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java rename flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/{MatchRemovingMatcher.java => MatchRemovingJoiner.java} (90%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java index 9a61c140660c3..c01afc7411342 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java @@ -115,20 +115,20 @@ public void abort() { } /** - * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come - * from different inputs. The output of the match() method is forwarded. + * Calls the JoinFunction#join() method for all two key-value pairs that share the same key and come + * from different inputs. The output of the join() method is forwarded. *

    * This method first zig-zags between the two sorted inputs in order to find a common - * key, and then calls the match stub with the cross product of the values. + * key, and then calls the join stub with the cross product of the values. * * @throws Exception Forwards all exceptions from the user code and the I/O system. * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) */ @Override - public abstract boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) + public abstract boolean callWithNextKey(final FlatJoinFunction joinFunction, final Collector collector) throws Exception; - protected void crossMatchingGroup(Iterator values1, Iterator values2, FlatJoinFunction matchFunction, Collector collector) throws Exception { + protected void crossMatchingGroup(Iterator values1, Iterator values2, FlatJoinFunction joinFunction, Collector collector) throws Exception { final T1 firstV1 = values1.next(); final T2 firstV2 = values2.next(); @@ -143,23 +143,23 @@ protected void crossMatchingGroup(Iterator values1, Iterator values2, Fl if (v2HasNext) { // both sides contain more than one value // TODO: Decide which side to spill and which to block! - crossMwithNValues(firstV1, values1, firstV2, values2, matchFunction, collector); + crossMwithNValues(firstV1, values1, firstV2, values2, joinFunction, collector); } else { - crossSecond1withNValues(firstV2, firstV1, values1, matchFunction, collector); + crossSecond1withNValues(firstV2, firstV1, values1, joinFunction, collector); } } else { if (v2HasNext) { - crossFirst1withNValues(firstV1, firstV2, values2, matchFunction, collector); + crossFirst1withNValues(firstV1, firstV2, values2, joinFunction, collector); } else { // both sides contain only one value - matchFunction.join(firstV1, firstV2, collector); + joinFunction.join(firstV1, firstV2, collector); } } } /** * Crosses a single value from the first input with N values, all sharing a common key. - * Effectively realizes a 1:N match (join). + * Effectively realizes a 1:N join. * * @param val1 The value form the 1 side. * @param firstValN The first of the values from the N side. @@ -167,21 +167,21 @@ protected void crossMatchingGroup(Iterator values1, Iterator values2, Fl * @throws Exception Forwards all exceptions thrown by the stub. */ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, - final Iterator valsN, final FlatJoinFunction matchFunction, final Collector collector) + final Iterator valsN, final FlatJoinFunction joinFunction, final Collector collector) throws Exception { T1 copy1 = createCopy(serializer1, val1, this.copy1); - matchFunction.join(copy1, firstValN, collector); + joinFunction.join(copy1, firstValN, collector); - // set copy and match first element + // set copy and join first element boolean more = true; do { final T2 nRec = valsN.next(); if (valsN.hasNext()) { copy1 = createCopy(serializer1, val1, this.copy1); - matchFunction.join(copy1, nRec, collector); + joinFunction.join(copy1, nRec, collector); } else { - matchFunction.join(val1, nRec, collector); + joinFunction.join(val1, nRec, collector); more = false; } } @@ -190,7 +190,7 @@ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, /** * Crosses a single value from the second side with N values, all sharing a common key. - * Effectively realizes a N:1 match (join). + * Effectively realizes a N:1 join. * * @param val1 The value form the 1 side. * @param firstValN The first of the values from the N side. @@ -198,20 +198,20 @@ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, * @throws Exception Forwards all exceptions thrown by the stub. */ private void crossSecond1withNValues(T2 val1, T1 firstValN, - Iterator valsN, FlatJoinFunction matchFunction, Collector collector) throws Exception { + Iterator valsN, FlatJoinFunction joinFunction, Collector collector) throws Exception { T2 copy2 = createCopy(serializer2, val1, this.copy2); - matchFunction.join(firstValN, copy2, collector); + joinFunction.join(firstValN, copy2, collector); - // set copy and match first element + // set copy and join first element boolean more = true; do { final T1 nRec = valsN.next(); if (valsN.hasNext()) { copy2 = createCopy(serializer2, val1, this.copy2); - matchFunction.join(nRec, copy2, collector); + joinFunction.join(nRec, copy2, collector); } else { - matchFunction.join(nRec, val1, collector); + joinFunction.join(nRec, val1, collector); more = false; } } @@ -220,7 +220,7 @@ private void crossSecond1withNValues(T2 val1, T1 firstValN, private void crossMwithNValues(final T1 firstV1, Iterator spillVals, final T2 firstV2, final Iterator blockVals, - final FlatJoinFunction matchFunction, final Collector collector) throws Exception { + final FlatJoinFunction joinFunction, final Collector collector) throws Exception { // ================================================== // We have one first (head) element from both inputs (firstV1 and firstV2) // We have an iterator for both inputs. @@ -237,13 +237,13 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, // 5) cross the head of the spilling side with the next block // 6) cross the spilling iterator with the next block. - // match the first values first + // join the first values first T1 copy1 = this.createCopy(serializer1, firstV1, this.copy1); T2 blockHeadCopy = this.createCopy(serializer2, firstV2, this.blockHeadCopy); T1 spillHeadCopy = null; // --------------- 1) Cross the heads ------------------- - matchFunction.join(copy1, firstV2, collector); + joinFunction.join(copy1, firstV2, collector); // for the remaining values, we do a block-nested-loops join SpillingResettableIterator spillIt = null; @@ -256,7 +256,7 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, while (this.blockIt.hasNext()) { final T2 nextBlockRec = this.blockIt.next(); copy1 = this.createCopy(serializer1, firstV1, this.copy1); - matchFunction.join(copy1, nextBlockRec, collector); + joinFunction.join(copy1, nextBlockRec, collector); } this.blockIt.reset(); @@ -286,7 +286,7 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, // -------- 3) cross the iterator of the spilling side with the head of the block side -------- T2 copy2 = this.createCopy(serializer2, blockHeadCopy, this.copy2); - matchFunction.join(copy1, copy2, collector); + joinFunction.join(copy1, copy2, collector); // -------- 4) cross the iterator of the spilling side with the first block -------- while (this.blockIt.hasNext()) { @@ -294,7 +294,7 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, // get instances of key and block value copy1 = this.createCopy(serializer1, nextSpillVal, this.copy1); - matchFunction.join(copy1, nextBlockRec, collector); + joinFunction.join(copy1, nextBlockRec, collector); } // reset block iterator this.blockIt.reset(); @@ -316,7 +316,7 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, while (this.blockIt.hasNext()) { copy1 = this.createCopy(serializer1, spillHeadCopy, this.copy1); final T2 nextBlockVal = blockIt.next(); - matchFunction.join(copy1, nextBlockVal, collector); + joinFunction.join(copy1, nextBlockVal, collector); } this.blockIt.reset(); @@ -329,7 +329,7 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, // get instances of key and block value final T2 nextBlockVal = this.blockIt.next(); copy1 = this.createCopy(serializer1, nextSpillVal, this.copy1); - matchFunction.join(copy1, nextBlockVal, collector); + joinFunction.join(copy1, nextBlockVal, collector); } // reset block iterator diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java new file mode 100644 index 0000000000000..01b371e0660df --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java @@ -0,0 +1,189 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; + +import java.util.Iterator; + +/** + * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the + * outer join through a sort-merge join strategy. + */ +public abstract class AbstractMergeOuterJoinIterator extends AbstractMergeIterator { + + public static enum OuterJoinType {LEFT, RIGHT, FULL} + + private final OuterJoinType outerJoinType; + + private boolean initialized = false; + private boolean it1Empty = false; + private boolean it2Empty = false; + + + public AbstractMergeOuterJoinIterator( + OuterJoinType outerJoinType, + MutableObjectIterator input1, + MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) + throws MemoryAllocationException { + super(input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + + this.outerJoinType = outerJoinType; + } + + /** + * Calls the JoinFunction#join() method for all two key-value pairs that share the same key and come + * from different inputs. Furthermore, depending on the outer join type (LEFT, RIGHT, FULL), all key-value pairs where no + * matching partner from the other input exists are joined with null. + * The output of the join() method is forwarded. + * + * @throws Exception Forwards all exceptions from the user code and the I/O system. + * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) + */ + @Override + public boolean callWithNextKey(final FlatJoinFunction joinFunction, final Collector collector) throws Exception { + if (!initialized) { + //first run, set iterators to first elements + it1Empty = !this.iterator1.nextKey(); + it2Empty = !this.iterator2.nextKey(); + initialized = true; + } + + if (it1Empty && it2Empty) { + return false; + } else if (it2Empty) { + if (outerJoinType == OuterJoinType.LEFT || outerJoinType == OuterJoinType.FULL) { + joinLeftKeyValuesWithNull(iterator1.getValues(), joinFunction, collector); + it1Empty = !iterator1.nextKey(); + return true; + } else { + //consume rest of left side + while (iterator1.nextKey()) ; + it1Empty = true; + return false; + } + } else if (it1Empty) { + if (outerJoinType == OuterJoinType.RIGHT || outerJoinType == OuterJoinType.FULL) { + joinRightKeyValuesWithNull(iterator2.getValues(), joinFunction, collector); + it2Empty = !iterator2.nextKey(); + return true; + } else { + //consume rest of right side + while (iterator2.nextKey()) ; + it2Empty = true; + return false; + } + } else { + final TypePairComparator comparator = super.pairComparator; + comparator.setReference(this.iterator1.getCurrent()); + T2 current2 = this.iterator2.getCurrent(); + + // zig zag + while (true) { + // determine the relation between the (possibly composite) keys + final int comp = comparator.compareToReference(current2); + + if (comp == 0) { + break; + } + + if (comp < 0) { + //right key < left key + if (outerJoinType == OuterJoinType.RIGHT || outerJoinType == OuterJoinType.FULL) { + //join right key values with null in case of right or full outer join + joinRightKeyValuesWithNull(iterator2.getValues(), joinFunction, collector); + it2Empty = !iterator2.nextKey(); + return true; + } else { + //skip this right key if it is a left outer join + if (!this.iterator2.nextKey()) { + //if right side is empty, join current left key values with null + joinLeftKeyValuesWithNull(iterator1.getValues(), joinFunction, collector); + it1Empty = !iterator1.nextKey(); + it2Empty = true; + return true; + } + current2 = this.iterator2.getCurrent(); + } + } else { + //right key > left key + if (outerJoinType == OuterJoinType.LEFT || outerJoinType == OuterJoinType.FULL) { + //join left key values with null in case of left or full outer join + joinLeftKeyValuesWithNull(iterator1.getValues(), joinFunction, collector); + it1Empty = !iterator1.nextKey(); + return true; + } else { + //skip this left key if it is a right outer join + if (!this.iterator1.nextKey()) { + //if right side is empty, join current right key values with null + joinRightKeyValuesWithNull(iterator2.getValues(), joinFunction, collector); + it1Empty = true; + it2Empty = !iterator2.nextKey(); + return true; + } + comparator.setReference(this.iterator1.getCurrent()); + } + } + } + + // here, we have a common key! call the join function with the cross product of the + // values + final Iterator values1 = this.iterator1.getValues(); + final Iterator values2 = this.iterator2.getValues(); + + crossMatchingGroup(values1, values2, joinFunction, collector); + it1Empty = !iterator1.nextKey(); + it2Empty = !iterator2.nextKey(); + return true; + } + } + + private void joinLeftKeyValuesWithNull(Iterator values, FlatJoinFunction joinFunction, Collector collector) throws Exception { + while (values.hasNext()) { + T1 next = values.next(); + this.copy1 = createCopy(serializer1, next, copy1); + joinFunction.join(copy1, null, collector); + } + } + + private void joinRightKeyValuesWithNull(Iterator values, FlatJoinFunction joinFunction, Collector collector) throws Exception { + while (values.hasNext()) { + T2 next = values.next(); + this.copy2 = createCopy(serializer2, next, copy2); + joinFunction.join(null, copy2, collector); + } + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java new file mode 100644 index 0000000000000..ac49eceaf98b5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java @@ -0,0 +1,60 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; +import org.apache.flink.util.MutableObjectIterator; + +public class NonReusingMergeOuterJoinIterator extends AbstractMergeOuterJoinIterator { + + public NonReusingMergeOuterJoinIterator( + OuterJoinType outerJoinType, + MutableObjectIterator input1, + MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) + throws MemoryAllocationException { + super(outerJoinType, input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + } + + @Override + protected KeyGroupedIterator createKeyGroupedIterator(MutableObjectIterator input, TypeSerializer serializer, TypeComparator comparator) { + return new NonReusingKeyGroupedIterator(input, comparator); + } + + @Override + protected T createCopy(TypeSerializer serializer, T value, T reuse) { + return serializer.copy(value); + } + + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java new file mode 100644 index 0000000000000..0cefbc56232da --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java @@ -0,0 +1,63 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; +import org.apache.flink.util.MutableObjectIterator; + +public class ReusingMergeOuterJoinIterator extends AbstractMergeOuterJoinIterator { + + public ReusingMergeOuterJoinIterator( + OuterJoinType outerJoinType, + MutableObjectIterator input1, + MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) + throws MemoryAllocationException { + super(outerJoinType, input1, input2, serializer1, comparator1, serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + + this.copy1 = serializer1.createInstance(); + this.spillHeadCopy = serializer1.createInstance(); + this.copy2 = serializer2.createInstance(); + this.blockHeadCopy = serializer2.createInstance(); + } + + @Override + protected KeyGroupedIterator createKeyGroupedIterator(MutableObjectIterator input, TypeSerializer serializer, TypeComparator comparator) { + return new ReusingKeyGroupedIterator(input, serializer, comparator); + } + + @Override + protected T createCopy(TypeSerializer serializer, T value, T reuse) { return serializer.copy(value, reuse); } + + +} \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java new file mode 100644 index 0000000000000..1fbe0255296ac --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java @@ -0,0 +1,462 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.util.ListCollector; +import org.apache.flink.api.common.typeutils.GenericPairComparator; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntComparator; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.api.java.typeutils.runtime.TupleComparator; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType; +import org.apache.flink.runtime.operators.testutils.*; +import org.apache.flink.runtime.operators.testutils.TestData.TupleConstantValueIterator; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; +import org.apache.flink.runtime.operators.testutils.TestData.TupleGeneratorIterator; +import org.apache.flink.runtime.util.ResettableMutableObjectIterator; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import java.util.*; +import java.util.Map.Entry; + +public abstract class AbstractSortMergeOuterJoinIteratorITCase { + + // total memory + private static final int MEMORY_SIZE = 1024 * 1024 * 16; + private static final int PAGES_FOR_BNLJN = 2; + + // the size of the left and right inputs + private static final int INPUT_1_SIZE = 20000; + + private static final int INPUT_2_SIZE = 1000; + + // random seeds for the left and right input data generators + private static final long SEED1 = 561349061987311L; + + private static final long SEED2 = 231434613412342L; + + // dummy abstract task + private final AbstractInvokable parentTask = new DummyInvokable(); + + private IOManager ioManager; + private MemoryManager memoryManager; + + private TupleTypeInfo> typeInfo1; + private TupleTypeInfo> typeInfo2; + private TupleSerializer> serializer1; + private TupleSerializer> serializer2; + private TypeComparator> comparator1; + private TypeComparator> comparator2; + private TypePairComparator, Tuple2> pairComp; + + + @Before + public void beforeTest() { + ExecutionConfig config = new ExecutionConfig(); + config.disableObjectReuse(); + + typeInfo1 = TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class); + typeInfo2 = TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class); + serializer1 = typeInfo1.createSerializer(config); + serializer2 = typeInfo2.createSerializer(config); + comparator1 = typeInfo1.createComparator(new int[]{0}, new boolean[]{true}, 0, config); + comparator2 = typeInfo2.createComparator(new int[]{0}, new boolean[]{true}, 0, config); + pairComp = new GenericPairComparator, Tuple2>(comparator1, comparator2); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.ioManager = new IOManagerAsync(); + } + + @After + public void afterTest() { + if (this.ioManager != null) { + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + Assert.fail("I/O manager failed to properly shut down."); + } + this.ioManager = null; + } + + if (this.memoryManager != null) { + Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", + this.memoryManager.verifyEmpty()); + this.memoryManager.shutdown(); + this.memoryManager = null; + } + } + + protected void testFullOuterWithSample() throws Exception { + CollectionIterator> input1 = CollectionIterator.of( + new Tuple2("Jack", "Engineering"), + new Tuple2("Tim", "Sales"), + new Tuple2("Zed", "HR") + ); + CollectionIterator> input2 = CollectionIterator.of( + new Tuple2("Allison", 100), + new Tuple2("Jack", 200), + new Tuple2("Zed", 150), + new Tuple2("Zed", 250) + ); + + OuterJoinType outerJoinType = OuterJoinType.FULL; + List> actual = computeOuterJoin(input1, input2, outerJoinType); + + List> expected = Arrays.asList( + new Tuple4(null, null, "Allison", 100), + new Tuple4("Jack", "Engineering", "Jack", 200), + new Tuple4("Tim", "Sales", null, null), + new Tuple4("Zed", "HR", "Zed", 150), + new Tuple4("Zed", "HR", "Zed", 250) + ); + + Assert.assertEquals(expected, actual); + } + + protected void testLeftOuterWithSample() throws Exception { + CollectionIterator> input1 = CollectionIterator.of( + new Tuple2("Jack", "Engineering"), + new Tuple2("Tim", "Sales"), + new Tuple2("Zed", "HR") + ); + CollectionIterator> input2 = CollectionIterator.of( + new Tuple2("Allison", 100), + new Tuple2("Jack", 200), + new Tuple2("Zed", 150), + new Tuple2("Zed", 250) + ); + + List> actual = computeOuterJoin(input1, input2, OuterJoinType.LEFT); + + List> expected = Arrays.asList( + new Tuple4("Jack", "Engineering", "Jack", 200), + new Tuple4("Tim", "Sales", null, null), + new Tuple4("Zed", "HR", "Zed", 150), + new Tuple4("Zed", "HR", "Zed", 250) + ); + + Assert.assertEquals(expected, actual); + } + + protected void testRightOuterWithSample() throws Exception { + CollectionIterator> input1 = CollectionIterator.of( + new Tuple2("Jack", "Engineering"), + new Tuple2("Tim", "Sales"), + new Tuple2("Zed", "HR") + ); + CollectionIterator> input2 = CollectionIterator.of( + new Tuple2("Allison", 100), + new Tuple2("Jack", 200), + new Tuple2("Zed", 150), + new Tuple2("Zed", 250) + ); + + List> actual = computeOuterJoin(input1, input2, OuterJoinType.RIGHT); + + List> expected = Arrays.asList( + new Tuple4(null, null, "Allison", 100), + new Tuple4("Jack", "Engineering", "Jack", 200), + new Tuple4("Zed", "HR", "Zed", 150), + new Tuple4("Zed", "HR", "Zed", 250) + ); + + Assert.assertEquals(expected, actual); + } + + protected void testRightSideEmpty() throws Exception { + CollectionIterator> input1 = CollectionIterator.of( + new Tuple2("Jack", "Engineering"), + new Tuple2("Tim", "Sales"), + new Tuple2("Zed", "HR") + ); + CollectionIterator> input2 = CollectionIterator.of(); + + List> actualLeft = computeOuterJoin(input1, input2, OuterJoinType.LEFT); + List> actualRight = computeOuterJoin(input1, input2, OuterJoinType.RIGHT); + List> actualFull = computeOuterJoin(input1, input2, OuterJoinType.FULL); + + List> expected = Arrays.asList( + new Tuple4("Jack", "Engineering", null, null), + new Tuple4("Tim", "Sales", null, null), + new Tuple4("Zed", "HR", null, null) + ); + + Assert.assertEquals(expected, actualLeft); + Assert.assertEquals(expected, actualFull); + Assert.assertEquals(Collections.>emptyList(), actualRight); + } + + protected void testLeftSideEmpty() throws Exception { + CollectionIterator> input1 = CollectionIterator.of(); + CollectionIterator> input2 = CollectionIterator.of( + new Tuple2("Allison", 100), + new Tuple2("Jack", 200), + new Tuple2("Zed", 150), + new Tuple2("Zed", 250) + ); + + List> actualLeft = computeOuterJoin(input1, input2, OuterJoinType.LEFT); + List> actualRight = computeOuterJoin(input1, input2, OuterJoinType.RIGHT); + List> actualFull = computeOuterJoin(input1, input2, OuterJoinType.FULL); + + List> expected = Arrays.asList( + new Tuple4(null, null, "Allison", 100), + new Tuple4(null, null, "Jack", 200), + new Tuple4(null, null, "Zed", 150), + new Tuple4(null, null, "Zed", 250) + ); + + Assert.assertEquals(Collections.>emptyList(), actualLeft); + Assert.assertEquals(expected, actualRight); + Assert.assertEquals(expected, actualFull); + } + + private List> computeOuterJoin(ResettableMutableObjectIterator> input1, + ResettableMutableObjectIterator> input2, + OuterJoinType outerJoinType) throws Exception { + input1.reset(); + input2.reset(); + AbstractMergeOuterJoinIterator, Tuple2, Tuple4> iterator = + createOuterJoinIterator(outerJoinType, input1, input2, serializer1, comparator1, serializer2, comparator2, + pairComp, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + List> actual = new ArrayList>(); + ListCollector> collector = new ListCollector>(actual); + while (iterator.callWithNextKey(new SimpleTupleJoinFunction(), collector)) ; + iterator.close(); + + return actual; + } + + protected void testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType outerJoinType, int input1Size, int input1Duplicates, int input1ValueLength, + float input1KeyDensity, int input2Size, int input2Duplicates, int input2ValueLength, float input2KeyDensity) { + TypeSerializer> serializer1 = new TupleSerializer>( + (Class>) (Class) Tuple2.class, + new TypeSerializer[] { IntSerializer.INSTANCE, StringSerializer.INSTANCE }); + TypeSerializer> serializer2 = new TupleSerializer>( + (Class>) (Class) Tuple2.class, + new TypeSerializer[] { IntSerializer.INSTANCE, StringSerializer.INSTANCE }); + TypeComparator> comparator1 = new TupleComparator>( + new int[]{0}, + new TypeComparator[] { new IntComparator(true) }, + new TypeSerializer[] { IntSerializer.INSTANCE }); + TypeComparator> comparator2 = new TupleComparator>( + new int[]{0}, + new TypeComparator[] { new IntComparator(true) }, + new TypeSerializer[] { IntSerializer.INSTANCE }); + + TypePairComparator, Tuple2> pairComparator = + new GenericPairComparator, Tuple2>(comparator1, comparator2); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.ioManager = new IOManagerAsync(); + + final int DUPLICATE_KEY = 13; + + try { + final TupleGenerator generator1 = new TupleGenerator(SEED1, 500, input1KeyDensity, input1ValueLength, KeyMode.SORTED_SPARSE, ValueMode.RANDOM_LENGTH, null); + final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, input2KeyDensity, input2ValueLength, KeyMode.SORTED_SPARSE, ValueMode.RANDOM_LENGTH, null); + + final TupleGeneratorIterator gen1Iter = new TupleGeneratorIterator(generator1, input1Size); + final TupleGeneratorIterator gen2Iter = new TupleGeneratorIterator(generator2, input2Size); + + final TupleConstantValueIterator const1Iter = new TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", input1Duplicates); + final TupleConstantValueIterator const2Iter = new TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", input2Duplicates); + + final List>> inList1 = new ArrayList>>(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + final List>> inList2 = new ArrayList>>(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + MutableObjectIterator> input1 = new MergeIterator>(inList1, comparator1.duplicate()); + MutableObjectIterator> input2 = new MergeIterator>(inList2, comparator2.duplicate()); + + // collect expected data + final Map> expectedMatchesMap = joinValues( + collectData(input1), + collectData(input2), + outerJoinType); + + // re-create the whole thing for actual processing + + // reset the generators and iterators + generator1.reset(); + generator2.reset(); + const1Iter.reset(); + const2Iter.reset(); + gen1Iter.reset(); + gen2Iter.reset(); + + inList1.clear(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + inList2.clear(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + input1 = new MergeIterator>(inList1, comparator1.duplicate()); + input2 = new MergeIterator>(inList2, comparator2.duplicate()); + + final FlatJoinFunction, Tuple2, Tuple2> joinFunction = + new MatchRemovingJoiner(expectedMatchesMap); + + final Collector> collector = new DiscardingOutputCollector>(); + + + // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it + // needs to spill for the duplicate keys + AbstractMergeOuterJoinIterator, Tuple2, Tuple2> iterator = + createOuterJoinIterator( + outerJoinType, input1, input2, serializer1, comparator1, serializer2, comparator2, + pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(joinFunction, collector)) ; + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + protected abstract AbstractMergeOuterJoinIterator createOuterJoinIterator(OuterJoinType outerJoinType, + MutableObjectIterator input1, + MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) throws Exception; + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + + private Map> joinValues( + Map> leftMap, + Map> rightMap, + OuterJoinType outerJoinType) { + Map> map = new HashMap>(); + + for (Integer key : leftMap.keySet()) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + + if (outerJoinType == OuterJoinType.RIGHT && rightValues == null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection joinedValues = map.get(key); + + for (String leftValue : leftValues) { + if (rightValues != null) { + for (String rightValue : rightValues) { + joinedValues.add(new Match(leftValue, rightValue)); + } + } else { + joinedValues.add(new Match(leftValue, null)); + } + } + } + + if (outerJoinType == OuterJoinType.RIGHT || outerJoinType == OuterJoinType.FULL) { + for (Integer key : rightMap.keySet()) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + + if (leftValues != null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection joinedValues = map.get(key); + + for (String rightValue : rightValues) { + joinedValues.add(new Match(null, rightValue)); + } + } + } + + return map; + } + + + private Map> collectData(MutableObjectIterator> iter) + throws Exception { + final Map> map = new HashMap>(); + Tuple2 pair = new Tuple2(); + + while ((pair = iter.next(pair)) != null) { + final Integer key = pair.getField(0); + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection values = map.get(key); + final String value = pair.getField(1); + values.add(value); + } + + return map; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java index 7fc37349fb47a..65480529831d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java @@ -135,7 +135,7 @@ public void testMerge() { collectData(input2)); final FlatJoinFunction, Tuple2, Tuple2> joinFunction = - new MatchRemovingMatcher(expectedMatchesMap); + new MatchRemovingJoiner(expectedMatchesMap); final Collector> collector = new DiscardingOutputCollector>(); @@ -226,7 +226,7 @@ public void testMergeWithHighNumberOfCommonKeys() input1 = new MergeIterator>(inList1, comparator1.duplicate()); input2 = new MergeIterator>(inList2, comparator2.duplicate()); - final FlatJoinFunction, Tuple2, Tuple2> joinFunction = new MatchRemovingMatcher(expectedMatchesMap); + final FlatJoinFunction, Tuple2, Tuple2> joinFunction = new MatchRemovingJoiner(expectedMatchesMap); final Collector> collector = new DiscardingOutputCollector>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java new file mode 100644 index 0000000000000..1205bc13f24e2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java @@ -0,0 +1,82 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.Test; + +public class NonReusingSortMergeOuterJoinIteratorITCase extends AbstractSortMergeOuterJoinIteratorITCase { + + @Override + protected AbstractMergeOuterJoinIterator createOuterJoinIterator(OuterJoinType outerJoinType, MutableObjectIterator input1, + MutableObjectIterator input2, TypeSerializer serializer1, + TypeComparator comparator1, TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, MemoryManager memoryManager, IOManager ioManager, + int numMemoryPages, AbstractInvokable parentTask) throws Exception { + return new NonReusingMergeOuterJoinIterator(outerJoinType, input1, input2, serializer1, comparator1, + serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + } + + @Test + public void testFullOuterWithSample() throws Exception { + super.testFullOuterWithSample(); + } + + @Test + public void testLeftOuterWithSample() throws Exception { + super.testLeftOuterWithSample(); + } + + @Test + public void testRightOuterWithSample() throws Exception { + super.testRightOuterWithSample(); + } + + @Test + public void testRightSideEmpty() throws Exception { + super.testRightSideEmpty(); + } + + @Test + public void testLeftSideEmpty() throws Exception { + super.testLeftSideEmpty(); + } + + @Test + public void testFullOuterJoinWithHighNumberOfCommonKeys() { + testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType.FULL, 200, 500, 2048, 0.02f, 200, 500, 2048, 0.02f); + } + + @Test + public void testLeftOuterJoinWithHighNumberOfCommonKeys() { + testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType.LEFT, 200, 10, 4096, 0.02f, 100, 4000, 2048, 0.02f); + } + + @Test + public void testRightOuterJoinWithHighNumberOfCommonKeys() { + testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType.RIGHT, 100, 10, 2048, 0.02f, 200, 4000, 4096, 0.02f); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java index e4eec86acba56..39316e3360fd3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java @@ -135,7 +135,7 @@ public void testMerge() { collectData(input2)); final FlatJoinFunction, Tuple2, Tuple2> joinFunction = - new MatchRemovingMatcher(expectedMatchesMap); + new MatchRemovingJoiner(expectedMatchesMap); final Collector> collector = new DiscardingOutputCollector>(); @@ -226,7 +226,7 @@ public void testMergeWithHighNumberOfCommonKeys() input1 = new MergeIterator>(inList1, comparator1.duplicate()); input2 = new MergeIterator>(inList2, comparator2.duplicate()); - final FlatJoinFunction, Tuple2, Tuple2> matcher = new MatchRemovingMatcher(expectedMatchesMap); + final FlatJoinFunction, Tuple2, Tuple2> matcher = new MatchRemovingJoiner(expectedMatchesMap); final Collector> collector = new DiscardingOutputCollector>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java new file mode 100644 index 0000000000000..b4fbd80c1d541 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java @@ -0,0 +1,82 @@ +/* + * 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.runtime.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.Test; + +public class ReusingSortMergeOuterJoinIteratorITCase extends AbstractSortMergeOuterJoinIteratorITCase { + + @Override + protected AbstractMergeOuterJoinIterator createOuterJoinIterator(OuterJoinType outerJoinType, MutableObjectIterator input1, + MutableObjectIterator input2, TypeSerializer serializer1, + TypeComparator comparator1, TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, MemoryManager memoryManager, IOManager ioManager, + int numMemoryPages, AbstractInvokable parentTask) throws Exception { + return new ReusingMergeOuterJoinIterator(outerJoinType, input1, input2, serializer1, comparator1, + serializer2, comparator2, pairComparator, memoryManager, ioManager, numMemoryPages, parentTask); + } + + @Test + public void testFullOuterWithSample() throws Exception { + super.testFullOuterWithSample(); + } + + @Test + public void testLeftOuterWithSample() throws Exception { + super.testLeftOuterWithSample(); + } + + @Test + public void testRightOuterWithSample() throws Exception { + super.testRightOuterWithSample(); + } + + @Test + public void testRightSideEmpty() throws Exception { + super.testRightSideEmpty(); + } + + @Test + public void testLeftSideEmpty() throws Exception { + super.testLeftSideEmpty(); + } + + @Test + public void testFullOuterJoinWithHighNumberOfCommonKeys() { + testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType.FULL, 200, 500, 2048, 0.02f, 200, 500, 2048, 0.02f); + } + + @Test + public void testLeftOuterJoinWithHighNumberOfCommonKeys() { + testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType.LEFT, 200, 10, 4096, 0.02f, 100, 4000, 2048, 0.02f); + } + + @Test + public void testRightOuterJoinWithHighNumberOfCommonKeys() { + testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType.RIGHT, 100, 10, 2048, 0.02f, 200, 4000, 4096, 0.02f); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java index 539d8648097ca..4ac90932b87e5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/Match.java @@ -21,7 +21,7 @@ /** * Utility class for keeping track of matches in join operator tests. * - * @see org.apache.flink.runtime.operators.testutils.MatchRemovingMatcher + * @see MatchRemovingJoiner */ public class Match { private final String left; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingJoiner.java similarity index 90% rename from flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingJoiner.java index f69b4d74f762c..e588d9258040b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingMatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MatchRemovingJoiner.java @@ -27,12 +27,12 @@ import java.util.Map; -public final class MatchRemovingMatcher implements FlatJoinFunction,Tuple2,Tuple2> { +public final class MatchRemovingJoiner implements FlatJoinFunction,Tuple2,Tuple2> { private static final long serialVersionUID = 1L; private final Map> toRemoveFrom; - public MatchRemovingMatcher(Map> map) { + public MatchRemovingJoiner(Map> map) { this.toRemoveFrom = map; } From 5546a1efabcc9c5f500abdb4e38a5cc05d35980a Mon Sep 17 00:00:00 2001 From: twalthr Date: Tue, 4 Aug 2015 15:30:28 +0200 Subject: [PATCH 080/175] [FLINK-2447] [java api] TypeExtractor returns wrong type info when a Tuple has two fields of the same POJO type This closes #986 --- .../api/java/typeutils/TypeExtractor.java | 53 ++++++++++++---- .../extractor/PojoTypeExtractionTest.java | 63 +++++++++++++++++++ 2 files changed, 104 insertions(+), 12 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index 41644f9c16955..1ae8d3d62fa1f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -26,9 +26,7 @@ import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; -import java.util.Set; import org.apache.avro.specific.SpecificRecordBase; import org.apache.flink.api.common.functions.CoGroupFunction; @@ -66,15 +64,33 @@ * functions. */ public class TypeExtractor { + + /* + * NOTE: Most methods of the TypeExtractor work with a so-called "typeHierarchy". + * The type hierarchy describes all types (Classes, ParameterizedTypes, TypeVariables etc. ) and intermediate + * types from a given type of a function or type (e.g. MyMapper, Tuple2) until a current type + * (depends on the method, e.g. MyPojoFieldType). + * + * Thus, it fully qualifies types until tuple/POJO field level. + * + * A typical typeHierarchy could look like: + * + * UDF: MyMapFunction.class + * top-level UDF: MyMapFunctionBase.class + * RichMapFunction: RichMapFunction.class + * MapFunction: MapFunction.class + * Function's OUT: Tuple1 + * user-defined POJO: MyPojo.class + * user-defined top-level POJO: MyPojoBase.class + * POJO field: Tuple1 + * Field type: String.class + * + */ private static final Logger LOG = LoggerFactory.getLogger(TypeExtractor.class); - // We need this to detect recursive types and not get caught - // in an endless recursion - private Set> alreadySeen; - protected TypeExtractor() { - alreadySeen = new HashSet>(); + // only create instances for special use cases } // -------------------------------------------------------------------------------------------- @@ -416,10 +432,12 @@ private TypeInformation createTypeInfoWithTypeHierarchy(Arr TypeInformation[] tupleSubTypes = new TypeInformation[subtypes.length]; for (int i = 0; i < subtypes.length; i++) { + ArrayList subTypeHierarchy = new ArrayList(typeHierarchy); + subTypeHierarchy.add(subtypes[i]); // sub type could not be determined with materializing // try to derive the type info of the TypeVariable from the immediate base child input as a last attempt if (subtypes[i] instanceof TypeVariable) { - tupleSubTypes[i] = createTypeInfoFromInputs((TypeVariable) subtypes[i], typeHierarchy, in1Type, in2Type); + tupleSubTypes[i] = createTypeInfoFromInputs((TypeVariable) subtypes[i], subTypeHierarchy, in1Type, in2Type); // variable could not be determined if (tupleSubTypes[i] == null) { @@ -430,7 +448,7 @@ private TypeInformation createTypeInfoWithTypeHierarchy(Arr + "all variables in the return type can be deduced from the input type(s)."); } } else { - tupleSubTypes[i] = createTypeInfoWithTypeHierarchy(new ArrayList(typeHierarchy), subtypes[i], in1Type, in2Type); + tupleSubTypes[i] = createTypeInfoWithTypeHierarchy(subTypeHierarchy, subtypes[i], in1Type, in2Type); } } @@ -912,6 +930,19 @@ else if (typeInfo instanceof GenericTypeInfo) { // -------------------------------------------------------------------------------------------- // Utility methods // -------------------------------------------------------------------------------------------- + + /** + * @return number of items with equal type or same raw type + */ + private static int countTypeInHierarchy(ArrayList typeHierarchy, Type type) { + int count = 0; + for (Type t : typeHierarchy) { + if (t == type || (isClassType(type) && t == typeToClass(type))) { + count++; + } + } + return count; + } /** * @param curT : start type @@ -1183,12 +1214,10 @@ private TypeInformation privateGetForClass(Class clazz, return (TypeInformation) new AvroTypeInfo(clazz); } - if (alreadySeen.contains(clazz)) { + if (countTypeInHierarchy(typeHierarchy, clazz) > 1) { return new GenericTypeInfo(clazz); } - alreadySeen.add(clazz); - if (Modifier.isInterface(clazz.getModifiers())) { // Interface has no members and is therefore not handled as POJO return new GenericTypeInfo(clazz); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java index 1f3f71ca782e6..34fde2051d362 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java @@ -810,4 +810,67 @@ public void testGenericPojoTypeInference7() { + ">")); Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); } + + public static class RecursivePojo1 { + public RecursivePojo1 field; + } + + public static class RecursivePojo2 { + public Tuple1 field; + } + + public static class RecursivePojo3 { + public NestedPojo field; + } + + public static class NestedPojo { + public RecursivePojo3 field; + } + + @Test + public void testRecursivePojo1() { + TypeInformation ti = TypeExtractor.createTypeInfo(RecursivePojo1.class); + Assert.assertTrue(ti instanceof PojoTypeInfo); + Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) ti).getPojoFieldAt(0).type.getClass()); + } + + @Test + public void testRecursivePojo2() { + TypeInformation ti = TypeExtractor.createTypeInfo(RecursivePojo2.class); + Assert.assertTrue(ti instanceof PojoTypeInfo); + PojoField pf = ((PojoTypeInfo) ti).getPojoFieldAt(0); + Assert.assertTrue(pf.type instanceof TupleTypeInfo); + Assert.assertEquals(GenericTypeInfo.class, ((TupleTypeInfo) pf.type).getTypeAt(0).getClass()); + } + + @Test + public void testRecursivePojo3() { + TypeInformation ti = TypeExtractor.createTypeInfo(RecursivePojo3.class); + Assert.assertTrue(ti instanceof PojoTypeInfo); + PojoField pf = ((PojoTypeInfo) ti).getPojoFieldAt(0); + Assert.assertTrue(pf.type instanceof PojoTypeInfo); + Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) pf.type).getPojoFieldAt(0).type.getClass()); + } + + public static class FooBarPojo { + public int foo, bar; + public FooBarPojo() {} + } + + public static class DuplicateMapper implements MapFunction> { + @Override + public Tuple2 map(FooBarPojo value) throws Exception { + return null; + } + } + + @Test + public void testDualUseOfPojo() { + MapFunction function = new DuplicateMapper(); + TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) TypeExtractor.createTypeInfo(FooBarPojo.class)); + Assert.assertTrue(ti instanceof TupleTypeInfo); + TupleTypeInfo tti = ((TupleTypeInfo) ti); + Assert.assertTrue(tti.getTypeAt(0) instanceof PojoTypeInfo); + Assert.assertTrue(tti.getTypeAt(1) instanceof PojoTypeInfo); + } } From 100e8c5ff9f6d25b3d5db326a5f31b9c4432e334 Mon Sep 17 00:00:00 2001 From: zentol Date: Sat, 25 Jul 2015 15:17:55 +0200 Subject: [PATCH 081/175] [FLINK-1882] Removed RemotedCollector classes This closes #985 --- .../RemoteCollectorOutputFormatExample.java | 114 --------- .../flink/api/java/io/RemoteCollector.java | 46 ---- .../api/java/io/RemoteCollectorConsumer.java | 26 -- .../api/java/io/RemoteCollectorImpl.java | 228 ------------------ .../java/io/RemoteCollectorOutputFormat.java | 175 -------------- 5 files changed, 589 deletions(-) delete mode 100644 flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/RemoteCollectorOutputFormatExample.java delete mode 100644 flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollector.java delete mode 100644 flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorConsumer.java delete mode 100644 flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorImpl.java delete mode 100644 flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorOutputFormat.java diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/RemoteCollectorOutputFormatExample.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/RemoteCollectorOutputFormatExample.java deleted file mode 100644 index f524718407247..0000000000000 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/RemoteCollectorOutputFormatExample.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.examples.java.misc; - -import java.util.HashSet; -import java.util.Set; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.io.RemoteCollectorConsumer; -import org.apache.flink.api.java.io.RemoteCollectorImpl; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.util.Collector; - -/** - * Implements the "WordCount" program that computes a simple word occurrence - * histogram over some sample data and collects the results with an - * implementation of a {@link RemoteCollectorConsumer}. - */ -@SuppressWarnings("serial") -public class RemoteCollectorOutputFormatExample { - - public static void main(String[] args) throws Exception { - - /** - * We create a remote {@link ExecutionEnvironment} here, because this - * OutputFormat is designed for use in a distributed setting. For local - * use you should consider using the {@link LocalCollectionOutputFormat - * }. - */ - final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("", 6124, - "/path/to/your/file.jar"); - - // get input data - DataSet text = env.fromElements( - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,"); - - DataSet> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new LineSplitter()) - // group by the tuple field "0" and sum up tuple field "1" - .groupBy(0).aggregate(Aggregations.SUM, 1); - - // emit result - RemoteCollectorImpl.collectLocal(counts, - new RemoteCollectorConsumer>() { - // user defined IRemoteCollectorConsumer - @Override - public void collect(Tuple2 element) { - System.out.println("word/occurrences:" + element); - } - }); - - // local collection to store results in - Set> collection = new HashSet>(); - // collect results from remote in local collection - RemoteCollectorImpl.collectLocal(counts, collection); - - // execute program - env.execute("WordCount Example with RemoteCollectorOutputFormat"); - - System.out.println(collection); - - RemoteCollectorImpl.shutdownAll(); - } - - // - // User Functions - // - - /** - * Implements the string tokenizer that splits sentences into words as a - * user-defined FlatMapFunction. The function takes a line (String) and - * splits it into multiple pairs in the form of "(word,1)" (Tuple2). - */ - public static final class LineSplitter implements - FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } - } -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollector.java b/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollector.java deleted file mode 100644 index bcfc33245a88c..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollector.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.api.java.io; - -import org.apache.flink.api.java.DataSet; - -import java.rmi.Remote; -import java.rmi.RemoteException; - -/** - * This interface is the counterpart to the {@link RemoteCollectorOutputFormat} - * and implementations will receive remote results through the collect function. - * - * @param - * The type of the records the collector will receive - * - * @deprecated Results are retrieved through {@link org.apache.flink.api.common.accumulators.Accumulator} - * and the {@link DataSet#collect()} method respectively. - */ -@Deprecated -public interface RemoteCollector extends Remote { - - public void collect(T element) throws RemoteException; - - public RemoteCollectorConsumer getConsumer() throws RemoteException; - - public void setConsumer(RemoteCollectorConsumer consumer) - throws RemoteException; - -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorConsumer.java b/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorConsumer.java deleted file mode 100644 index 439c6af222c2e..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorConsumer.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.api.java.io; - -/** - * This interface describes consumers of {@link RemoteCollector} implementations. - */ -public interface RemoteCollectorConsumer { - public void collect(T element); -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorImpl.java b/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorImpl.java deleted file mode 100644 index 2d080ab60ecc4..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorImpl.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * 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.api.java.io; - -import java.net.Inet4Address; -import java.net.InetAddress; -import java.net.NetworkInterface; -import java.net.ServerSocket; -import java.rmi.AccessException; -import java.rmi.AlreadyBoundException; -import java.rmi.NotBoundException; -import java.rmi.Remote; -import java.rmi.RemoteException; -import java.rmi.registry.LocateRegistry; -import java.rmi.registry.Registry; -import java.rmi.server.UnicastRemoteObject; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Enumeration; -import java.util.List; -import java.util.UUID; - -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.DataSink; - -/** - * This class provides a counterpart implementation for the - * {@link RemoteCollectorOutputFormat}. - * - * @deprecated Results are retrieved through {@link org.apache.flink.api.common.accumulators.Accumulator} - * and the {@link DataSet#collect()} method respectively. - */ - -@Deprecated -public class RemoteCollectorImpl extends UnicastRemoteObject implements - RemoteCollector { - - private static final long serialVersionUID = 1L; - - /** - * Instance of an implementation of a {@link RemoteCollectorConsumer}. This - * instance will get the records passed. - */ - - private RemoteCollectorConsumer consumer; - - /** - * This list stores all created {@link Registry}s to unbind and unexport all - * exposed {@link Remote} objects ({@link RemoteCollectorConsumer} in our - * case) in the shutdown phase. - */ - private static List registries = new ArrayList(); - - /** - * This factory method creates an instance of the - * {@link RemoteCollectorImpl} and binds it in the local RMI - * {@link Registry}. - * - * @param port - * The port where the local colector is listening. - * @param consumer - * The consumer instance. - * @param rmiId - * An ID to register the collector in the RMI registry. - */ - public static void createAndBind(Integer port, RemoteCollectorConsumer consumer, String rmiId) { - RemoteCollectorImpl collectorInstance = null; - - try { - collectorInstance = new RemoteCollectorImpl(); - - Registry registry; - - registry = LocateRegistry.createRegistry(port); - registry.bind(rmiId, collectorInstance); - - registries.add(registry); - } catch (RemoteException e) { - e.printStackTrace(); - } catch (AlreadyBoundException e) { - e.printStackTrace(); - } - - collectorInstance.setConsumer(consumer); - } - - /** - * Writes a DataSet to a {@link RemoteCollectorConsumer} through an - * {@link RemoteCollector} remotely called from the - * {@link RemoteCollectorOutputFormat}.
    - * - * @return The DataSink that writes the DataSet. - */ - public static DataSink collectLocal(DataSet source, - RemoteCollectorConsumer consumer) { - // if the RMI parameter was not set by the user make a "good guess" - String ip = System.getProperty("java.rmi.server.hostname"); - if (ip == null) { - Enumeration networkInterfaces = null; - try { - networkInterfaces = NetworkInterface.getNetworkInterfaces(); - } catch (Throwable t) { - throw new RuntimeException(t); - } - while (networkInterfaces.hasMoreElements()) { - NetworkInterface networkInterface = (NetworkInterface) networkInterfaces - .nextElement(); - Enumeration inetAddresses = networkInterface - .getInetAddresses(); - while (inetAddresses.hasMoreElements()) { - InetAddress inetAddress = (InetAddress) inetAddresses - .nextElement(); - if (!inetAddress.isLoopbackAddress() - && inetAddress instanceof Inet4Address) { - ip = inetAddress.getHostAddress(); - System.setProperty("java.rmi.server.hostname", ip); - } - } - } - } - - // get some random free port - Integer randomPort = 0; - try { - ServerSocket tmp = new ServerSocket(0); - randomPort = tmp.getLocalPort(); - tmp.close(); - } catch (Throwable t) { - throw new RuntimeException(t); - } - - // create an ID for this output format instance - String rmiId = String.format("%s-%s", RemoteCollectorOutputFormat.class.getName(), UUID.randomUUID()); - - // create the local listening object and bind it to the RMI registry - RemoteCollectorImpl.createAndBind(randomPort, consumer, rmiId); - - // create and configure the output format - OutputFormat remoteCollectorOutputFormat = new RemoteCollectorOutputFormat(ip, randomPort, rmiId); - - // create sink - return source.output(remoteCollectorOutputFormat); - } - - /** - * Writes a DataSet to a local {@link Collection} through an - * {@link RemoteCollector} and a standard {@link RemoteCollectorConsumer} - * implementation remotely called from the - * {@link RemoteCollectorOutputFormat}.
    - * - * @param source the source data set - * @param collection the local collection - */ - public static void collectLocal(DataSet source, - Collection collection) { - final Collection synchronizedCollection = Collections - .synchronizedCollection(collection); - collectLocal(source, new RemoteCollectorConsumer() { - @Override - public void collect(T element) { - synchronizedCollection.add(element); - } - }); - } - - /** - * Necessary private default constructor. - * - * @throws RemoteException - */ - private RemoteCollectorImpl() throws RemoteException { - super(); - } - - /** - * This method is called by the remote to collect records. - */ - @Override - public void collect(T element) throws RemoteException { - this.consumer.collect(element); - } - - @Override - public RemoteCollectorConsumer getConsumer() { - return this.consumer; - } - - @Override - public void setConsumer(RemoteCollectorConsumer consumer) { - this.consumer = consumer; - } - - /** - * This method unbinds and unexports all exposed {@link Remote} objects - * - * @throws AccessException - * @throws RemoteException - * @throws NotBoundException - */ - public static void shutdownAll() throws AccessException, RemoteException, NotBoundException { - for (Registry registry : registries) { - for (String id : registry.list()) { - Remote remote = registry.lookup(id); - registry.unbind(id); - UnicastRemoteObject.unexportObject(remote, true); - } - - } - } -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorOutputFormat.java deleted file mode 100644 index 3fe5cefd7ba69..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/RemoteCollectorOutputFormat.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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.api.java.io; - -import java.io.IOException; -import java.rmi.AccessException; -import java.rmi.NotBoundException; -import java.rmi.RemoteException; -import java.rmi.registry.LocateRegistry; -import java.rmi.registry.Registry; - -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.configuration.Configuration; - -/** - * An output format that sends results through JAVA RMI to an - * {@link RemoteCollector} implementation. The client has to provide an - * implementation of {@link RemoteCollector} and has to write it's plan's output - * into an instance of {@link RemoteCollectorOutputFormat}. Further in the - * client's VM parameters -Djava.rmi.server.hostname should be set to the own IP - * address. - * - * @deprecated Results are retrieved through {@link org.apache.flink.api.common.accumulators.Accumulator} - * and the {@link DataSet#collect()} method respectively. - */ -@Deprecated -public class RemoteCollectorOutputFormat implements OutputFormat { - - private static final long serialVersionUID = 1922744224032398102L; - - /** - * The reference of the {@link RemoteCollector} object - */ - private transient RemoteCollector remoteCollector; - - transient private Registry registry; - - /** - * Config parameter for the remote's port number - */ - public static final String PORT = "port"; - /** - * Config parameter for the remote's address - */ - public static final String REMOTE = "remote"; - /** - * An id used necessary for Java RMI - */ - public static final String RMI_ID = "rmiId"; - - private String remote; - - private int port; - - private String rmiId; - - /** - * Create a new {@link RemoteCollectorOutputFormat} instance. The remote and - * port for this output are by default localhost:8888 but can be configured - * via a {@link Configuration} object. - * - * @see RemoteCollectorOutputFormat#REMOTE - * @see RemoteCollectorOutputFormat#PORT - */ - public RemoteCollectorOutputFormat() { - this("localhost", 8888, null); - } - - /** - * Creates a new {@link RemoteCollectorOutputFormat} instance for the - * specified remote and port. - * - * @param rmiId - */ - public RemoteCollectorOutputFormat(String remote, int port, String rmiId) { - super(); - this.remote = remote; - this.port = port; - this.rmiId = rmiId; - - if (this.remote == null) { - throw new IllegalStateException(String.format( - "No remote configured for %s.", this)); - } - - if (this.rmiId == null) { - throw new IllegalStateException(String.format( - "No registry ID configured for %s.", this)); - } - } - - @Override - /** - * This method receives the Configuration object, where the fields "remote" and "port" must be set. - */ - public void configure(Configuration parameters) { - this.remote = parameters.getString(REMOTE, this.remote); - this.port = parameters.getInteger(PORT, this.port); - this.rmiId = parameters.getString(RMI_ID, this.rmiId); - - if (this.remote == null) { - throw new IllegalStateException(String.format( - "No remote configured for %s.", this)); - } - - if (this.rmiId == null) { - throw new IllegalStateException(String.format( - "No registry ID configured for %s.", this)); - } - } - - @SuppressWarnings("unchecked") - @Override - public void open(int taskNumber, int numTasks) throws IOException { - // get the remote's RMI Registry - try { - registry = LocateRegistry.getRegistry(this.remote, this.port); - } catch (RemoteException e) { - throw new IllegalStateException(e); - } - - // try to get an intance of an IRemoteCollector implementation - try { - this.remoteCollector = (RemoteCollector) registry - .lookup(this.rmiId); - } catch (AccessException e) { - throw new IllegalStateException(e); - } catch (RemoteException e) { - throw new IllegalStateException(e); - } catch (NotBoundException e) { - throw new IllegalStateException(e); - } - } - - /** - * This method forwards records simply to the remote's - * {@link RemoteCollector} implementation - */ - @Override - public void writeRecord(T record) throws IOException { - remoteCollector.collect(record); - } - - /** - * This method unbinds the reference of the implementation of - * {@link RemoteCollector}. - */ - @Override - public void close() throws IOException { - } - - @Override - public String toString() { - return "RemoteCollectorOutputFormat(" + remote + ":" + port + ", " - + rmiId + ")"; - } - -} From 0aa6f0cb7b3510e0e9aa938411b6db77f7f7505e Mon Sep 17 00:00:00 2001 From: mjsax Date: Mon, 3 Aug 2015 15:51:46 +0200 Subject: [PATCH 082/175] [FLINK-2467] Example WordCountStorm.jar is not packaged correctly - fixed assembly xml file This closes #974 --- .../src/assembly/word-count-storm.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/assembly/word-count-storm.xml b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/assembly/word-count-storm.xml index 9721115ff3212..96ac4291df113 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/assembly/word-count-storm.xml +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/assembly/word-count-storm.xml @@ -34,9 +34,9 @@ under the License. / true - org.apache.storm:storm-core:jar - org.apache.flink:flink-storm-examples:jar + org.apache.flink:flink-storm-compatibility-core:jar + org.apache.flink:flink-storm-compatibility-examples:jar @@ -60,10 +60,10 @@ under the License. org/apache/flink/stormcompatibility/api/*.class org/apache/flink/stormcompatibility/wrappers/*.class - org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.class - + org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.class org/apache/flink/stormcompatibility/wordcount/WordCountTopology.class org/apache/flink/stormcompatibility/wordcount/stormoperators/*.class + org/apache/flink/stormcompatibility/util/*.class org/apache/flink/examples/java/wordcount/util/WordCountData.class From fb7e6342211d116a2db13933241d3546bbf8d4e8 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 4 Aug 2015 13:35:12 +0200 Subject: [PATCH 083/175] [FLINK-1680] Remove Tachyon test and rename Maven module to "flink-fs-tests" This closes #987 --- .../{flink-tachyon => flink-fs-tests}/pom.xml | 39 +--- .../flink/tachyon/FileStateHandleTest.java | 0 .../org/apache/flink/tachyon/HDFSTest.java | 19 +- .../src/test/resources/log4j.properties | 0 .../tachyon/TachyonFileSystemWrapperTest.java | 167 ------------------ .../src/test/resources/tachyonHadoopConf.xml | 28 --- flink-staging/pom.xml | 4 +- 7 files changed, 22 insertions(+), 235 deletions(-) rename flink-staging/{flink-tachyon => flink-fs-tests}/pom.xml (70%) rename flink-staging/{flink-tachyon => flink-fs-tests}/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java (100%) rename flink-staging/{flink-tachyon => flink-fs-tests}/src/test/java/org/apache/flink/tachyon/HDFSTest.java (89%) rename flink-staging/{flink-tachyon => flink-fs-tests}/src/test/resources/log4j.properties (100%) delete mode 100644 flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java delete mode 100644 flink-staging/flink-tachyon/src/test/resources/tachyonHadoopConf.xml diff --git a/flink-staging/flink-tachyon/pom.xml b/flink-staging/flink-fs-tests/pom.xml similarity index 70% rename from flink-staging/flink-tachyon/pom.xml rename to flink-staging/flink-fs-tests/pom.xml index 7ad9139ce4b14..fe1abb3e9584b 100644 --- a/flink-staging/flink-tachyon/pom.xml +++ b/flink-staging/flink-fs-tests/pom.xml @@ -27,8 +27,8 @@ under the License. .. - flink-tachyon - flink-tachyon + flink-fs-tests + flink-fs-tests jar @@ -74,40 +74,5 @@ under the License. test-jar ${hadoop.version} - - org.tachyonproject - tachyon - 0.5.0 - test - - - org.tachyonproject - tachyon - 0.5.0 - test-jar - test - - - org.eclipse.jetty - jetty-util - 7.6.8.v20121106 - test - - - - - org.eclipse.jetty - jetty-server - 7.6.8.v20121106 - test - - - org.eclipse.jetty - jetty-servlet - 7.6.8.v20121106 - test - - - diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java similarity index 100% rename from flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java rename to flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java similarity index 89% rename from flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java rename to flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java index a7617121861c4..633d0224b68b7 100644 --- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java +++ b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java @@ -20,6 +20,9 @@ import org.apache.commons.io.IOUtils; import org.apache.flink.api.common.io.FileOutputFormat; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.ExecutionEnvironmentFactory; +import org.apache.flink.api.java.LocalEnvironment; import org.apache.flink.api.java.io.AvroOutputFormat; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; @@ -96,7 +99,7 @@ public void testHDFS() { try { FileSystem fs = file.getFileSystem(); Assert.assertTrue("Must be HadoopFileSystem", fs instanceof HadoopFileSystem); - new TachyonFileSystemWrapperTest.DopOneTestEnvironment(); + new DopOneTestEnvironment(); try { WordCount.main(new String[]{file.toString(), result.toString()}); } catch(Throwable t) { @@ -154,4 +157,18 @@ public void testAvroOut() { Assert.fail(e.getMessage()); } } + + // package visible + static final class DopOneTestEnvironment extends LocalEnvironment { + static { + initializeContextEnvironment(new ExecutionEnvironmentFactory() { + @Override + public ExecutionEnvironment createExecutionEnvironment() { + LocalEnvironment le = new LocalEnvironment(); + le.setParallelism(1); + return le; + } + }); + } + } } diff --git a/flink-staging/flink-tachyon/src/test/resources/log4j.properties b/flink-staging/flink-fs-tests/src/test/resources/log4j.properties similarity index 100% rename from flink-staging/flink-tachyon/src/test/resources/log4j.properties rename to flink-staging/flink-fs-tests/src/test/resources/log4j.properties diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java deleted file mode 100644 index 3b2fb7fa1beec..0000000000000 --- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * 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.tachyon; - -import org.apache.commons.io.IOUtils; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.ExecutionEnvironmentFactory; -import org.apache.flink.api.java.LocalEnvironment; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.examples.java.wordcount.WordCount; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import tachyon.client.InStream; -import tachyon.client.OutStream; -import tachyon.client.ReadType; -import tachyon.client.TachyonFS; -import tachyon.client.TachyonFile; -import tachyon.client.WriteType; -import tachyon.master.LocalTachyonCluster; - -import java.io.File; -import java.io.StringWriter; -import java.net.URISyntaxException; -import java.net.URL; - -public class TachyonFileSystemWrapperTest { - private static final long TACHYON_WORKER_CAPACITY = 1024 * 1024 * 32; - private static final String TACHYON_TEST_IN_FILE_NAME = "tachyontest"; - private static final String TACHYON_TEST_OUT_FILE_NAME = "result"; - private static final Path HADOOP_CONFIG_PATH; - - static { - URL resource = TachyonFileSystemWrapperTest.class.getResource("/tachyonHadoopConf.xml"); - File file = null; - try { - file = new File(resource.toURI()); - } catch (URISyntaxException e) { - throw new RuntimeException("Unable to load req. res", e); - } - if(!file.exists()) { - throw new RuntimeException("Unable to load required resource"); - } - HADOOP_CONFIG_PATH = new Path(file.getAbsolutePath()); - } - - private LocalTachyonCluster cluster; - private TachyonFS client; - private String input; - private String output; - - @Before - public void startTachyon() { - try { - cluster = new LocalTachyonCluster(TACHYON_WORKER_CAPACITY); - cluster.start(); - client = cluster.getClient(); - int id = client.createFile("/" + TACHYON_TEST_IN_FILE_NAME, 1024 * 32); - Assert.assertNotEquals("Unable to create file", -1, id); - - TachyonFile testFile = client.getFile(id); - Assert.assertNotNull(testFile); - - - OutStream outStream = testFile.getOutStream(WriteType.MUST_CACHE); - for(int i = 0; i < 10; i++) { - outStream.write("Hello Tachyon\n".getBytes()); - } - outStream.close(); - final String tachyonBase = "tachyon://" + cluster.getMasterHostname() + ":" + cluster.getMasterPort(); - input = tachyonBase + "/" + TACHYON_TEST_IN_FILE_NAME; - output = tachyonBase + "/" + TACHYON_TEST_OUT_FILE_NAME; - - } catch(Exception e) { - e.printStackTrace(); - Assert.fail("Test preparation failed with exception: "+e.getMessage()); - } - } - - @After - public void stopTachyon() { - try { - cluster.stop(); - } catch(Exception e) { - e.printStackTrace(); - Assert.fail("Test teardown failed with exception: "+e.getMessage()); - } - } - // Verify that Hadoop's FileSystem can load the TFS (Tachyon File System) - @Test - public void testHadoopLoadability() { - try { - Path tPath = new Path(input); - Configuration conf = new Configuration(); - conf.addResource(HADOOP_CONFIG_PATH); - Assert.assertEquals("tachyon.hadoop.TFS", conf.get("fs.tachyon.impl", null)); - tPath.getFileSystem(conf); - } catch(Exception e) { - e.printStackTrace(); - Assert.fail("Test failed with exception: "+e.getMessage()); - } - } - - - @Test - public void testTachyon() { - try { - org.apache.flink.configuration.Configuration addHDConfToFlinkConf = new org.apache.flink.configuration.Configuration(); - addHDConfToFlinkConf.setString(ConfigConstants.HDFS_DEFAULT_CONFIG, HADOOP_CONFIG_PATH.toString()); - GlobalConfiguration.includeConfiguration(addHDConfToFlinkConf); - - new DopOneTestEnvironment(); // initialize parallelism one - - WordCount.main(new String[]{input, output}); - - // verify result - TachyonFile resultFile = client.getFile("/" + TACHYON_TEST_OUT_FILE_NAME); - Assert.assertNotNull("Result file has not been created", resultFile); - InStream inStream = resultFile.getInStream(ReadType.CACHE); - Assert.assertNotNull("Result file has not been created", inStream); - StringWriter writer = new StringWriter(); - IOUtils.copy(inStream, writer); - String resultString = writer.toString(); - - Assert.assertEquals("hello 10\n" + - "tachyon 10\n", resultString); - - } catch(Exception e) { - e.printStackTrace(); - Assert.fail("Test failed with exception: "+e.getMessage()); - } - } - - // package visible - static final class DopOneTestEnvironment extends LocalEnvironment { - static { - initializeContextEnvironment(new ExecutionEnvironmentFactory() { - @Override - public ExecutionEnvironment createExecutionEnvironment() { - LocalEnvironment le = new LocalEnvironment(); - le.setParallelism(1); - return le; - } - }); - } - } - -} diff --git a/flink-staging/flink-tachyon/src/test/resources/tachyonHadoopConf.xml b/flink-staging/flink-tachyon/src/test/resources/tachyonHadoopConf.xml deleted file mode 100644 index 0af819018bbb8..0000000000000 --- a/flink-staging/flink-tachyon/src/test/resources/tachyonHadoopConf.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - - - fs.tachyon.impl - tachyon.hadoop.TFS - - \ No newline at end of file diff --git a/flink-staging/pom.xml b/flink-staging/pom.xml index a05c8b1017394..b3aec14c87e8f 100644 --- a/flink-staging/pom.xml +++ b/flink-staging/pom.xml @@ -59,10 +59,10 @@ under the License. - - flink-tachyon + flink-fs-tests From d738430cb7e26b5e31d953efa5e0036082d8de6e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 4 Aug 2015 15:18:32 +0200 Subject: [PATCH 084/175] [FLINK-2484] [streaming] BarrierBuffer releases temp files properly. --- .../streaming/runtime/io/BarrierBuffer.java | 3 +- .../runtime/io/BarrierBufferTest.java | 33 ++++++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index b7766eea995c7..fd896c9d83697 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -76,7 +76,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler { /** * - * @param inputGate Teh input gate to draw the buffers and events from. + * @param inputGate The input gate to draw the buffers and events from. * @param ioManager The I/O manager that gives access to the temp directories. * * @throws IOException Thrown, when the spilling to temp files cannot be initialized. @@ -102,6 +102,7 @@ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedExceptio if (currentBuffered != null) { next = currentBuffered.getNext(); if (next == null) { + currentBuffered.cleanup(); currentBuffered = queuedBuffered.pollFirst(); if (currentBuffered != null) { currentBuffered.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index 872e22678bb52..dd4d395990226 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -34,6 +34,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.io.File; import java.util.ArrayDeque; import java.util.Arrays; import java.util.List; @@ -94,6 +95,8 @@ public void testSingleChannelNoBarriers() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -125,6 +128,8 @@ public void testMultiChannelNoBarriers() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -166,6 +171,8 @@ public void testSingleChannelWithBarriers() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -283,6 +290,8 @@ public void testMultiChannelWithBarriers() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -338,8 +347,10 @@ public void testMultiChannelTrailingBlockedData() { assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); - buffer.cleanup(); + buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -456,6 +467,8 @@ public void testMultiChannelWithQueuedFutureBarriers() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -533,6 +546,8 @@ public void testMultiChannelSkippingCheckpoints() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -620,6 +635,8 @@ public void testMultiChannelJumpingOverCheckpoint() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -719,6 +736,8 @@ public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { assertNull(buffer.getNextNonBlocked()); buffer.cleanup(); + + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -769,6 +788,7 @@ public void testEarlyCleanup() { buffer.getNextNonBlocked(); buffer.cleanup(); + checkNoTempFilesRemain(); } catch (Exception e) { e.printStackTrace(); @@ -811,6 +831,17 @@ private static void check(BufferOrEvent expected, BufferOrEvent present) { } } + private static void checkNoTempFilesRemain() { + // validate that all temp files have been removed + for (File dir : IO_MANAGER.getSpillingDirectories()) { + for (String file : dir.list()) { + if (file != null && !(file.equals(".") || file.equals(".."))) { + fail("barrier buffer did not clean up temp files. remaining file: " + file); + } + } + } + } + // ------------------------------------------------------------------------ // Testing Mocks // ------------------------------------------------------------------------ From 3e73496c59c6e468be265afd931370efa6ba2d8a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 5 Aug 2015 17:16:34 +0200 Subject: [PATCH 085/175] [FLINK-2464] [tests] Make buffer spilling test robust to Java 6. --- .../runtime/io/BufferSpillerTest.java | 221 +++++++++--------- 1 file changed, 112 insertions(+), 109 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java index fbc19ec8515f3..355b7c80c627b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java @@ -30,14 +30,15 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Random; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicReference; + import static org.junit.Assert.*; @@ -91,6 +92,8 @@ public void cleanupSpiller() { assertFalse(spiller.getCurrentChannel().isOpen()); assertFalse(spiller.getCurrentSpillFile().exists()); } + + checkNoTempFilesRemain(); } // ------------------------------------------------------------------------ @@ -189,14 +192,17 @@ public void testSpillWhileReading() { final int sequences = 10; final Random rnd = new Random(); - final AtomicReference error = new AtomicReference(); - - final SequenceConsumer consumer = new SequenceConsumer(error, sequences); - consumer.start(); final int maxNumEventsAndBuffers = 30000; final int maxNumChannels = 1656; + int sequencesConsumed = 0; + + ArrayDeque pendingSequences = new ArrayDeque(); + SequenceToConsume currentSequence = null; + int currentNumEvents = 0; + int currentNumRecordAndEvents = 0; + // do multiple spilling / rolling over rounds for (int round = 0; round < 2*sequences; round++) { @@ -214,43 +220,110 @@ public void testSpillWhileReading() { final ArrayList events = new ArrayList(128); - // generate sequence - for (int i = 0; i < numEventsAndBuffers; i++) { - boolean isEvent = rnd.nextDouble() < 0.05d; - if (isEvent) { - BufferOrEvent evt = generateRandomEvent(rnd, numChannels); - events.add(evt); - spiller.add(evt); + int generated = 0; + while (generated < numEventsAndBuffers) { + + if (currentSequence == null || rnd.nextDouble() < 0.5) { + // add a new record + boolean isEvent = rnd.nextDouble() < 0.05; + if (isEvent) { + BufferOrEvent evt = generateRandomEvent(rnd, numChannels); + events.add(evt); + spiller.add(evt); + } + else { + BufferOrEvent evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); + spiller.add(evt); + } + generated++; } else { - BufferOrEvent evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); - spiller.add(evt); + // consume a record + BufferOrEvent next = currentSequence.sequence.getNext(); + assertNotNull(next); + if (next.isEvent()) { + BufferOrEvent expected = currentSequence.events.get(currentNumEvents++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); + } + else { + Random validationRnd = currentSequence.bufferRnd; + validateBuffer(next, validationRnd.nextInt(PAGE_SIZE) + 1, validationRnd.nextInt(currentSequence.numChannels)); + } + + currentNumRecordAndEvents++; + if (currentNumRecordAndEvents == currentSequence.numBuffersAndEvents) { + // done with the sequence + currentSequence.sequence.cleanup(); + sequencesConsumed++; + + // validate we had all events + assertEquals(currentSequence.events.size(), currentNumEvents); + + // reset + currentSequence = pendingSequences.pollFirst(); + if (currentSequence != null) { + currentSequence.sequence.open(); + } + + currentNumRecordAndEvents = 0; + currentNumEvents = 0; + } } } - // reset and create reader + // done generating a sequence. queue it for consumption bufferRnd.setSeed(bufferSeed); BufferSpiller.SpilledBufferOrEventSequence seq = spiller.rollOver(); SequenceToConsume stc = new SequenceToConsume(bufferRnd, events, seq, numEventsAndBuffers, numChannels); - consumer.queue(stc); + + if (currentSequence == null) { + currentSequence = stc; + stc.sequence.open(); + } + else { + pendingSequences.addLast(stc); + } } } - // wait for the consumer - consumer.join(180000); - assertFalse("sequence consumer did not finish its work in time", consumer.isAlive()); - - // validate there was no error in the consumer - if (error.get() != null) { - Throwable t = error.get(); - if (t instanceof Error) { - throw (Error) t; + // consume all the remainder + while (currentSequence != null) { + // consume a record + BufferOrEvent next = currentSequence.sequence.getNext(); + assertNotNull(next); + if (next.isEvent()) { + BufferOrEvent expected = currentSequence.events.get(currentNumEvents++); + assertEquals(expected.getEvent(), next.getEvent()); + assertEquals(expected.getChannelIndex(), next.getChannelIndex()); } else { - throw new Exception("Error while consuming the spilled records", t); + Random validationRnd = currentSequence.bufferRnd; + validateBuffer(next, validationRnd.nextInt(PAGE_SIZE) + 1, validationRnd.nextInt(currentSequence.numChannels)); + } + + currentNumRecordAndEvents++; + if (currentNumRecordAndEvents == currentSequence.numBuffersAndEvents) { + // done with the sequence + currentSequence.sequence.cleanup(); + sequencesConsumed++; + + // validate we had all events + assertEquals(currentSequence.events.size(), currentNumEvents); + + // reset + currentSequence = pendingSequences.pollFirst(); + if (currentSequence != null) { + currentSequence.sequence.open(); + } + + currentNumRecordAndEvents = 0; + currentNumEvents = 0; } } + + assertEquals(sequences, sequencesConsumed); } catch (Exception e) { e.printStackTrace(); @@ -300,10 +373,17 @@ private static void validateBuffer(BufferOrEvent boe, int expectedSize, int expe } } } - - // ------------------------------------------------------------------------ - // Async Consumer - // ------------------------------------------------------------------------ + + private static void checkNoTempFilesRemain() { + // validate that all temp files have been removed + for (File dir : IO_MANAGER.getSpillingDirectories()) { + for (String file : dir.list()) { + if (file != null && !(file.equals(".") || file.equals(".."))) { + fail("barrier buffer did not clean up temp files. remaining file: " + file); + } + } + } + } private static class SequenceToConsume { @@ -323,81 +403,4 @@ private SequenceToConsume(Random bufferRnd, ArrayList events, this.numChannels = numChannels; } } - - private static class SequenceConsumer extends Thread { - - private final AtomicReference error; - private final BlockingQueue sequences; - - private final int numSequencesToConsume; - - private int consumedSequences; - - private SequenceConsumer(AtomicReference error, int numSequencesToConsume) { - super("Sequence Consumer"); - setDaemon(true); - - this.error = error; - this.numSequencesToConsume = numSequencesToConsume; - this.sequences = new LinkedBlockingQueue(); - } - - - @Override - public void run() { - try { - while (consumedSequences < numSequencesToConsume) { - // get next sequence - SequenceToConsume nextSequence = sequences.take(); - - // wait a bit, allow some stuff to queue up - Thread.sleep(50); - - BufferSpiller.SpilledBufferOrEventSequence seq = nextSequence.sequence; - ArrayList events = nextSequence.events; - Random bufferRnd = nextSequence.bufferRnd; - int numBuffersAndEvents = nextSequence.numBuffersAndEvents; - int numChannels = nextSequence.numChannels; - - LOG.info("Reading sequence {}", consumedSequences); - - // consume sequence - seq.open(); - - int numEvent = 0; - for (int i = 0; i < numBuffersAndEvents; i++) { - BufferOrEvent next = seq.getNext(); - assertNotNull(next); - if (next.isEvent()) { - BufferOrEvent expected = events.get(numEvent++); - assertEquals(expected.getEvent(), next.getEvent()); - assertEquals(expected.getChannelIndex(), next.getChannelIndex()); - } - else { - validateBuffer(next, bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels)); - } - } - - // no further data - assertNull(seq.getNext()); - - // all events need to be consumed - assertEquals(events.size(), numEvent); - - // remove all temp files - seq.cleanup(); - - consumedSequences++; - } - - } - catch (Throwable t) { - error.set(t); - } - } - - public void queue(SequenceToConsume next) { - sequences.add(next); - } - } } From 5226f0b4634da95f9cedfb092d6b736f0ad471b9 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 4 Aug 2015 15:35:00 +0200 Subject: [PATCH 086/175] [hotfix] Remove unused and outdated HashFunctionCollisionBenchmark --- .../hash/HashFunctionCollisionBenchmark.java | 448 ------------------ .../operators/hash/util/LastBitsToRange.java | 54 --- .../operators/hash/util/RandomIterator.java | 65 --- .../operators/hash/util/RangeCalculator.java | 46 -- .../operators/hash/util/RangeIterator.java | 61 --- .../hash/util/StepRangeIterator.java | 68 --- 6 files changed, 742 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashFunctionCollisionBenchmark.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/LastBitsToRange.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RandomIterator.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeCalculator.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeIterator.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/StepRangeIterator.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashFunctionCollisionBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashFunctionCollisionBenchmark.java deleted file mode 100644 index d96cd462b60b9..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashFunctionCollisionBenchmark.java +++ /dev/null @@ -1,448 +0,0 @@ -/* - * 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.runtime.operators.hash; - -import java.util.ArrayList; -import java.util.Formatter; -import java.util.HashMap; -import java.util.Iterator; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.runtime.operators.hash.MutableHashTable; -import org.apache.flink.runtime.operators.hash.MultiLevelHashTester.BucketBoundaries; -import org.apache.flink.runtime.operators.hash.util.LastBitsToRange; -import org.apache.flink.runtime.operators.hash.util.RandomIterator; -import org.apache.flink.runtime.operators.hash.util.RangeCalculator; -import org.apache.flink.runtime.operators.hash.util.RangeIterator; -import org.apache.flink.runtime.operators.hash.util.StepRangeIterator; -import org.junit.Assert; -import org.junit.Test; - -/** - * Test distribution of hash function for multiple levels - * - * - */ -public class HashFunctionCollisionBenchmark { - - private static final Logger LOG = LoggerFactory.getLogger(HashFunctionCollisionBenchmark.class); - - private static final long SEED = 561349061987311L; - - @Test - public void testStepSeventeen() { - - // Define numbers of buckets on each level - RangeCalculator[] rangeCalculators = { - new LastBitsToRange(10), // 2^10=1024 Buckets on level 0 - new LastBitsToRange(10), // 2^10=1024 Buckets on level 1 - new LastBitsToRange(10) }; // 2^10=1024 Buckets on level 2 - - Iterator importIterator = new StepRangeIterator(-30000000, - 30000000, 17); - - MultiLevelHashTester ht = new MultiLevelHashTester(importIterator, - rangeCalculators); - - BucketBoundaries[] boundaries = { - new BucketBoundaries(3000, 3700, 5, 0.01), - new BucketBoundaries(0, 20, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.0001), - new BucketBoundaries(0, 3, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.000001) }; - - LOG.debug("Start Step Seventeen hash test"); - ht.runTest(boundaries); - LOG.debug("End Step Seventeen hash test"); - } - - @Test - public void testThreeLevel() { - - // Define numbers of buckets on each level - RangeCalculator[] rangeCalculators = { - new LastBitsToRange(10), // 2^10=1024 Buckets on level 0 - new LastBitsToRange(10), // 2^10=1024 Buckets on level 1 - new LastBitsToRange(10) }; // 2^10=1024 Buckets on level 2 - - Iterator importIterator = new RangeIterator(-1000000, 1000000); - - MultiLevelHashTester ht = new MultiLevelHashTester(importIterator, - rangeCalculators); - - BucketBoundaries[] boundaries = { - new BucketBoundaries(1800, 2110, 5, 0.01), - new BucketBoundaries(0, 15, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.0001), - new BucketBoundaries(0, 2, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.000001) }; - - LOG.debug("Start Three Level hash test"); - ht.runTest(boundaries); - LOG.debug("End Three Level hash test"); - } - - @Test - public void testRandom() { - - // Define numbers of buckets on each level - RangeCalculator[] rangeCalculators = { - new LastBitsToRange(10), // 2^10=1024 Buckets on level 0 - new LastBitsToRange(10), // 2^10=1024 Buckets on level 1 - new LastBitsToRange(10) }; // 2^10=1024 Buckets on level 2 - - Iterator importIterator = new RandomIterator(SEED, 2000000); - - MultiLevelHashTester ht = new MultiLevelHashTester(importIterator, - rangeCalculators); - - BucketBoundaries[] boundaries = { - new BucketBoundaries(1800, 2110, 5, 0.01), - new BucketBoundaries(0, 15, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.0001), - new BucketBoundaries(0, 2, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.000001) }; - - LOG.debug("Start Random hash test"); - ht.runTest(boundaries); - LOG.debug("End Random hash test"); - } - - @Test - public void testTwoLevel() { - - // Define numbers of buckets on each level - RangeCalculator[] rangeCalculators = { - new LastBitsToRange(12), // 2^12=4096 Buckets on level 0 - new LastBitsToRange(12) }; // 2^12=4096 Buckets on level 1 - - Iterator importIterator = new RangeIterator(-1000000, 1000000); - - MultiLevelHashTester ht = new MultiLevelHashTester(importIterator, - rangeCalculators); - - BucketBoundaries[] boundaries = { - new BucketBoundaries(400, 600, 5, 0.01), - new BucketBoundaries(0, 4, - BucketBoundaries.MAX_EMPTY_UNBOUNDED, 0.0001) }; - - LOG.debug("Start Two Level hash test"); - ht.runTest(boundaries); - LOG.debug("End Two Level hash test"); - } - -} - -class MultiLevelHashTester { - - private static final Logger LOG = LoggerFactory.getLogger(MultiLevelHashTester.class); - - private final int maxLevel; - private final Iterator importIterator; - private final RangeCalculator[] rangeCalculators; - private final HashMap rootMap = new HashMap(); - private final ArrayList> bucketSizesPerLevel; - - /** - * @param importIterator - * Iterator over values to be used in test run - * @param rangeCalculators - * For each level a range calculator which defines how to map - * from hash to bucket - */ - public MultiLevelHashTester(Iterator importIterator, - RangeCalculator[] rangeCalculators) { - this.importIterator = importIterator; - this.rangeCalculators = rangeCalculators; - this.maxLevel = rangeCalculators.length; - this.bucketSizesPerLevel = new ArrayList>( - maxLevel); - - for (int i = 0; i < maxLevel; i++) { - bucketSizesPerLevel.add(i, new TreeMap()); - } - } - - /** - * Run the test by: - Adding values from iterator to map - Creating - * histogram over bucket sizes per level - Printing histogram informations - * - * @param boundaries - * Expected results for each level - */ - public void runTest(BucketBoundaries[] boundaries) { - addValues(); - collectStatistics(rootMap, 0); - if (LOG.isDebugEnabled() == true) { - printStatistics(); - } - checkBoundaries(boundaries); - } - - private void checkBoundaries(BucketBoundaries[] boundaries) { - for (int level = 0; level < boundaries.length; level++) { - int lowerBound = boundaries[level].getLowerBound(); - int upperBound = boundaries[level].getUpperBound(); - int bucketCountInLevel = 0; - int bucketCountOutOfRange = 0; - - SortedMap levelMap = bucketSizesPerLevel - .get(level); - Iterator bucketSizeIterator = levelMap.keySet().iterator(); - - while (bucketSizeIterator.hasNext()) { - int bucketSize = bucketSizeIterator.next(); - if (bucketSize != 0) { - int countForBucketSize = levelMap.get(bucketSize); - bucketCountInLevel += countForBucketSize; - if (lowerBound > bucketSize || upperBound < bucketSize) { - bucketCountOutOfRange += countForBucketSize; - } - - } - } - double bucketsOutOfRange = (double) bucketCountOutOfRange - / (double) bucketCountInLevel; - double maxBucketsOutOfRange = boundaries[level] - .getPercentOutOfRange(); - Assert.assertTrue("More than " + (maxBucketsOutOfRange * 100) - + "% of buckets out of range in level " + level, - bucketsOutOfRange <= maxBucketsOutOfRange); - - int maxEmpty = boundaries[level].getMaxEmpty(); - Assert.assertTrue( - "More than " + maxEmpty + " empty buckets in level " - + level, - (maxEmpty == BucketBoundaries.MAX_EMPTY_UNBOUNDED) - || (levelMap.get(0) <= boundaries[level] - .getMaxEmpty())); - } - } - - /** - * Find for each value the right bucket on the deepest level and increase - * its count - */ - @SuppressWarnings("unchecked") - private void addValues() { - - while (importIterator.hasNext()) { - int nextValue = importIterator.next(); - - HashMap mapForCurrentLevel = rootMap; - - for (int i = 0; i < maxLevel - 1; i++) { - int hashValue = MutableHashTable.hash(nextValue, i); - int bucket = rangeCalculators[i].getBucket(hashValue); - Object nextObject = mapForCurrentLevel.get(bucket); - if (nextObject == null) { - HashMap mapForNextLevel = new HashMap(); - mapForCurrentLevel.put(bucket, mapForNextLevel); - mapForCurrentLevel = mapForNextLevel; - - } else { - mapForCurrentLevel = (HashMap) nextObject; - } - } - - int lastHashValue = MutableHashTable.hash(nextValue, maxLevel - 1); - int deepestBucketNr = rangeCalculators[maxLevel - 1] - .getBucket(lastHashValue); - Object countOnDeepestLevel = mapForCurrentLevel - .get(deepestBucketNr); - if (countOnDeepestLevel == null) { - mapForCurrentLevel.put(deepestBucketNr, 1); - } else { - mapForCurrentLevel.put(deepestBucketNr, - ((Integer) countOnDeepestLevel) + 1); - } - - } - } - - private void printStatistics() { - for (int level = 0; level < maxLevel; level++) { - int bucketCountInLevel = 0; - - SortedMap levelMap = bucketSizesPerLevel - .get(level); - Iterator bucketSizeIterator = levelMap.keySet().iterator(); - - LOG.debug("Statistics for level: " + level); - LOG.debug("----------------------------------------------"); - LOG.debug(""); - LOG.debug("Bucket Size | Count"); - LOG.debug("------------------------"); - - int i = 0; - while (bucketSizeIterator.hasNext()) { - int bucketSize = bucketSizeIterator.next(); - if (bucketSize != 0) { - int countForBucketSize = levelMap.get(bucketSize); - bucketCountInLevel += countForBucketSize; - Formatter formatter = new Formatter(); - formatter.format(" %10d | %10d", bucketSize, countForBucketSize); - - if (levelMap.size() < 20 || i < 3 || i >= (levelMap.size() - 3)) { - LOG.debug(formatter.out().toString()); - } else if (levelMap.size() / 2 == i) { - LOG.debug(" .. | .."); - LOG.debug(formatter.out().toString()); - LOG.debug(" .. | .."); - } - i++; - formatter.close(); - } - } - - LOG.debug(""); - LOG.debug("Number of non-empty buckets in level: " - + bucketCountInLevel); - LOG.debug("Number of empty buckets in level : " - + levelMap.get(0)); - LOG.debug("Number of different bucket sizes : " - + (levelMap.size() - 1)); - LOG.debug(""); - LOG.debug(""); - LOG.debug(""); - } - } - - /** - * Create histogram over bucket sizes - * - * @param map - * Map to be analyzed - * @param level - * Level on which the map is located in - * @return The total count of hashed values in the map - */ - private int collectStatistics(HashMap map, int level) { - SortedMap bucketSizesForLevel = bucketSizesPerLevel - .get(level); - - Iterator bucketIterator = map.values().iterator(); - int bucketCount = 0; - int totalValueCount = 0; - - while (bucketIterator.hasNext()) { - bucketCount++; - - Integer hashValuesInBucket; - // If we are already on the deepest level, get the count in the - // bucket, otherwise - // recursively examine the subtree - if (level == maxLevel - 1) { - hashValuesInBucket = (Integer) bucketIterator.next(); - } else { - @SuppressWarnings("unchecked") - HashMap nextMap = (HashMap) bucketIterator - .next(); - hashValuesInBucket = collectStatistics(nextMap, level + 1); - } - totalValueCount += hashValuesInBucket; - Integer countOfBucketSizes = bucketSizesForLevel - .get(hashValuesInBucket); - if (countOfBucketSizes == null) { - countOfBucketSizes = 1; - } else { - countOfBucketSizes += 1; - } - bucketSizesForLevel.put(hashValuesInBucket, countOfBucketSizes); - } - - Integer countOfEmptyBuckets = bucketSizesForLevel.get(0); - if (countOfEmptyBuckets == null) { - countOfEmptyBuckets = rangeCalculators[level].getBucketCount() - - bucketCount; - } else { - countOfEmptyBuckets += rangeCalculators[level].getBucketCount() - - bucketCount; - } - bucketSizesForLevel.put(0, countOfEmptyBuckets); - - return totalValueCount; - } - - /** - * Expected results for bucket sizes per level - * - * - */ - static class BucketBoundaries { - - public static final int MAX_EMPTY_UNBOUNDED = -1; - private int lowerBound; - private int upperBound; - private int maxEmpty; - private double percentOutOfRange; - - /** - * - * - * @param lowerBound Lower bound for bucket sizes - * @param upperBound Upper bound for bucket sizes - * @param maxEmpty Maximum number of empty buckets - * @param percentOutOfRange Maximum percentage of buckets out of range - */ - public BucketBoundaries(int lowerBound, int upperBound, int maxEmpty, - double percentOutOfRange) { - this.lowerBound = lowerBound; - this.upperBound = upperBound; - this.maxEmpty = maxEmpty; - this.percentOutOfRange = percentOutOfRange; - } - - /** - * - * @return Lower bound for bucket sizes - */ - public int getLowerBound() { - return lowerBound; - } - - /** - * - * @return Upper bound for bucket sizes - */ - public int getUpperBound() { - return upperBound; - } - - /** - * - * @return Maximum number of empty buckets - */ - public int getMaxEmpty() { - return maxEmpty; - } - - /** - * - * @return Maximum percentage of buckets out of range - */ - public double getPercentOutOfRange() { - return percentOutOfRange; - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/LastBitsToRange.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/LastBitsToRange.java deleted file mode 100644 index 6b7caa183f7fc..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/LastBitsToRange.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.runtime.operators.hash.util; - -/** - * Takes the last m bits of a hash as bucket number - * - * - */ -public class LastBitsToRange implements RangeCalculator { - - private final int mask; - private final int bucketCount; - - /** - * Create object which calculates bucket number according to the last m bits - * - * @param numberOfLastBits - * Number of bits to be used for bucket calculation - */ - public LastBitsToRange(int numberOfLastBits) { - bucketCount = (int) Math.pow(2, numberOfLastBits); - mask = bucketCount - 1; - } - - - @Override - public int getBucket(int hash) { - return hash & mask; - } - - - public int getBucketCount() { - return bucketCount; - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RandomIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RandomIterator.java deleted file mode 100644 index 08e504b20848f..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RandomIterator.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.runtime.operators.hash.util; - -import java.util.Iterator; -import java.util.Random; - -/** - * Gives a number of random integer values - * - * - */ -public class RandomIterator implements Iterator { - - private final Random randomGenerator; - private final int count; - private int currentCount; - - /** - * Creates an iterator which gives random integer numbers - * - * @param seed - * Seed for random generator - * @param count - * Number of values to be returned - */ - public RandomIterator(long seed, int count) { - this.randomGenerator = new Random(seed); - this.count = count; - this.currentCount = 0; - } - - @Override - public boolean hasNext() { - return (currentCount < count); - } - - @Override - public Integer next() { - currentCount++; - return randomGenerator.nextInt(); - } - - @Override - public void remove() { - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeCalculator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeCalculator.java deleted file mode 100644 index 0f0162d4388ee..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeCalculator.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.runtime.operators.hash.util; - -/** - * Interface for defining which hash code will be placed in which bucket - * - * - */ -public interface RangeCalculator { - - /** - * Calculates the number of the bucket in which the value should be placed - * in - * - * @param hash - * The hash of the value - * @return The number of the bucket - */ - public int getBucket(int hash); - - /** - * Get the total bucket count - * - * @return The total bucket count - */ - public int getBucketCount(); - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeIterator.java deleted file mode 100644 index 82bac8e4881ff..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/RangeIterator.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.runtime.operators.hash.util; - -import java.util.Iterator; - -/** - * Iterates over a range of integer values - * - * - */ -public class RangeIterator implements Iterator { - - private final int maxValue; - private int currentValue; - - /** - * Create an iterator over the range from minValue to maxValue - * - * @param minValue - * Smallest value returned by the iterator - * @param maxValue - * Largest value returned by the iterator - */ - public RangeIterator(int minValue, int maxValue) { - this.maxValue = maxValue; - currentValue = minValue; - } - - @Override - public boolean hasNext() { - return (currentValue < maxValue); - } - - @Override - public Integer next() { - return currentValue++; - } - - @Override - public void remove() { - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/StepRangeIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/StepRangeIterator.java deleted file mode 100644 index 7355754614d9f..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/util/StepRangeIterator.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.runtime.operators.hash.util; - -import java.util.Iterator; - -/** - * Iterates over a range of integer values with a certain step width - * - * - */ -public class StepRangeIterator implements Iterator { - - private final int maxValue; - private int currentValue; - private final int step; - - /** - * Create an iterator over the range from minValue to maxValue with a - * certain step width - * - * @param minValue - * Smallest value returned by the iterator - * @param maxValue - * Largest value returned by the iterator - * @param step - * Step width of the iterator - */ - public StepRangeIterator(int minValue, int maxValue, int step) { - this.maxValue = maxValue; - currentValue = minValue; - this.step = step; - } - - @Override - public boolean hasNext() { - return (currentValue < maxValue); - } - - @Override - public Integer next() { - int temp = currentValue; - currentValue += step; - return temp; - } - - @Override - public void remove() { - } - -} From 925ac1f76bb84986764495407049a77552169d84 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 3 Aug 2015 20:46:17 +0200 Subject: [PATCH 087/175] [FLINK-1916] [FLINK-2361] [runtime] Fix EOFException and entry loss in CompactingHashTable Also a lot of code cleanups in CompactingHashTable --- .../typeutils/GenericPairComparator.java | 3 - .../SolutionSetFastUpdateOutputCollector.java | 12 +- .../io/SolutionSetUpdateOutputCollector.java | 13 +- .../task/AbstractIterativePactTask.java | 3 +- .../hash/AbstractMutableHashTable.java | 4 +- .../operators/hash/CompactingHashTable.java | 629 ++++++++---------- .../operators/hash/InMemoryPartition.java | 6 +- .../operators/hash/MutableHashTable.java | 2 +- .../flink/runtime/util/IntArrayList.java | 4 +- .../hash/CompactingHashTableTest.java | 254 +++++++ .../hash/HashTablePerformanceComparison.java | 15 +- .../operators/hash/MemoryHashTableTest.java | 28 +- 12 files changed, 564 insertions(+), 409 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericPairComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericPairComparator.java index 09d4aeef499e7..e3bf4ca5acbd7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericPairComparator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericPairComparator.java @@ -20,9 +20,6 @@ import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypePairComparator; - public class GenericPairComparator extends TypePairComparator implements Serializable { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetFastUpdateOutputCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetFastUpdateOutputCollector.java index 3b2eda53215e7..f326d89aff9bf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetFastUpdateOutputCollector.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetFastUpdateOutputCollector.java @@ -20,7 +20,6 @@ import java.io.IOException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.hash.CompactingHashTable; import org.apache.flink.util.Collector; @@ -38,23 +37,20 @@ public class SolutionSetFastUpdateOutputCollector implements Collector { private final Collector delegate; private final CompactingHashTable solutionSet; - - private final T tmpHolder; - public SolutionSetFastUpdateOutputCollector(CompactingHashTable solutionSet, TypeSerializer serializer) { - this(solutionSet, serializer, null); + public SolutionSetFastUpdateOutputCollector(CompactingHashTable solutionSet) { + this(solutionSet, null); } - public SolutionSetFastUpdateOutputCollector(CompactingHashTable solutionSet, TypeSerializer serializer, Collector delegate) { + public SolutionSetFastUpdateOutputCollector(CompactingHashTable solutionSet, Collector delegate) { this.solutionSet = solutionSet; this.delegate = delegate; - this.tmpHolder = serializer.createInstance(); } @Override public void collect(T record) { try { - solutionSet.insertOrReplaceRecord(record, tmpHolder); + solutionSet.insertOrReplaceRecord(record); if (delegate != null) { delegate.collect(record); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetUpdateOutputCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetUpdateOutputCollector.java index f400b7e6eba40..c39efa558075d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetUpdateOutputCollector.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SolutionSetUpdateOutputCollector.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.runtime.iterative.io; import java.io.IOException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.hash.CompactingHashTable; import org.apache.flink.util.Collector; @@ -40,23 +38,20 @@ public class SolutionSetUpdateOutputCollector implements Collector { private final Collector delegate; private final CompactingHashTable solutionSet; - - private final T tmpHolder; - public SolutionSetUpdateOutputCollector(CompactingHashTable solutionSet, TypeSerializer serializer) { - this(solutionSet, serializer, null); + public SolutionSetUpdateOutputCollector(CompactingHashTable solutionSet) { + this(solutionSet, null); } - public SolutionSetUpdateOutputCollector(CompactingHashTable solutionSet, TypeSerializer serializer, Collector delegate) { + public SolutionSetUpdateOutputCollector(CompactingHashTable solutionSet, Collector delegate) { this.solutionSet = solutionSet; this.delegate = delegate; - this.tmpHolder = serializer.createInstance(); } @Override public void collect(T record) { try { - solutionSet.insertOrReplaceRecord(record, tmpHolder); + solutionSet.insertOrReplaceRecord(record); if (delegate != null) { delegate.collect(record); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java index 72434e67ef826..67d8f56373bbb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java @@ -329,8 +329,7 @@ protected Collector createSolutionSetUpdateOutputCollector(Collector del if (ss instanceof CompactingHashTable) { @SuppressWarnings("unchecked") CompactingHashTable solutionSet = (CompactingHashTable) ss; - TypeSerializer serializer = getOutputSerializer(); - return new SolutionSetUpdateOutputCollector(solutionSet, serializer, delegate); + return new SolutionSetUpdateOutputCollector(solutionSet, delegate); } else if (ss instanceof JoinHashMap) { @SuppressWarnings("unchecked") diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java index f3e8a22bce69d..86ce0135e9396 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java @@ -61,15 +61,13 @@ public TypeComparator getBuildSideComparator() { public abstract void abort(); - public abstract void buildTable(final MutableObjectIterator input) throws IOException; - public abstract List getFreeMemory(); // ------------- Modifier ------------- public abstract void insert(T record) throws IOException; - public abstract void insertOrReplaceRecord(T record, T tempHolder) throws IOException; + public abstract void insertOrReplaceRecord(T record) throws IOException; // ------------- Accessors ------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java index 6533e194a6f1a..d07c7e38deb62 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java @@ -22,10 +22,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -37,41 +37,20 @@ import org.apache.flink.util.MutableObjectIterator; /** - * An implementation of an in-memory Hash Table for variable-length records. - *

    - * The design of this class follows on many parts the design presented in - * "Hash joins and hash teams in Microsoft SQL Server", by Goetz Graefe et al.. - *

    - *


    - * The layout of the buckets inside a memory segment is as follows: + * A hash table that uses Flink's managed memory and supports replacement of records or + * updates to records. For an overview of the general data structure of the hash table, please + * refer to the description of the {@link org.apache.flink.runtime.operators.hash.MutableHashTable}. + * + *

    The hash table is internally divided into two parts: The hash index, and the partition buffers + * that store the actual records. When records are inserted or updated, the hash table appends the + * records to its corresponding partition, and inserts or updates the entry in the hash index. + * In the case that the hash table runs out of memory, it compacts a partition by walking through the + * hash index and copying all reachable elements into a fresh partition. After that, it releases the + * memory of the partition to compact.

    * - *
    - * +----------------------------- Bucket x ----------------------------
    - * |Partition (1 byte) | reserved (3 bytes) | element count (4 bytes) |
    - * | next-bucket-in-chain-pointer (8 bytes) |
    - * |
    - * |hashCode 1 (4 bytes) | hashCode 2 (4 bytes) | hashCode 3 (4 bytes) |
    - * | ... hashCode n-1 (4 bytes) | hashCode n (4 bytes)
    - * |
    - * |pointer 1 (8 bytes) | pointer 2 (8 bytes) | pointer 3 (8 bytes) |
    - * | ... pointer n-1 (8 bytes) | pointer n (8 bytes)
    - * |
    - * +---------------------------- Bucket x + 1--------------------------
    - * |Partition (1 byte) | reserved (3 bytes) | element count (4 bytes) |
    - * | next-bucket-in-chain-pointer (8 bytes) |
    - * |
    - * |hashCode 1 (4 bytes) | hashCode 2 (4 bytes) | hashCode 3 (4 bytes) |
    - * | ... hashCode n-1 (4 bytes) | hashCode n (4 bytes)
    - * |
    - * |pointer 1 (8 bytes) | pointer 2 (8 bytes) | pointer 3 (8 bytes) |
    - * | ... pointer n-1 (8 bytes) | pointer n (8 bytes)
    - * +-------------------------------------------------------------------
    - * | ...
    - * |
    - * 
    * @param Record type stored in hash table */ -public class CompactingHashTable extends AbstractMutableHashTable{ +public class CompactingHashTable extends AbstractMutableHashTable { private static final Logger LOG = LoggerFactory.getLogger(CompactingHashTable.class); @@ -79,11 +58,10 @@ public class CompactingHashTable extends AbstractMutableHashTable{ // Internal Constants // ------------------------------------------------------------------------ + /** The minimum number of memory segments that the compacting hash table needs to work properly */ private static final int MIN_NUM_MEMORY_SEGMENTS = 33; - /** - * The maximum number of partitions - */ + /** The maximum number of partitions */ private static final int MAX_NUM_PARTITIONS = 32; /** @@ -91,7 +69,7 @@ public class CompactingHashTable extends AbstractMutableHashTable{ * used to determine the ratio of the number of memory segments intended for partition * buffers and the number of memory segments in the hash-table structure. */ - private static final int DEFAULT_RECORD_LEN = 24; //FIXME maybe find a better default + private static final int DEFAULT_RECORD_LEN = 24; /** * The length of the hash code stored in the bucket. @@ -155,14 +133,14 @@ public class CompactingHashTable extends AbstractMutableHashTable{ // Members // ------------------------------------------------------------------------ - /** - * The free memory segments currently available to the hash join. - */ + /** The lock to synchronize state changes on */ + private final Object stateLock = new Object(); + + /** The free memory segments currently available to the hash join. */ private final ArrayList availableMemory; - /** - * The size of the segments used by the hash join buckets. All segments must be of equal size to ease offset computations. - */ + /** The size of the segments used by the hash join buckets. + * All segments must be of equal size to ease offset computations. */ private final int segmentSize; /** @@ -173,63 +151,59 @@ public class CompactingHashTable extends AbstractMutableHashTable{ */ private final int bucketsPerSegmentMask; - /** - * The number of bits that describe the position of a bucket in a memory segment. Computed as log2(bucketsPerSegment). - */ + /** The number of bits that describe the position of a bucket in a memory segment. + * Computed as log2(bucketsPerSegment). */ private final int bucketsPerSegmentBits; - /** - * An estimate for the average record length. - */ + /** An estimate for the average record length. */ private final int avgRecordLen; + + private final int pageSizeInBits; // ------------------------------------------------------------------------ - /** - * The partitions of the hash table. - */ + /** The partitions of the hash table. */ private final ArrayList> partitions; - /** - * The array of memory segments that contain the buckets which form the actual hash-table - * of hash-codes and pointers to the elements. - */ + /** The array of memory segments that contain the buckets which form the actual hash-table + * of hash-codes and pointers to the elements. */ private MemorySegment[] buckets; - /** - * temporary storage for partition compaction (always attempts to allocate as many segments as the largest partition) - */ + /** Temporary storage for partition compaction (always attempts to allocate + * as many segments as the largest partition) */ private InMemoryPartition compactionMemory; - /** - * The number of buckets in the current table. The bucket array is not necessarily fully - * used, when not all buckets that would fit into the last segment are actually used. - */ + /** The number of buckets in the current table. The bucket array is not necessarily fully + * used, when not all buckets that would fit into the last segment are actually used. */ private int numBuckets; - /** - * flag necessary so a resize is never triggered during a resize since the code paths are interleaved - */ - private boolean isResizing = false; + /** Flag to interrupt closed loops */ + private boolean running = true; + + /** Flag to mark the table as open / closed */ + private boolean closed; - private AtomicBoolean closed = new AtomicBoolean(); + /** Flag necessary so a resize is never triggered during a resize since the code paths are interleaved */ + private boolean isResizing; - private boolean running = true; - - private int pageSizeInBits; // ------------------------------------------------------------------------ // Construction and Teardown // ------------------------------------------------------------------------ - public CompactingHashTable(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, List memorySegments) - { + public CompactingHashTable(TypeSerializer buildSideSerializer, + TypeComparator buildSideComparator, + List memorySegments) { this(buildSideSerializer, buildSideComparator, memorySegments, DEFAULT_RECORD_LEN); } - public CompactingHashTable(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, List memorySegments, int avgRecordLen) - { + public CompactingHashTable(TypeSerializer buildSideSerializer, + TypeComparator buildSideComparator, + List memorySegments, + int avgRecordLen) { + super(buildSideSerializer, buildSideComparator); + // some sanity checks first if (memorySegments == null) { throw new NullPointerException(); @@ -252,6 +226,9 @@ public CompactingHashTable(TypeSerializer buildSideSerializer, TypeComparator if ( (this.segmentSize & this.segmentSize - 1) != 0) { throw new IllegalArgumentException("Hash Table requires buffers whose size is a power of 2."); } + + this.pageSizeInBits = MathUtils.log2strict(this.segmentSize); + int bucketsPerSegment = this.segmentSize >> NUM_INTRA_BUCKET_BITS; if (bucketsPerSegment == 0) { throw new IllegalArgumentException("Hash Table requires buffers of at least " + HASH_BUCKET_SIZE + " bytes."); @@ -262,22 +239,26 @@ public CompactingHashTable(TypeSerializer buildSideSerializer, TypeComparator this.partitions = new ArrayList>(); // because we allow to open and close multiple times, the state is initially closed - this.closed.set(true); + this.closed = true; + // so far no partition has any MemorySegments } // ------------------------------------------------------------------------ - // Life-Cycle + // life cycle // ------------------------------------------------------------------------ /** - * Build the hash table + * Initialize the hash table */ + @Override public void open() { - // sanity checks - if (!this.closed.compareAndSet(true, false)) { - throw new IllegalStateException("Hash Table cannot be opened, because it is currently not closed."); + synchronized (stateLock) { + if (!closed) { + throw new IllegalStateException("currently not closed."); + } + closed = false; } // create the partitions @@ -290,7 +271,6 @@ public void open() { initTable(numBuckets, (byte) partitionFanOut); } - /** * Closes the hash table. This effectively releases all internal structures and closes all @@ -299,10 +279,14 @@ public void open() { * all resources that are currently held by the hash join. If another process still access the hash * table after close has been called no operations will be performed. */ + @Override public void close() { // make sure that we close only once - if (!this.closed.compareAndSet(false, true)) { - return; + synchronized (this.stateLock) { + if (this.closed) { + return; + } + this.closed = true; } LOG.debug("Closing hash table and releasing resources."); @@ -313,45 +297,41 @@ public void close() { // clear the memory in the partitions clearPartitions(); } - + + @Override public void abort() { this.running = false; - LOG.debug("Cancelling hash table operations."); } - + + @Override public List getFreeMemory() { - if (!this.closed.get()) { + if (!this.closed) { throw new IllegalStateException("Cannot return memory while join is open."); } return this.availableMemory; } - - - public void buildTable(final MutableObjectIterator input) throws IOException { - T record = this.buildSideSerializer.createInstance(); - - // go over the complete input and insert every element into the hash table - while (this.running && ((record = input.next(record)) != null)) { - insert(record); - } - } + // ------------------------------------------------------------------------ + // adding data to the hash table + // ------------------------------------------------------------------------ + public void buildTableWithUniqueKey(final MutableObjectIterator input) throws IOException { - T record = this.buildSideSerializer.createInstance(); - T tmp = this.buildSideSerializer.createInstance(); - // go over the complete input and insert every element into the hash table - while (this.running && ((record = input.next(record)) != null)) { - insertOrReplaceRecord(record, tmp); + + T value; + while (this.running && (value = input.next()) != null) { + insertOrReplaceRecord(value); } } - + + @Override public final void insert(T record) throws IOException { - if(this.closed.get()) { + if (this.closed) { return; } + final int hashCode = hash(this.buildSideComparator.hash(record)); final int posHashCode = hashCode % this.numBuckets; @@ -364,60 +344,8 @@ public final void insert(T record) throws IOException { final int partitionNumber = bucket.get(bucketInSegmentPos + HEADER_PARTITION_OFFSET); InMemoryPartition partition = this.partitions.get(partitionNumber); - - long pointer; - try { - pointer = partition.appendRecord(record); - if((pointer >> this.pageSizeInBits) > this.compactionMemory.getBlockCount()) { - this.compactionMemory.allocateSegments((int)(pointer >> this.pageSizeInBits)); - } - } catch (EOFException e) { - try { - compactPartition(partitionNumber); - // retry append - partition = this.partitions.get(partitionNumber); // compaction invalidates reference - pointer = partition.appendRecord(record); - } catch (EOFException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } catch (IndexOutOfBoundsException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } - } catch (IndexOutOfBoundsException e1) { - try { - compactPartition(partitionNumber); - // retry append - partition = this.partitions.get(partitionNumber); // compaction invalidates reference - pointer = partition.appendRecord(record); - } catch (EOFException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } catch (IndexOutOfBoundsException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } - } - insertBucketEntryFromStart(partition, bucket, bucketInSegmentPos, hashCode, pointer); - } - - - @Override - public HashTableProber getProber(TypeComparator probeSideComparator, TypePairComparator pairComparator) { - return new HashTableProber(probeSideComparator, pairComparator); - } - - /** - * - * @return Iterator over hash table - * @see EntryIterator - */ - public MutableObjectIterator getEntryIterator() { - return new EntryIterator(this); + long pointer = insertRecordIntoPartition(record, partition, false); + insertBucketEntryFromStart(bucket, bucketInSegmentPos, hashCode, pointer, partitionNumber); } /** @@ -425,11 +353,10 @@ public MutableObjectIterator getEntryIterator() { * May trigger expensive compaction. * * @param record record to insert or replace - * @param tempHolder instance of T that will be overwritten * @throws IOException */ - public void insertOrReplaceRecord(T record, T tempHolder) throws IOException { - if(this.closed.get()) { + public void insertOrReplaceRecord(T record) throws IOException { + if (this.closed) { return; } @@ -437,14 +364,15 @@ public void insertOrReplaceRecord(T record, T tempHolder) throws IOException { final int posHashCode = searchHashCode % this.numBuckets; // get the bucket for the given hash code - MemorySegment originalBucket = this.buckets[posHashCode >> this.bucketsPerSegmentBits]; - int originalBucketOffset = (posHashCode & this.bucketsPerSegmentMask) << NUM_INTRA_BUCKET_BITS; + final MemorySegment originalBucket = this.buckets[posHashCode >> this.bucketsPerSegmentBits]; + final int originalBucketOffset = (posHashCode & this.bucketsPerSegmentMask) << NUM_INTRA_BUCKET_BITS; + MemorySegment bucket = originalBucket; int bucketInSegmentOffset = originalBucketOffset; // get the basic characteristics of the bucket final int partitionNumber = bucket.get(bucketInSegmentOffset + HEADER_PARTITION_OFFSET); - InMemoryPartition partition = this.partitions.get(partitionNumber); + final InMemoryPartition partition = this.partitions.get(partitionNumber); final MemorySegment[] overflowSegments = partition.overflowSegments; this.buildSideComparator.setReference(record); @@ -471,58 +399,12 @@ public void insertOrReplaceRecord(T record, T tempHolder) throws IOException { numInSegment++; // deserialize the key to check whether it is really equal, or whether we had only a hash collision - try { - tempHolder = partition.readRecordAt(pointer, tempHolder); - if (this.buildSideComparator.equalToReference(tempHolder)) { - long newPointer = partition.appendRecord(record); - bucket.putLong(pointerOffset, newPointer); - partition.setCompaction(false); - if((newPointer >> this.pageSizeInBits) > this.compactionMemory.getBlockCount()) { - this.compactionMemory.allocateSegments((int)(newPointer >> this.pageSizeInBits)); - } - return; - } - } catch (EOFException e) { - // system is out of memory so we attempt to reclaim memory with a copy compact run - long newPointer; - try { - compactPartition(partition.getPartitionNumber()); - // retry append - partition = this.partitions.get(partitionNumber); // compaction invalidates reference - newPointer = partition.appendRecord(record); - } catch (EOFException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } catch (IndexOutOfBoundsException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } - bucket.putLong(pointerOffset, newPointer); - return; - } catch (IndexOutOfBoundsException e) { - // system is out of memory so we attempt to reclaim memory with a copy compact run - long newPointer; - try { - compactPartition(partition.getPartitionNumber()); - // retry append - partition = this.partitions.get(partitionNumber); // compaction invalidates reference - newPointer = partition.appendRecord(record); - } catch (EOFException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } catch (IndexOutOfBoundsException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } + T valueAtPosition = partition.readRecordAt(pointer); + if (this.buildSideComparator.equalToReference(valueAtPosition)) { + long newPointer = insertRecordIntoPartition(record, partition, true); bucket.putLong(pointerOffset, newPointer); return; - } catch (IOException e) { - throw new RuntimeException("Error deserializing record from the hashtable: " + e.getMessage(), e); - } + } } else { numInSegment++; @@ -532,28 +414,89 @@ public void insertOrReplaceRecord(T record, T tempHolder) throws IOException { // this segment is done. check if there is another chained bucket long newForwardPointer = bucket.getLong(bucketInSegmentOffset + HEADER_FORWARD_OFFSET); if (newForwardPointer == BUCKET_FORWARD_POINTER_NOT_SET) { + // nothing found. append and insert - long pointer = partition.appendRecord(record); - //insertBucketEntryFromStart(partition, originalBucket, originalBucketOffset, searchHashCode, pointer); - insertBucketEntryFromSearch(partition, originalBucket, bucket, originalBucketOffset, bucketInSegmentOffset, countInSegment, currentForwardPointer, searchHashCode, pointer); - if((pointer >> this.pageSizeInBits) > this.compactionMemory.getBlockCount()) { - this.compactionMemory.allocateSegments((int)(pointer >> this.pageSizeInBits)); + long pointer = insertRecordIntoPartition(record, partition, false); + + if (countInSegment < NUM_ENTRIES_PER_BUCKET) { + // we are good in our current bucket, put the values + bucket.putInt(bucketInSegmentOffset + BUCKET_HEADER_LENGTH + (countInSegment * HASH_CODE_LEN), searchHashCode); // hash code + bucket.putLong(bucketInSegmentOffset + BUCKET_POINTER_START_OFFSET + (countInSegment * POINTER_LEN), pointer); // pointer + bucket.putInt(bucketInSegmentOffset + HEADER_COUNT_OFFSET, countInSegment + 1); // update count + } + else { + insertBucketEntryFromStart(originalBucket, originalBucketOffset, searchHashCode, pointer, partitionNumber); } return; } final int overflowSegNum = (int) (newForwardPointer >>> 32); bucket = overflowSegments[overflowSegNum]; - bucketInSegmentOffset = (int) (newForwardPointer & 0xffffffff); + bucketInSegmentOffset = (int) newForwardPointer; countInSegment = bucket.getInt(bucketInSegmentOffset + HEADER_COUNT_OFFSET); posInSegment = bucketInSegmentOffset + BUCKET_HEADER_LENGTH; numInSegment = 0; currentForwardPointer = newForwardPointer; } } + + private long insertRecordIntoPartition(T record, InMemoryPartition partition, + boolean fragments) throws IOException { + try { + long pointer = partition.appendRecord(record); + if (fragments) { + partition.setIsCompacted(false); + } + if ((pointer >> this.pageSizeInBits) > this.compactionMemory.getBlockCount()) { + this.compactionMemory.allocateSegments((int)(pointer >> this.pageSizeInBits)); + } + return pointer; + } + catch (Exception e) { + if (e instanceof EOFException || e instanceof IndexOutOfBoundsException) { + // this indicates an out of memory situation + try { + final int partitionNumber = partition.getPartitionNumber(); + compactPartition(partitionNumber); + + // retry append + partition = this.partitions.get(partitionNumber); // compaction invalidates reference + long newPointer = partition.appendRecord(record); + if ((newPointer >> this.pageSizeInBits) > this.compactionMemory.getBlockCount()) { + this.compactionMemory.allocateSegments((int)(newPointer >> this.pageSizeInBits)); + } + return newPointer; + } + catch (EOFException ex) { + throw new RuntimeException("Memory ran out. Compaction failed. " + + getMemoryConsumptionString() + " Message: " + ex.getMessage()); + } + catch (IndexOutOfBoundsException ex) { + throw new RuntimeException("Memory ran out. Compaction failed. " + + getMemoryConsumptionString() + " Message: " + ex.getMessage()); + } + } + else if (e instanceof IOException) { + throw (IOException) e; + } + else //noinspection ConstantConditions + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } + else { + throw new RuntimeException("Writing record to compacting hash table failed", e); + } + } + } - private void insertBucketEntryFromStart(InMemoryPartition p, MemorySegment bucket, - int bucketInSegmentPos, int hashCode, long pointer) + + /** + * IMPORTANT!!! We pass only the partition number, because we must make sure we get a fresh + * partition reference. The partition reference used during search for the key may have become + * invalid during the compaction. + */ + private void insertBucketEntryFromStart(MemorySegment bucket, int bucketInSegmentPos, + int hashCode, long pointer, int partitionNumber) throws IOException { boolean checkForResize = false; @@ -564,8 +507,11 @@ private void insertBucketEntryFromStart(InMemoryPartition p, MemorySegment bu bucket.putInt(bucketInSegmentPos + BUCKET_HEADER_LENGTH + (count * HASH_CODE_LEN), hashCode); // hash code bucket.putLong(bucketInSegmentPos + BUCKET_POINTER_START_OFFSET + (count * POINTER_LEN), pointer); // pointer bucket.putInt(bucketInSegmentPos + HEADER_COUNT_OFFSET, count + 1); // update count - } else { + } + else { // we need to go to the overflow buckets + final InMemoryPartition p = this.partitions.get(partitionNumber); + final long originalForwardPointer = bucket.getLong(bucketInSegmentPos + HEADER_FORWARD_OFFSET); final long forwardForNewBucket; @@ -573,7 +519,7 @@ private void insertBucketEntryFromStart(InMemoryPartition p, MemorySegment bu // forward pointer set final int overflowSegNum = (int) (originalForwardPointer >>> 32); - final int segOffset = (int) (originalForwardPointer & 0xffffffff); + final int segOffset = (int) originalForwardPointer; final MemorySegment seg = p.overflowSegments[overflowSegNum]; final int obCount = seg.getInt(segOffset + HEADER_COUNT_OFFSET); @@ -635,31 +581,42 @@ private void insertBucketEntryFromStart(InMemoryPartition p, MemorySegment bu bucket.putLong(bucketInSegmentPos + HEADER_FORWARD_OFFSET, pointerToNewBucket); // finally, insert the values into the overflow buckets - overflowSeg.putInt(overflowBucketOffset + BUCKET_HEADER_LENGTH, hashCode); // hash code + overflowSeg.putInt(overflowBucketOffset + BUCKET_HEADER_LENGTH, hashCode); // hash code overflowSeg.putLong(overflowBucketOffset + BUCKET_POINTER_START_OFFSET, pointer); // pointer // set the count to one - overflowSeg.putInt(overflowBucketOffset + HEADER_COUNT_OFFSET, 1); - if(checkForResize && !this.isResizing) { + overflowSeg.putInt(overflowBucketOffset + HEADER_COUNT_OFFSET, 1); + + if (checkForResize && !this.isResizing) { // check if we should resize buckets - if(this.buckets.length <= getOverflowSegmentCount()) { + if (this.buckets.length <= getOverflowSegmentCount()) { resizeHashTable(); } } } } - - private void insertBucketEntryFromSearch(InMemoryPartition partition, MemorySegment originalBucket, MemorySegment currentBucket, int originalBucketOffset, int currentBucketOffset, int countInCurrentBucket, long currentForwardPointer, int hashCode, long pointer) throws IOException { + + /** + * IMPORTANT!!! We pass only the partition number, because we must make sure we get a fresh + * partition reference. The partition reference used during search for the key may have become + * invalid during the compaction. + */ + private void insertBucketEntryFromSearch(MemorySegment originalBucket, MemorySegment currentBucket, + int originalBucketOffset, int currentBucketOffset, + int countInCurrentBucket, long originalForwardPointer, + int hashCode, long pointer, int partitionNumber) throws IOException { boolean checkForResize = false; if (countInCurrentBucket < NUM_ENTRIES_PER_BUCKET) { // we are good in our current bucket, put the values - currentBucket.putInt(currentBucketOffset + BUCKET_HEADER_LENGTH + (countInCurrentBucket * HASH_CODE_LEN), hashCode); // hash code + currentBucket.putInt(currentBucketOffset + BUCKET_HEADER_LENGTH + (countInCurrentBucket * HASH_CODE_LEN), hashCode); // hash code currentBucket.putLong(currentBucketOffset + BUCKET_POINTER_START_OFFSET + (countInCurrentBucket * POINTER_LEN), pointer); // pointer currentBucket.putInt(currentBucketOffset + HEADER_COUNT_OFFSET, countInCurrentBucket + 1); // update count - } else { - // we need a new overflow bucket + } + else { + // we go to a new overflow bucket + final InMemoryPartition partition = this.partitions.get(partitionNumber); MemorySegment overflowSeg; - final int overflowBucketNum; + final int overflowSegmentNum; final int overflowBucketOffset; // first, see if there is space for an overflow bucket remaining in the last overflow segment @@ -667,7 +624,7 @@ private void insertBucketEntryFromSearch(InMemoryPartition partition, MemoryS // no space left in last bucket, or no bucket yet, so create an overflow segment overflowSeg = getNextBuffer(); overflowBucketOffset = 0; - overflowBucketNum = partition.numOverflowSegments; + overflowSegmentNum = partition.numOverflowSegments; // add the new overflow segment if (partition.overflowSegments.length <= partition.numOverflowSegments) { @@ -678,10 +635,11 @@ private void insertBucketEntryFromSearch(InMemoryPartition partition, MemoryS partition.overflowSegments[partition.numOverflowSegments] = overflowSeg; partition.numOverflowSegments++; checkForResize = true; - } else { + } + else { // there is space in the last overflow segment - overflowBucketNum = partition.numOverflowSegments - 1; - overflowSeg = partition.overflowSegments[overflowBucketNum]; + overflowSegmentNum = partition.numOverflowSegments - 1; + overflowSeg = partition.overflowSegments[overflowSegmentNum]; overflowBucketOffset = partition.nextOverflowBucket << NUM_INTRA_BUCKET_BITS; } @@ -690,10 +648,11 @@ private void insertBucketEntryFromSearch(InMemoryPartition partition, MemoryS partition.nextOverflowBucket = (partition.nextOverflowBucket == this.bucketsPerSegmentMask ? 0 : partition.nextOverflowBucket + 1); // insert the new overflow bucket in the chain of buckets + // 1) set the old forward pointer // 2) let the bucket in the main table point to this one - overflowSeg.putLong(overflowBucketOffset + HEADER_FORWARD_OFFSET, currentForwardPointer); - final long pointerToNewBucket = (((long) overflowBucketNum) << 32) | ((long) overflowBucketOffset); + overflowSeg.putLong(overflowBucketOffset + HEADER_FORWARD_OFFSET, originalForwardPointer); + final long pointerToNewBucket = (((long) overflowSegmentNum) << 32) | ((long) overflowBucketOffset); originalBucket.putLong(originalBucketOffset + HEADER_FORWARD_OFFSET, pointerToNewBucket); // finally, insert the values into the overflow buckets @@ -710,16 +669,33 @@ private void insertBucketEntryFromSearch(InMemoryPartition partition, MemoryS } } } + + // -------------------------------------------------------------------------------------------- + // Access to the entries + // -------------------------------------------------------------------------------------------- + + @Override + public HashTableProber getProber(TypeComparator probeSideComparator, TypePairComparator pairComparator) { + return new HashTableProber(probeSideComparator, pairComparator); + } + + /** + * + * @return Iterator over hash table + * @see EntryIterator + */ + public MutableObjectIterator getEntryIterator() { + return new EntryIterator(this); + } // -------------------------------------------------------------------------------------------- - // Setup and Tear Down of Structures + // Setup and Tear Down of Structures // -------------------------------------------------------------------------------------------- private void createPartitions(int numPartitions) { this.partitions.clear(); ListMemorySegmentSource memSource = new ListMemorySegmentSource(this.availableMemory); - this.pageSizeInBits = MathUtils.log2strict(this.segmentSize); for (int i = 0; i < numPartitions; i++) { this.partitions.add(new InMemoryPartition(this.buildSideSerializer, i, memSource, this.segmentSize, pageSizeInBits)); @@ -728,8 +704,7 @@ private void createPartitions(int numPartitions) { } private void clearPartitions() { - for (int i = 0; i < this.partitions.size(); i++) { - InMemoryPartition p = this.partitions.get(i); + for (InMemoryPartition p : this.partitions) { p.clearAllMemory(this.availableMemory); } this.partitions.clear(); @@ -768,14 +743,14 @@ private void releaseTable() { // set the counters back this.numBuckets = 0; if (this.buckets != null) { - for (int i = 0; i < this.buckets.length; i++) { - this.availableMemory.add(this.buckets[i]); + for (MemorySegment bucket : this.buckets) { + this.availableMemory.add(bucket); } this.buckets = null; } } - private final MemorySegment getNextBuffer() { + private MemorySegment getNextBuffer() { // check if the list directly offers memory int s = this.availableMemory.size(); if (s > 0) { @@ -799,7 +774,7 @@ private final MemorySegment getNextBuffer() { * @param numBuffers The number of buffers available. * @return The number of partitions to use. */ - private static final int getPartitioningFanOutNoEstimates(int numBuffers) { + private static int getPartitioningFanOutNoEstimates(int numBuffers) { return Math.max(10, Math.min(numBuffers / 10, MAX_NUM_PARTITIONS)); } @@ -807,14 +782,13 @@ private static final int getPartitioningFanOutNoEstimates(int numBuffers) { * @return String containing a summary of the memory consumption for error messages */ private String getMemoryConsumptionString() { - String result = new String("numPartitions: " + this.partitions.size() + + return "numPartitions: " + this.partitions.size() + " minPartition: " + getMinPartition() + " maxPartition: " + getMaxPartition() + " number of overflow segments: " + getOverflowSegmentCount() + " bucketSize: " + this.buckets.length + - " Overall memory: " + getSize() + - " Partition memory: " + getPartitionSize()); - return result; + " Overall memory: " + getSize() + + " Partition memory: " + getPartitionSize(); } /** @@ -878,7 +852,7 @@ private int getMinPartition() { */ private int getOverflowSegmentCount() { int result = 0; - for(InMemoryPartition p : this.partitions) { + for (InMemoryPartition p : this.partitions) { result += p.numOverflowSegments; } return result; @@ -890,7 +864,7 @@ private int getOverflowSegmentCount() { * * @return number of buckets */ - private static final int getInitialTableSize(int numBuffers, int bufferSize, int numPartitions, int recordLenBytes) { + private static int getInitialTableSize(int numBuffers, int bufferSize, int numPartitions, int recordLenBytes) { final long totalSize = ((long) bufferSize) * numBuffers; final long numRecordsStorable = totalSize / (recordLenBytes + RECORD_OVERHEAD_BYTES); final long bucketBytes = numRecordsStorable * RECORD_OVERHEAD_BYTES; @@ -908,7 +882,7 @@ private static final int getInitialTableSize(int numBuffers, int bufferSize, int * @param numPartitions number of partitions * @return The hash code for the integer. */ - private static final byte assignPartition(int bucket, byte numPartitions) { + private static byte assignPartition(int bucket, byte numPartitions) { return (byte) (bucket % numPartitions); } @@ -924,26 +898,27 @@ private boolean resizeHashTable() throws IOException { final int newNumSegments = (newNumBuckets + (bucketsPerSegment-1)) / bucketsPerSegment; final int additionalSegments = newNumSegments-this.buckets.length; final int numPartitions = this.partitions.size(); - if(this.availableMemory.size() < additionalSegments) { - for(int i = 0; i < numPartitions; i++) { + + if (this.availableMemory.size() < additionalSegments) { + for (int i = 0; i < numPartitions; i++) { compactPartition(i); if(this.availableMemory.size() >= additionalSegments) { break; } } } - if(this.availableMemory.size() < additionalSegments || this.closed.get()) { + + if (this.availableMemory.size() < additionalSegments || this.closed) { return false; - } else { + } + else { this.isResizing = true; // allocate new buckets final int startOffset = (this.numBuckets * HASH_BUCKET_SIZE) % this.segmentSize; - MemorySegment[] newBuckets = new MemorySegment[additionalSegments]; final int oldNumBuckets = this.numBuckets; final int oldNumSegments = this.buckets.length; MemorySegment[] mergedBuckets = new MemorySegment[newNumSegments]; System.arraycopy(this.buckets, 0, mergedBuckets, 0, this.buckets.length); - System.arraycopy(newBuckets, 0, mergedBuckets, this.buckets.length, newBuckets.length); this.buckets = mergedBuckets; this.numBuckets = newNumBuckets; // initialize all new buckets @@ -951,7 +926,7 @@ private boolean resizeHashTable() throws IOException { final int startSegment = oldSegment ? (oldNumSegments-1) : oldNumSegments; for (int i = startSegment, bucket = oldNumBuckets; i < newNumSegments && bucket < this.numBuckets; i++) { MemorySegment seg; - int bucketOffset = 0; + int bucketOffset; if(oldSegment) { // the first couple of new buckets may be located on an old segment seg = this.buckets[i]; for (int k = (oldNumBuckets % bucketsPerSegment) ; k < bucketsPerSegment && bucket < this.numBuckets; k++, bucket++) { @@ -965,7 +940,7 @@ private boolean resizeHashTable() throws IOException { seg = getNextBuffer(); // go over all buckets in the segment for (int k = 0; k < bucketsPerSegment && bucket < this.numBuckets; k++, bucket++) { - bucketOffset = k * HASH_BUCKET_SIZE; + bucketOffset = k * HASH_BUCKET_SIZE; // initialize the header fields seg.put(bucketOffset + HEADER_PARTITION_OFFSET, assignPartition(bucket, (byte)numPartitions)); seg.putInt(bucketOffset + HEADER_COUNT_OFFSET, 0); @@ -975,19 +950,21 @@ private boolean resizeHashTable() throws IOException { this.buckets[i] = seg; oldSegment = false; // we write on at most one old segment } - int hashOffset = 0; - int hash = 0; - int pointerOffset = 0; - long pointer = 0; + int hashOffset; + int hash; + int pointerOffset; + long pointer; IntArrayList hashList = new IntArrayList(NUM_ENTRIES_PER_BUCKET); LongArrayList pointerList = new LongArrayList(NUM_ENTRIES_PER_BUCKET); IntArrayList overflowHashes = new IntArrayList(64); LongArrayList overflowPointers = new LongArrayList(64); + // go over all buckets and split them between old and new buckets - for(int i = 0; i < numPartitions; i++) { + for (int i = 0; i < numPartitions; i++) { InMemoryPartition partition = this.partitions.get(i); final MemorySegment[] overflowSegments = partition.overflowSegments; - int posHashCode = 0; + + int posHashCode; for (int j = 0, bucket = i; j < this.buckets.length && bucket < oldNumBuckets; j++) { MemorySegment segment = this.buckets[j]; // go over all buckets in the segment belonging to the partition @@ -1021,7 +998,7 @@ private boolean resizeHashTable() throws IOException { } final int overflowSegNum = (int) (forwardPointer >>> 32); segment = overflowSegments[overflowSegNum]; - bucketOffset = (int)(forwardPointer & 0xffffffff); + bucketOffset = (int) forwardPointer; countInSegment = segment.getInt(bucketOffset + HEADER_COUNT_OFFSET); pointerOffset = bucketOffset + BUCKET_POINTER_START_OFFSET; hashOffset = bucketOffset + BUCKET_HEADER_LENGTH; @@ -1038,25 +1015,29 @@ private boolean resizeHashTable() throws IOException { } int newSegmentIndex = (bucket + oldNumBuckets) / bucketsPerSegment; MemorySegment newSegment = this.buckets[newSegmentIndex]; + // we need to avoid overflows in the first run int oldBucketCount = 0; int newBucketCount = 0; - while(!hashList.isEmpty()) { + while (!hashList.isEmpty()) { hash = hashList.removeLast(); pointer = pointerList.removeLong(pointerList.size()-1); posHashCode = hash % this.numBuckets; - if(posHashCode == bucket && oldBucketCount < NUM_ENTRIES_PER_BUCKET) { + if (posHashCode == bucket && oldBucketCount < NUM_ENTRIES_PER_BUCKET) { bucketOffset = (bucket % bucketsPerSegment) * HASH_BUCKET_SIZE; - insertBucketEntryFromStart(partition, segment, bucketOffset, hash, pointer); + insertBucketEntryFromStart(segment, bucketOffset, hash, pointer, partition.getPartitionNumber()); oldBucketCount++; - } else if(posHashCode == (bucket + oldNumBuckets) && newBucketCount < NUM_ENTRIES_PER_BUCKET) { + } + else if (posHashCode == (bucket + oldNumBuckets) && newBucketCount < NUM_ENTRIES_PER_BUCKET) { bucketOffset = ((bucket + oldNumBuckets) % bucketsPerSegment) * HASH_BUCKET_SIZE; - insertBucketEntryFromStart(partition, newSegment, bucketOffset, hash, pointer); + insertBucketEntryFromStart(newSegment, bucketOffset, hash, pointer, partition.getPartitionNumber()); newBucketCount++; - } else if(posHashCode == (bucket + oldNumBuckets) || posHashCode == bucket) { + } + else if (posHashCode == (bucket + oldNumBuckets) || posHashCode == bucket) { overflowHashes.add(hash); overflowPointers.add(pointer); - } else { + } + else { throw new IOException("Accessed wrong bucket. Target: " + bucket + " or " + (bucket + oldNumBuckets) + " Hit: " + posHashCode); } } @@ -1067,9 +1048,9 @@ private boolean resizeHashTable() throws IOException { // reset partition's overflow buckets and reclaim their memory this.availableMemory.addAll(partition.resetOverflowBuckets()); // clear overflow lists - int bucketArrayPos = 0; - int bucketInSegmentPos = 0; - MemorySegment bucket = null; + int bucketArrayPos; + int bucketInSegmentPos; + MemorySegment bucket; while(!overflowHashes.isEmpty()) { hash = overflowHashes.removeLast(); pointer = overflowPointers.removeLong(overflowPointers.size()-1); @@ -1077,7 +1058,7 @@ private boolean resizeHashTable() throws IOException { bucketArrayPos = posHashCode >>> this.bucketsPerSegmentBits; bucketInSegmentPos = (posHashCode & this.bucketsPerSegmentMask) << NUM_INTRA_BUCKET_BITS; bucket = this.buckets[bucketArrayPos]; - insertBucketEntryFromStart(partition, bucket, bucketInSegmentPos, hash, pointer); + insertBucketEntryFromStart(bucket, bucketInSegmentPos, hash, pointer, partition.getPartitionNumber()); } overflowHashes.clear(); overflowPointers.clear(); @@ -1095,7 +1076,7 @@ private boolean resizeHashTable() throws IOException { */ private void compactPartition(final int partitionNumber) throws IOException { // do nothing if table was closed, parameter is invalid or no garbage exists - if(this.closed.get() || partitionNumber >= this.partitions.size() || this.partitions.get(partitionNumber).isCompacted()) { + if (this.closed || partitionNumber >= this.partitions.size() || this.partitions.get(partitionNumber).isCompacted()) { return; } // release all segments owned by compaction partition @@ -1106,9 +1087,9 @@ private void compactPartition(final int partitionNumber) throws IOException { final int numPartitions = this.partitions.size(); InMemoryPartition partition = this.partitions.remove(partitionNumber); MemorySegment[] overflowSegments = partition.overflowSegments; - long pointer = 0L; - int pointerOffset = 0; - int bucketOffset = 0; + long pointer; + int pointerOffset; + int bucketOffset; final int bucketsPerSegment = this.bucketsPerSegmentMask + 1; for (int i = 0, bucket = partitionNumber; i < this.buckets.length && bucket < this.numBuckets; i++) { MemorySegment segment = this.buckets[i]; @@ -1138,7 +1119,7 @@ private void compactPartition(final int partitionNumber) throws IOException { } final int overflowSegNum = (int) (forwardPointer >>> 32); segment = overflowSegments[overflowSegNum]; - bucketOffset = (int)(forwardPointer & 0xffffffff); + bucketOffset = (int) forwardPointer; countInSegment = segment.getInt(bucketOffset + HEADER_COUNT_OFFSET); pointerOffset = bucketOffset + BUCKET_POINTER_START_OFFSET; numInSegment = 0; @@ -1152,7 +1133,7 @@ private void compactPartition(final int partitionNumber) throws IOException { this.partitions.get(partitionNumber).overflowSegments = partition.overflowSegments; this.partitions.get(partitionNumber).numOverflowSegments = partition.numOverflowSegments; this.partitions.get(partitionNumber).nextOverflowBucket = partition.nextOverflowBucket; - this.partitions.get(partitionNumber).setCompaction(true); + this.partitions.get(partitionNumber).setIsCompacted(true); //this.partitions.get(partitionNumber).pushDownPages(); this.compactionMemory = partition; this.compactionMemory.resetRecordCounter(); @@ -1168,22 +1149,6 @@ private void compactPartition(final int partitionNumber) throws IOException { this.compactionMemory.pushDownPages(); } - /** - * Compacts partition but may not reclaim all garbage - * - * @param partitionNumber partition number - * @throws IOException - */ - @SuppressWarnings("unused") - private void fastCompactPartition(int partitionNumber) throws IOException { - // stop if no garbage exists - if(this.partitions.get(partitionNumber).isCompacted()) { - return; - } - //TODO IMPLEMENT ME - return; - } - /** * This function hashes an integer value. It is adapted from Bob Jenkins' website * http://www.burtleburtle.net/bob/hash/integer.html. @@ -1193,7 +1158,7 @@ private void fastCompactPartition(int partitionNumber) throws IOException { * @param code The integer to be hashed. * @return The hash code for the integer. */ - private static final int hash(int code) { + private static int hash(int code) { code = (code + 0x7ed55d16) + (code << 12); code = (code ^ 0xc761c23c) ^ (code >>> 19); code = (code + 0x165667b1) + (code << 5); @@ -1235,7 +1200,7 @@ public T next(T reuse) throws IOException { @Override public T next() throws IOException { - if(done || this.table.closed.get()) { + if (done || this.table.closed) { return null; } else if(!cache.isEmpty()) { return cache.remove(cache.size()-1); @@ -1294,7 +1259,7 @@ private boolean fillCache() throws IOException { } final int overflowSegNum = (int) (forwardPointer >>> 32); bucket = overflowSegments[overflowSegNum]; - bucketOffset = (int)(forwardPointer & 0xffffffff); + bucketOffset = (int) forwardPointer; countInSegment = bucket.getInt(bucketOffset + HEADER_COUNT_OFFSET); posInSegment = bucketOffset + BUCKET_POINTER_START_OFFSET; numInSegment = 0; @@ -1326,7 +1291,7 @@ private HashTableProber(TypeComparator probeTypeComparator, TypePairComparat } public T getMatchFor(PT probeSideRecord, T reuse) { - if(closed.get()) { + if (closed) { return null; } final int searchHashCode = hash(this.probeTypeComparator.hash(probeSideRecord)); @@ -1391,7 +1356,7 @@ public T getMatchFor(PT probeSideRecord, T reuse) { final int overflowSegNum = (int) (forwardPointer >>> 32); bucket = overflowSegments[overflowSegNum]; - bucketInSegmentOffset = (int) (forwardPointer & 0xffffffff); + bucketInSegmentOffset = (int) forwardPointer; countInSegment = bucket.getInt(bucketInSegmentOffset + HEADER_COUNT_OFFSET); posInSegment = bucketInSegmentOffset + BUCKET_HEADER_LENGTH; numInSegment = 0; @@ -1399,7 +1364,7 @@ public T getMatchFor(PT probeSideRecord, T reuse) { } public T getMatchFor(PT probeSideRecord) { - if(closed.get()) { + if (closed) { return null; } final int searchHashCode = hash(this.probeTypeComparator.hash(probeSideRecord)); @@ -1464,7 +1429,7 @@ public T getMatchFor(PT probeSideRecord) { final int overflowSegNum = (int) (forwardPointer >>> 32); bucket = overflowSegments[overflowSegNum]; - bucketInSegmentOffset = (int) (forwardPointer & 0xffffffff); + bucketInSegmentOffset = (int) forwardPointer; countInSegment = bucket.getInt(bucketInSegmentOffset + HEADER_COUNT_OFFSET); posInSegment = bucketInSegmentOffset + BUCKET_HEADER_LENGTH; numInSegment = 0; @@ -1472,49 +1437,11 @@ public T getMatchFor(PT probeSideRecord) { } public void updateMatch(T record) throws IOException { - if(closed.get()) { + if (closed) { return; } - long newPointer; - try { - newPointer = this.partition.appendRecord(record); - } catch (EOFException e) { - // system is out of memory so we attempt to reclaim memory with a copy compact run - try { - int partitionNumber = this.partition.getPartitionNumber(); - compactPartition(partitionNumber); - // retry append - this.partition = partitions.get(partitionNumber); - newPointer = this.partition.appendRecord(record); - } catch (EOFException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } catch (IndexOutOfBoundsException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } - } catch (IndexOutOfBoundsException e) { - // system is out of memory so we attempt to reclaim memory with a copy compact run - try { - int partitionNumber = this.partition.getPartitionNumber(); - compactPartition(partitionNumber); - // retry append - this.partition = partitions.get(partitionNumber); - newPointer = this.partition.appendRecord(record); - } catch (EOFException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } catch (IndexOutOfBoundsException ex) { - throw new RuntimeException("Memory ran out. Compaction failed. " + - getMemoryConsumptionString() + - " Message: " + ex.getMessage()); - } - } + long newPointer = insertRecordIntoPartition(record, this.partition, true); this.bucket.putLong(this.pointerOffsetInBucket, newPointer); - this.partition.setCompaction(false); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java index 7fb997e0fa9c8..ffb66fc3b85e3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java @@ -199,7 +199,7 @@ public boolean isCompacted() { * * @param compacted compaction status */ - public void setCompaction(boolean compacted) { + public void setIsCompacted(boolean compacted) { this.compacted = compacted; } @@ -281,9 +281,9 @@ public void clearAllMemory(List target) { * @param numberOfSegments allocation count */ public void allocateSegments(int numberOfSegments) { - while(getBlockCount() < numberOfSegments) { + while (getBlockCount() < numberOfSegments) { MemorySegment next = this.availableMemory.nextSegment(); - if(next != null) { + if (next != null) { this.partitionPages.add(next); } else { return; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java index 94167969abd18..7f07cfbf2c692 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java @@ -886,7 +886,7 @@ final void insertBucketEntry(final HashPartition p, final MemorySegment // forward pointer set final int overflowSegNum = (int) (originalForwardPointer >>> 32); - final int segOffset = (int) (originalForwardPointer & 0xffffffff); + final int segOffset = (int) originalForwardPointer; final MemorySegment seg = p.overflowSegments[overflowSegNum]; final short obCount = seg.getShort(segOffset + HEADER_COUNT_OFFSET); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/IntArrayList.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/IntArrayList.java index 999c4b0aa26e0..27d958aa2602d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/IntArrayList.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/IntArrayList.java @@ -71,10 +71,12 @@ private void grow(final int length) { public static final IntArrayList EMPTY = new IntArrayList(0) { + @Override public boolean add(int number) { throw new UnsupportedOperationException(); } - + + @Override public int removeLast() { throw new UnsupportedOperationException(); }; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java new file mode 100644 index 0000000000000..e3b697e6d9e2d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java @@ -0,0 +1,254 @@ +/* + * 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.runtime.operators.hash; + +import static org.junit.Assert.*; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongComparator; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.TupleComparator; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.util.MutableObjectIterator; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.List; + +public class CompactingHashTableTest { + + private final TypeSerializer> serializer; + private final TypeComparator> comparator; + + private final TypeComparator probeComparator; + + private final TypePairComparator> pairComparator; + + + public CompactingHashTableTest() { + TypeSerializer[] fieldSerializers = { LongSerializer.INSTANCE, StringSerializer.INSTANCE }; + @SuppressWarnings("unchecked") + Class> clazz = (Class>) (Class) Tuple2.class; + this.serializer = new TupleSerializer>(clazz, fieldSerializers); + + TypeComparator[] comparators = { new LongComparator(true) }; + TypeSerializer[] comparatorSerializers = { LongSerializer.INSTANCE }; + + this.comparator = new TupleComparator>(new int[] {0}, comparators, comparatorSerializers); + + this.probeComparator = new LongComparator(true); + + this.pairComparator = new TypePairComparator>() { + + private long ref; + + @Override + public void setReference(Long reference) { + ref = reference; + } + + @Override + public boolean equalToReference(Tuple2 candidate) { + //noinspection UnnecessaryUnboxing + return candidate.f0.longValue() == ref; + } + + @Override + public int compareToReference(Tuple2 candidate) { + long x = ref; + long y = candidate.f0; + return (x < y) ? -1 : ((x == y) ? 0 : 1); + } + }; + } + + // ------------------------------------------------------------------------ + // tests + // ------------------------------------------------------------------------ + + @Test + public void testHashTableGrowthWithInsert() { + try { + final int numElements = 1000000; + + List memory = getMemory(10000, 32 * 1024); + + // we create a hash table that thinks the records are super large. that makes it choose initially + // a lot of memory for the partition buffers, and start with a smaller hash table. that way + // we trigger a hash table growth early. + CompactingHashTable> table = new CompactingHashTable>( + serializer, comparator, memory, 10000); + table.open(); + + for (long i = 0; i < numElements; i++) { + table.insert(new Tuple2(i, String.valueOf(i))); + } + + // make sure that all elements are contained via the entry iterator + { + BitSet bitSet = new BitSet(numElements); + MutableObjectIterator> iter = table.getEntryIterator(); + Tuple2 next; + while ((next = iter.next()) != null) { + assertNotNull(next.f0); + assertNotNull(next.f1); + assertEquals(next.f0.longValue(), Long.parseLong(next.f1)); + + bitSet.set(next.f0.intValue()); + } + + assertEquals(numElements, bitSet.cardinality()); + } + + // make sure all entries are contained via the prober + { + CompactingHashTable>.HashTableProber proper = + table.getProber(probeComparator, pairComparator); + + for (long i = 0; i < numElements; i++) { + assertNotNull(proper.getMatchFor(i)); + assertNull(proper.getMatchFor(i + numElements)); + } + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + /** + * This test validates that records are not lost via "insertOrReplace()" as in bug [FLINK-2361] + */ + @Test + public void testHashTableGrowthWithInsertOrReplace() { + try { + final int numElements = 1000000; + + List memory = getMemory(10000, 32 * 1024); + + // we create a hash table that thinks the records are super large. that makes it choose initially + // a lot of memory for the partition buffers, and start with a smaller hash table. that way + // we trigger a hash table growth early. + CompactingHashTable> table = new CompactingHashTable>( + serializer, comparator, memory, 10000); + table.open(); + + for (long i = 0; i < numElements; i++) { + table.insertOrReplaceRecord(new Tuple2(i, String.valueOf(i))); + } + + // make sure that all elements are contained via the entry iterator + { + BitSet bitSet = new BitSet(numElements); + MutableObjectIterator> iter = table.getEntryIterator(); + Tuple2 next; + while ((next = iter.next()) != null) { + assertNotNull(next.f0); + assertNotNull(next.f1); + assertEquals(next.f0.longValue(), Long.parseLong(next.f1)); + + bitSet.set(next.f0.intValue()); + } + + assertEquals(numElements, bitSet.cardinality()); + } + + // make sure all entries are contained via the prober + { + CompactingHashTable>.HashTableProber proper = + table.getProber(probeComparator, pairComparator); + + for (long i = 0; i < numElements; i++) { + assertNotNull(proper.getMatchFor(i)); + assertNull(proper.getMatchFor(i + numElements)); + } + } + + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + /** + * This test validates that new inserts (rather than updates) in "insertOrReplace()" properly + * react to out of memory conditions. + */ + @Test + public void testInsertsWithInsertOrReplace() { + try { + final int numElements = 1000; + + final String longString = getLongString(10000); + List memory = getMemory(1000, 32 * 1024); + + // we create a hash table that thinks the records are super large. that makes it choose initially + // a lot of memory for the partition buffers, and start with a smaller hash table. that way + // we trigger a hash table growth early. + CompactingHashTable> table = new CompactingHashTable>( + serializer, comparator, memory, 100); + table.open(); + + // first, we insert some elements + for (long i = 0; i < numElements; i++) { + table.insertOrReplaceRecord(new Tuple2(i, longString)); + } + + // now, we replace the same elements, causing fragmentation + for (long i = 0; i < numElements; i++) { + table.insertOrReplaceRecord(new Tuple2(i, longString)); + } + + // now we insert an additional set of elements. without compaction during this insertion, + // the memory will run out + for (long i = 0; i < numElements; i++) { + table.insertOrReplaceRecord(new Tuple2(i + numElements, longString)); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + private static List getMemory(int numSegments, int segmentSize) { + ArrayList list = new ArrayList(numSegments); + for (int i = 0; i < numSegments; i++) { + list.add(new MemorySegment(new byte[segmentSize])); + } + return list; + } + + private static String getLongString(int length) { + StringBuilder bld = new StringBuilder(length); + for (int i = 0; i < length; i++) { + bld.append('a'); + } + return bld.toString(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java index a8941a4ba9958..0c656d6e00042 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java @@ -27,10 +27,6 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.operators.hash.AbstractHashTableProber; -import org.apache.flink.runtime.operators.hash.AbstractMutableHashTable; -import org.apache.flink.runtime.operators.hash.CompactingHashTable; -import org.apache.flink.runtime.operators.hash.MutableHashTable; import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator; import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator; import org.apache.flink.runtime.operators.testutils.types.IntPair; @@ -38,6 +34,7 @@ import org.apache.flink.runtime.operators.testutils.types.IntPairPairComparator; import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer; import org.apache.flink.util.MutableObjectIterator; + import org.junit.Test; import static org.junit.Assert.*; @@ -72,8 +69,8 @@ public void testCompactingHashMapPerformance() { MutableObjectIterator updateTester = new UniformIntPairGenerator(NUM_PAIRS, 1, false); - long start = 0L; - long end = 0L; + long start; + long end; long first = System.currentTimeMillis(); @@ -105,7 +102,7 @@ public void testCompactingHashMapPerformance() { start = System.currentTimeMillis(); while(updater.next(target) != null) { target.setValue(target.getValue()*-1); - table.insertOrReplaceRecord(target, temp); + table.insertOrReplaceRecord(target); } end = System.currentTimeMillis(); System.out.println("Update done. Time: " + (end-start) + " ms"); @@ -147,8 +144,8 @@ public void testMutableHashMapPerformance() { MutableObjectIterator updateTester = new UniformIntPairGenerator(NUM_PAIRS, 1, false); - long start = 0L; - long end = 0L; + long start; + long end; long first = System.currentTimeMillis(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java index 2ebcd43c5600a..3dcf688c8999a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java @@ -27,9 +27,6 @@ import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.operators.hash.AbstractHashTableProber; -import org.apache.flink.runtime.operators.hash.AbstractMutableHashTable; -import org.apache.flink.runtime.operators.hash.CompactingHashTable; import org.apache.flink.runtime.operators.testutils.UniformStringPairGenerator; import org.apache.flink.runtime.operators.testutils.types.IntList; import org.apache.flink.runtime.operators.testutils.types.IntListComparator; @@ -45,7 +42,9 @@ import org.apache.flink.runtime.operators.testutils.types.StringPairPairComparator; import org.apache.flink.runtime.operators.testutils.types.StringPairSerializer; import org.apache.flink.util.MutableObjectIterator; + import org.junit.Test; + import org.powermock.reflect.Whitebox; import static org.junit.Assert.*; @@ -235,9 +234,8 @@ public void testVariableLengthBuildAndRetrieve() { final IntList[] overwriteLists = getRandomizedIntLists(NUM_LISTS, rnd); // test replacing - IntList tempHolder = new IntList(); for (int i = 0; i < NUM_LISTS; i++) { - table.insertOrReplaceRecord(overwriteLists[i], tempHolder); + table.insertOrReplaceRecord(overwriteLists[i]); } for (int i = 0; i < NUM_LISTS; i++) { @@ -278,10 +276,9 @@ public void testVariableLengthBuildAndRetrieveMajorityUpdated() { final IntList[] overwriteLists = getRandomizedIntLists(NUM_LISTS, rnd); // test replacing - IntList tempHolder = new IntList(); for (int i = 0; i < NUM_LISTS; i++) { if( i % 100 != 0) { - table.insertOrReplaceRecord(overwriteLists[i], tempHolder); + table.insertOrReplaceRecord(overwriteLists[i]); lists[i] = overwriteLists[i]; } } @@ -327,10 +324,9 @@ public void testVariableLengthBuildAndRetrieveMinorityUpdated() { final IntList[] overwriteLists = getRandomizedIntLists(NUM_LISTS/STEP_SIZE, rnd); // test replacing - IntList tempHolder = new IntList(); for (int i = 0; i < NUM_LISTS; i += STEP_SIZE) { overwriteLists[i/STEP_SIZE].setKey(overwriteLists[i/STEP_SIZE].getKey()*STEP_SIZE); - table.insertOrReplaceRecord(overwriteLists[i/STEP_SIZE], tempHolder); + table.insertOrReplaceRecord(overwriteLists[i/STEP_SIZE]); lists[i] = overwriteLists[i/STEP_SIZE]; } @@ -379,9 +375,8 @@ public void testRepeatedBuildAndRetrieve() { for(int k = 0; k < NUM_REWRITES; k++) { overwriteLists = getRandomizedIntLists(NUM_LISTS, rnd); // test replacing - IntList tempHolder = new IntList(); for (int i = 0; i < NUM_LISTS; i++) { - table.insertOrReplaceRecord(overwriteLists[i], tempHolder); + table.insertOrReplaceRecord(overwriteLists[i]); } for (int i = 0; i < NUM_LISTS; i++) { @@ -409,11 +404,7 @@ public void testProberUpdate() { table.open(); for (int i = 0; i < NUM_LISTS; i++) { - try { - table.insert(lists[i]); - } catch (Exception e) { - throw e; - } + table.insert(lists[i]); } final IntList[] overwriteLists = getRandomizedIntLists(NUM_LISTS, rnd); @@ -630,9 +621,8 @@ public void testResizeWithCompaction(){ final IntList[] overwriteLists = getRandomizedIntLists(NUM_LISTS, rnd); // test replacing - IntList tempHolder = new IntList(); for (int i = 0; i < NUM_LISTS; i++) { - table.insertOrReplaceRecord(overwriteLists[i], tempHolder); + table.insertOrReplaceRecord(overwriteLists[i]); } Field list = Whitebox.getField(CompactingHashTable.class, "partitions"); @@ -691,7 +681,7 @@ public void testVariableLengthStringBuildAndRetrieve() { while(updater.next(target) != null) { target.setValue(target.getValue()); - table.insertOrReplaceRecord(target, temp); + table.insertOrReplaceRecord(target); } while (updateTester.next(target) != null) { From 249fa2bcdfc72bd6ce134ccdcb3921547af02752 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 14:52:39 +0200 Subject: [PATCH 088/175] [FLINK-2454] [buikd] Update Travis to drop JDK6 for tests --- .travis.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 77406f6c44447..792afb7a80dc9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -14,13 +14,13 @@ language: java #See https://issues.apache.org/jira/browse/FLINK-1072 matrix: include: - - jdk: "oraclejdk7" + - jdk: "openjdk7" env: PROFILE="-Dhadoop.profile=1 -Dscala-2.11" - - jdk: "openjdk6" # this will also deploy a uberjar to s3 at some point + - jdk: "oraclejdk7" # this will also deploy a uberjar to s3 at some point env: PROFILE="-Dhadoop.profile=1" - jdk: "openjdk7" env: PROFILE="-P!include-yarn -Dhadoop.version=2.0.0-alpha" - - jdk: "openjdk6" # we must use openjdk6 here to deploy a java6 compatible uber-jar for YARN + - jdk: "oraclejdk7" env: PROFILE="-Dhadoop.version=2.2.0" - jdk: "oraclejdk8" env: PROFILE="-Dhadoop.version=2.6.0 -Dscala-2.11 -Pinclude-tez -Dmaven.javadoc.skip=true" From 5a788ec23d50d36201ebb2fb0ad2b521272d034f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 14:40:53 +0200 Subject: [PATCH 089/175] [FLINK-2453] [pom] Move Java source and target version to 1.7 --- .../main/resources/archetype-resources/pom.xml | 4 ++-- .../main/resources/archetype-resources/pom.xml | 4 ++-- .../main/resources/archetype-resources/pom.xml | 4 ++-- flink-staging/flink-avro/pom.xml | 16 ---------------- pom.xml | 4 ++-- 5 files changed, 8 insertions(+), 24 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index de481d74c597d..fe915dfc1db48 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -231,8 +231,8 @@ under the License. maven-compiler-plugin 3.1 - 1.6 - 1.6 + 1.7 + 1.7 diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 90358d2f38ca1..cca652edd5ced 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -222,8 +222,8 @@ under the License. maven-compiler-plugin 3.1 - 1.6 - 1.6 + 1.7 + 1.7 diff --git a/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/pom.xml index 5274f47b6c069..2bad10a681b1d 100644 --- a/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/pom.xml @@ -114,8 +114,8 @@ under the License. maven-compiler-plugin 3.1 - 1.6 - 1.6 + 1.7 + 1.7 diff --git a/flink-staging/flink-avro/pom.xml b/flink-staging/flink-avro/pom.xml index e2d1c29c3a074..71a99d717d1c1 100644 --- a/flink-staging/flink-avro/pom.xml +++ b/flink-staging/flink-avro/pom.xml @@ -189,20 +189,4 @@ under the License. - - - - disable-javadocs-in-java6 - - (,1.6] - - - true - - - - diff --git a/pom.xml b/pom.xml index 455153aee6a74..f215fe4b5fd48 100644 --- a/pom.xml +++ b/pom.xml @@ -813,8 +813,8 @@ under the License. maven-compiler-plugin 3.1 - 1.6 - 1.6 + 1.7 + 1.7 -Xlint:all From a5b84b2b8284bcdaa649050b5090d79d8b58344c Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 16:55:32 +0200 Subject: [PATCH 090/175] [hotfix] Increase timeout for YARN tests to 180 seconds to prevent occasional CI failures. --- .../org/apache/flink/yarn/YarnTestBase.java | 36 +++++++++++-------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java index 23b89408109d7..2d227000f03e4 100644 --- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java @@ -425,13 +425,15 @@ protected void runWithArgs(String[] args, String terminateAfterString, String[] System.setErr(new PrintStream(errContent)); - final int START_TIMEOUT_SECONDS = 60; - + // we wait for at most three minutes + final int START_TIMEOUT_SECONDS = 180; + final long deadline = System.currentTimeMillis() + (START_TIMEOUT_SECONDS * 1000); + Runner runner = new Runner(args, type); runner.start(); boolean expectedStringSeen = false; - for(int second = 0; second < START_TIMEOUT_SECONDS; second++) { + do { sleep(1000); String outContentString = outContent.toString(); String errContentString = errContent.toString(); @@ -448,8 +450,7 @@ protected void runWithArgs(String[] args, String terminateAfterString, String[] } } // check output for correct TaskManager startup. - if(outContentString.contains(terminateAfterString) - || errContentString.contains(terminateAfterString) ) { + if (outContentString.contains(terminateAfterString) || errContentString.contains(terminateAfterString) ) { expectedStringSeen = true; LOG.info("Found expected output in redirected streams"); // send "stop" command to command line interface @@ -457,23 +458,28 @@ protected void runWithArgs(String[] args, String terminateAfterString, String[] runner.sendStop(); // wait for the thread to stop try { - runner.join(10000); - } catch (InterruptedException e) { + runner.join(30000); + } + catch (InterruptedException e) { LOG.debug("Interrupted while stopping runner", e); } LOG.warn("RunWithArgs runner stopped."); - break; } - // check if thread died - if(!runner.isAlive()) { - sendOutput(); - if(runner.getReturnValue() != 0) { - Assert.fail("Runner thread died before the test was finished. Return value = " + runner.getReturnValue()); - } else { - LOG.info("Runner stopped earlier than expected with return value = 0"); + else { + // check if thread died + if (!runner.isAlive()) { + sendOutput(); + if (runner.getReturnValue() != 0) { + Assert.fail("Runner thread died before the test was finished. Return value = " + + runner.getReturnValue()); + } else { + LOG.info("Runner stopped earlier than expected with return value = 0"); + } } } } + while (!expectedStringSeen && System.currentTimeMillis() < deadline); + sendOutput(); Assert.assertTrue("During the timeout period of " + START_TIMEOUT_SECONDS + " seconds the " + "expected string did not show up", expectedStringSeen); From 685086a3dd9afcec2eec76485298bc7b3f031a3c Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 15:18:14 +0200 Subject: [PATCH 091/175] [FLINK-2492] [runtime] Rename former 'match' classes to 'join' to reflect consistent naming scheme. --- .../plantranslate/JobGraphGenerator.java | 4 +- ...=> AbstractCachedBuildSideJoinDriver.java} | 4 +- ...r.java => BuildFirstCachedJoinDriver.java} | 4 +- ....java => BuildSecondCachedJoinDriver.java} | 4 +- .../runtime/operators/DriverStrategy.java | 10 +-- .../{MatchDriver.java => JoinDriver.java} | 64 +++++++++---------- .../operators/CachedMatchTaskTest.java | 22 +++---- .../operators/MatchTaskExternalITCase.java | 6 +- .../runtime/operators/MatchTaskTest.java | 46 ++++++------- .../ConnectedComponentsNepheleITCase.java | 4 +- .../CustomCompensatableDanglingPageRank.java | 4 +- ...ensatableDanglingPageRankWithCombiner.java | 4 +- .../CompensatableDanglingPageRank.java | 4 +- 13 files changed, 87 insertions(+), 93 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/{AbstractCachedBuildSideMatchDriver.java => AbstractCachedBuildSideJoinDriver.java} (96%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/{BuildFirstCachedMatchDriver.java => BuildFirstCachedJoinDriver.java} (85%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/{BuildSecondCachedMatchDriver.java => BuildSecondCachedJoinDriver.java} (84%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/{MatchDriver.java => JoinDriver.java} (58%) diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index 6fd2796776d9e..d440063838776 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -70,7 +70,7 @@ import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.JoinWithSolutionSetFirstDriver; import org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver; -import org.apache.flink.runtime.operators.MatchDriver; +import org.apache.flink.runtime.operators.JoinDriver; import org.apache.flink.runtime.operators.NoOpDriver; import org.apache.flink.runtime.operators.RegularPactTask; import org.apache.flink.runtime.operators.chaining.ChainedDriver; @@ -336,7 +336,7 @@ else if (node instanceof SolutionSetPlanNode) { } // adjust the driver - if (conf.getDriver().equals(MatchDriver.class)) { + if (conf.getDriver().equals(JoinDriver.class)) { conf.setDriver(inputNum == 0 ? JoinWithSolutionSetFirstDriver.class : JoinWithSolutionSetSecondDriver.class); } else if (conf.getDriver().equals(CoGroupDriver.class)) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java index f3b2dfd1a2420..aff8d01c70b0d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java @@ -33,7 +33,7 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public abstract class AbstractCachedBuildSideMatchDriver extends MatchDriver implements ResettablePactDriver, OT> { +public abstract class AbstractCachedBuildSideJoinDriver extends JoinDriver implements ResettablePactDriver, OT> { private volatile JoinTaskIterator matchIterator; @@ -44,7 +44,7 @@ public abstract class AbstractCachedBuildSideMatchDriver extends M private boolean objectReuseEnabled = false; - protected AbstractCachedBuildSideMatchDriver(int buildSideIndex, int probeSideIndex) { + protected AbstractCachedBuildSideJoinDriver(int buildSideIndex, int probeSideIndex) { this.buildSideIndex = buildSideIndex; this.probeSideIndex = probeSideIndex; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildFirstCachedMatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildFirstCachedJoinDriver.java similarity index 85% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildFirstCachedMatchDriver.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildFirstCachedJoinDriver.java index c141767725d18..6da221f732caa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildFirstCachedMatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildFirstCachedJoinDriver.java @@ -19,9 +19,9 @@ package org.apache.flink.runtime.operators; -public class BuildFirstCachedMatchDriver extends AbstractCachedBuildSideMatchDriver { +public class BuildFirstCachedJoinDriver extends AbstractCachedBuildSideJoinDriver { - public BuildFirstCachedMatchDriver() { + public BuildFirstCachedJoinDriver() { super(0, 1); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildSecondCachedMatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildSecondCachedJoinDriver.java similarity index 84% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildSecondCachedMatchDriver.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildSecondCachedJoinDriver.java index eed03abe8676c..44824c577325a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildSecondCachedMatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BuildSecondCachedJoinDriver.java @@ -19,9 +19,9 @@ package org.apache.flink.runtime.operators; -public class BuildSecondCachedMatchDriver extends AbstractCachedBuildSideMatchDriver { +public class BuildSecondCachedJoinDriver extends AbstractCachedBuildSideJoinDriver { - public BuildSecondCachedMatchDriver() { + public BuildSecondCachedJoinDriver() { super(1, 0); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java index 4a0035c94ab1b..3aadf2f6979fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java @@ -72,7 +72,7 @@ public enum DriverStrategy { ALL_GROUP_COMBINE(AllGroupCombineDriver.class, null, PIPELINED, 0), // both inputs are merged, but materialized to the side for block-nested-loop-join among values with equal key - MERGE(MatchDriver.class, null, MATERIALIZING, MATERIALIZING, 2), + MERGE(JoinDriver.class, null, MATERIALIZING, MATERIALIZING, 2), // co-grouping inputs CO_GROUP(CoGroupDriver.class, null, PIPELINED, PIPELINED, 2), @@ -81,13 +81,13 @@ public enum DriverStrategy { // the first input is build side, the second side is probe side of a hybrid hash table - HYBRIDHASH_BUILD_FIRST(MatchDriver.class, null, FULL_DAM, MATERIALIZING, 2), + HYBRIDHASH_BUILD_FIRST(JoinDriver.class, null, FULL_DAM, MATERIALIZING, 2), // the second input is build side, the first side is probe side of a hybrid hash table - HYBRIDHASH_BUILD_SECOND(MatchDriver.class, null, MATERIALIZING, FULL_DAM, 2), + HYBRIDHASH_BUILD_SECOND(JoinDriver.class, null, MATERIALIZING, FULL_DAM, 2), // a cached variant of HYBRIDHASH_BUILD_FIRST, that can only be used inside of iterations - HYBRIDHASH_BUILD_FIRST_CACHED(BuildFirstCachedMatchDriver.class, null, FULL_DAM, MATERIALIZING, 2), + HYBRIDHASH_BUILD_FIRST_CACHED(BuildFirstCachedJoinDriver.class, null, FULL_DAM, MATERIALIZING, 2), // cached variant of HYBRIDHASH_BUILD_SECOND, that can only be used inside of iterations - HYBRIDHASH_BUILD_SECOND_CACHED(BuildSecondCachedMatchDriver.class, null, MATERIALIZING, FULL_DAM, 2), + HYBRIDHASH_BUILD_SECOND_CACHED(BuildSecondCachedJoinDriver.class, null, MATERIALIZING, FULL_DAM, 2), // the second input is inner loop, the first input is outer loop and block-wise processed NESTEDLOOP_BLOCKED_OUTER_FIRST(CrossDriver.class, null, MATERIALIZING, FULL_DAM, 0), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java similarity index 58% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java index e54fca57d8f70..af3da5590eecd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import org.apache.flink.api.common.ExecutionConfig; @@ -40,26 +39,21 @@ import org.apache.flink.util.MutableObjectIterator; /** - * Match task which is executed by a Task Manager. The task has two inputs and one or multiple outputs. - * It is provided with a JoinFunction implementation. - *

    - * The MatchTask matches all pairs of records that share the same key and come from different inputs. Each pair of - * matching records is handed to the match() method of the JoinFunction. + * The join driver implements the logic of a join operator at runtime. It instantiates either + * hash or sort-merge based strategies to find joining pairs of records. * * @see org.apache.flink.api.common.functions.FlatJoinFunction */ -public class MatchDriver implements PactDriver, OT> { +public class JoinDriver implements PactDriver, OT> { - protected static final Logger LOG = LoggerFactory.getLogger(MatchDriver.class); + protected static final Logger LOG = LoggerFactory.getLogger(JoinDriver.class); protected PactTaskContext, OT> taskContext; - private volatile JoinTaskIterator matchIterator; // the iterator that does the actual matching + private volatile JoinTaskIterator joinIterator; // the iterator that does the actual join protected volatile boolean running; - private boolean objectReuseEnabled = false; - // ------------------------------------------------------------------------ @Override @@ -112,80 +106,80 @@ public void prepare() throws Exception{ final TypePairComparatorFactory pairComparatorFactory = config.getPairComparatorFactory( this.taskContext.getUserCodeClassLoader()); if (pairComparatorFactory == null) { - throw new Exception("Missing pair comparator factory for Match driver"); + throw new Exception("Missing pair comparator factory for join driver"); } ExecutionConfig executionConfig = taskContext.getExecutionConfig(); - this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled(); if (LOG.isDebugEnabled()) { - LOG.debug("MatchDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + LOG.debug("Join Driver object reuse: " + (objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); } - // create and return MatchTaskIterator according to provided local strategy. - if (this.objectReuseEnabled) { + // create and return joining iterator according to provided local strategy. + if (objectReuseEnabled) { switch (ls) { case MERGE: - this.matchIterator = new ReusingMergeInnerJoinIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.joinIterator = new ReusingMergeInnerJoinIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: - this.matchIterator = new ReusingBuildFirstHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new ReusingBuildFirstHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); break; case HYBRIDHASH_BUILD_SECOND: - this.matchIterator = new ReusingBuildSecondHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new ReusingBuildSecondHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); break; default: - throw new Exception("Unsupported driver strategy for Match driver: " + ls.name()); + throw new Exception("Unsupported driver strategy for join driver: " + ls.name()); } } else { switch (ls) { case MERGE: - this.matchIterator = new NonReusingMergeInnerJoinIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.joinIterator = new NonReusingMergeInnerJoinIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: - this.matchIterator = new NonReusingBuildFirstHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new NonReusingBuildFirstHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); break; case HYBRIDHASH_BUILD_SECOND: - this.matchIterator = new NonReusingBuildSecondHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new NonReusingBuildSecondHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); break; default: - throw new Exception("Unsupported driver strategy for Match driver: " + ls.name()); + throw new Exception("Unsupported driver strategy for join driver: " + ls.name()); } } - // open MatchTaskIterator - this triggers the sorting or hash-table building + // open the iterator - this triggers the sorting or hash-table building // and blocks until the iterator is ready - this.matchIterator.open(); + this.joinIterator.open(); if (LOG.isDebugEnabled()) { - LOG.debug(this.taskContext.formatLogString("Match task iterator ready.")); + LOG.debug(this.taskContext.formatLogString("join task iterator ready.")); } } @Override public void run() throws Exception { - final FlatJoinFunction matchStub = this.taskContext.getStub(); + final FlatJoinFunction joinStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); - final JoinTaskIterator matchIterator = this.matchIterator; + final JoinTaskIterator joinIterator = this.joinIterator; - while (this.running && matchIterator.callWithNextKey(matchStub, collector)); + while (this.running && joinIterator.callWithNextKey(joinStub, collector)); } @Override public void cleanup() throws Exception { - if (this.matchIterator != null) { - this.matchIterator.close(); - this.matchIterator = null; + if (this.joinIterator != null) { + this.joinIterator.close(); + this.joinIterator = null; } } @Override public void cancel() { this.running = false; - if (this.matchIterator != null) { - this.matchIterator.abort(); + if (this.joinIterator != null) { + this.joinIterator.abort(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java index a57287a7fe747..c93c302249d75 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java @@ -78,7 +78,7 @@ public void testHash1MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildFirstCachedMatchDriver testTask = new BuildFirstCachedMatchDriver(); + BuildFirstCachedJoinDriver testTask = new BuildFirstCachedJoinDriver(); try { testResettableDriver(testTask, MockMatchStub.class, 3); @@ -109,7 +109,7 @@ public void testHash2MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildSecondCachedMatchDriver testTask = new BuildSecondCachedMatchDriver(); + BuildSecondCachedJoinDriver testTask = new BuildSecondCachedJoinDriver(); try { testResettableDriver(testTask, MockMatchStub.class, 3); @@ -140,7 +140,7 @@ public void testHash3MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildFirstCachedMatchDriver testTask = new BuildFirstCachedMatchDriver(); + BuildFirstCachedJoinDriver testTask = new BuildFirstCachedJoinDriver(); try { testResettableDriver(testTask, MockMatchStub.class, 3); @@ -171,7 +171,7 @@ public void testHash4MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildSecondCachedMatchDriver testTask = new BuildSecondCachedMatchDriver(); + BuildSecondCachedJoinDriver testTask = new BuildSecondCachedJoinDriver(); try { testResettableDriver(testTask, MockMatchStub.class, 3); @@ -202,7 +202,7 @@ public void testHash5MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildFirstCachedMatchDriver testTask = new BuildFirstCachedMatchDriver(); + BuildFirstCachedJoinDriver testTask = new BuildFirstCachedJoinDriver(); try { testResettableDriver(testTask, MockMatchStub.class, 3); @@ -233,7 +233,7 @@ public void testFailingHashFirstMatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildFirstCachedMatchDriver testTask = new BuildFirstCachedMatchDriver(); + BuildFirstCachedJoinDriver testTask = new BuildFirstCachedJoinDriver(); try { testResettableDriver(testTask, MockFailingMatchStub.class, 3); @@ -263,7 +263,7 @@ public void testFailingHashSecondMatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - BuildSecondCachedMatchDriver testTask = new BuildSecondCachedMatchDriver(); + BuildSecondCachedJoinDriver testTask = new BuildSecondCachedJoinDriver(); try { testResettableDriver(testTask, MockFailingMatchStub.class, 3); @@ -294,7 +294,7 @@ public void testCancelHashMatchTaskWhileBuildFirst() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - final BuildFirstCachedMatchDriver testTask = new BuildFirstCachedMatchDriver(); + final BuildFirstCachedJoinDriver testTask = new BuildFirstCachedJoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); @@ -338,7 +338,7 @@ public void testHashCancelMatchTaskWhileBuildSecond() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED); getTaskConfig().setRelativeMemoryDriver(1.0f); - final BuildSecondCachedMatchDriver testTask = new BuildSecondCachedMatchDriver(); + final BuildSecondCachedJoinDriver testTask = new BuildSecondCachedJoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); @@ -382,7 +382,7 @@ public void testHashFirstCancelMatchTaskWhileMatching() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(1.0f); - final BuildFirstCachedMatchDriver testTask = new BuildFirstCachedMatchDriver(); + final BuildFirstCachedJoinDriver testTask = new BuildFirstCachedJoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); @@ -426,7 +426,7 @@ public void testHashSecondCancelMatchTaskWhileMatching() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(1.0f); - final BuildSecondCachedMatchDriver testTask = new BuildSecondCachedMatchDriver(); + final BuildSecondCachedJoinDriver testTask = new BuildSecondCachedJoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java index 29be8f846f062..30c1610fcca4b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java @@ -80,7 +80,7 @@ public void testExternalSort1MatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -113,7 +113,7 @@ public void testExternalHash1MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); @@ -144,7 +144,7 @@ public void testExternalHash2MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java index 16aea693f4121..8fbf05eac8b34 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java @@ -90,7 +90,7 @@ public void testSortBoth1MatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -124,7 +124,7 @@ public void testSortBoth2MatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -160,7 +160,7 @@ public void testSortBoth3MatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -196,7 +196,7 @@ public void testSortBoth4MatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -232,7 +232,7 @@ public void testSortBoth5MatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -268,7 +268,7 @@ public void testSortFirstMatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new UniformRecordGenerator(keyCnt1, valCnt1, false), this.comparator1.duplicate()); @@ -304,7 +304,7 @@ public void testSortSecondMatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInput(new UniformRecordGenerator(keyCnt1, valCnt1, true)); @@ -339,7 +339,7 @@ public void testMergeMatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); addInput(new UniformRecordGenerator(keyCnt1, valCnt1, true)); addInput(new UniformRecordGenerator(keyCnt2, valCnt2, true)); @@ -375,7 +375,7 @@ public void testFailingMatchTask() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); addInput(new UniformRecordGenerator(keyCnt1, valCnt1, true)); addInput(new UniformRecordGenerator(keyCnt2, valCnt2, true)); @@ -404,7 +404,7 @@ public void testCancelMatchTaskWhileSort1() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInputSorted(new DelayingInfinitiveInputIterator(100), this.comparator1.duplicate()); @@ -455,7 +455,7 @@ public void testCancelMatchTaskWhileSort2() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); try { addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); @@ -506,7 +506,7 @@ public void testCancelMatchTaskWhileMatching() { getTaskConfig().setRelativeMemoryDriver(bnljn_frac); setNumFileHandlesForSort(4); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); @@ -556,7 +556,7 @@ public void testHash1MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); @@ -587,7 +587,7 @@ public void testHash2MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); @@ -618,7 +618,7 @@ public void testHash3MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); @@ -649,7 +649,7 @@ public void testHash4MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); @@ -680,7 +680,7 @@ public void testHash5MatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockMatchStub.class); @@ -711,7 +711,7 @@ public void testFailingHashFirstMatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockFailingMatchStub.class); @@ -741,7 +741,7 @@ public void testFailingHashSecondMatchTask() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(hash_frac); - MatchDriver testTask = new MatchDriver(); + JoinDriver testTask = new JoinDriver(); try { testDriver(testTask, MockFailingMatchStub.class); @@ -773,7 +773,7 @@ public void testCancelHashMatchTaskWhileBuildFirst() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); @@ -827,7 +827,7 @@ public void testHashCancelMatchTaskWhileBuildSecond() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(hash_frac); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); @@ -876,7 +876,7 @@ public void testHashFirstCancelMatchTaskWhileMatching() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST); getTaskConfig().setRelativeMemoryDriver(hash_frac); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); @@ -920,7 +920,7 @@ public void testHashSecondCancelMatchTaskWhileMatching() { getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); getTaskConfig().setRelativeMemoryDriver(hash_frac); - final MatchDriver testTask = new MatchDriver(); + final JoinDriver testTask = new JoinDriver(); final AtomicBoolean success = new AtomicBoolean(false); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java index 8e9266d8cf56d..7a3639bb6dffb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java @@ -47,7 +47,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.OutputFormatVertex; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver; +import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver; import org.apache.flink.runtime.operators.CollectorMapDriver; import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.GroupReduceDriver; @@ -276,7 +276,7 @@ private static JobVertex createIterationHead(JobGraph jobGraph, int numSubTasks, headConfig.setIterationHeadIndexOfSyncOutput(2); // the driver - headConfig.setDriver(BuildSecondCachedMatchDriver.class); + headConfig.setDriver(BuildSecondCachedJoinDriver.class); headConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); headConfig.setStubWrapper( new UserCodeClassWrapper(NeighborWithComponentIDJoin.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java index c29e868bdb60e..a6e6b6ecf9d2e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.OutputFormatVertex; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver; +import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver; import org.apache.flink.runtime.operators.CoGroupDriver; import org.apache.flink.runtime.operators.CollectorMapDriver; import org.apache.flink.runtime.operators.DriverStrategy; @@ -204,7 +204,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); - intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class); + intermediateConfig.setDriver(BuildSecondCachedJoinDriver.class); intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); intermediateConfig.setRelativeMemoryDriver((double) matchMemory / totalMemoryConsumption); intermediateConfig.addInputToGroup(0); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java index de9e0a17c65fc..0bf258feae99b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.OutputFormatVertex; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver; +import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver; import org.apache.flink.runtime.operators.CoGroupDriver; import org.apache.flink.runtime.operators.CollectorMapDriver; import org.apache.flink.runtime.operators.DriverStrategy; @@ -204,7 +204,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); - intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class); + intermediateConfig.setDriver(BuildSecondCachedJoinDriver.class); intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); intermediateConfig.setRelativeMemoryDriver((double)matchMemory/totalMemoryConsumption); intermediateConfig.addInputToGroup(0); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java index eb2ccdc6a07d8..78038b34cac1e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java @@ -37,7 +37,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.OutputFormatVertex; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver; +import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver; import org.apache.flink.runtime.operators.CoGroupDriver; import org.apache.flink.runtime.operators.CollectorMapDriver; import org.apache.flink.runtime.operators.DriverStrategy; @@ -183,7 +183,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); - intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class); + intermediateConfig.setDriver(BuildSecondCachedJoinDriver.class); intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); intermediateConfig.setRelativeMemoryDriver((double)matchMemory/totalMemoryConsumption); intermediateConfig.addInputToGroup(0); From 61dcae391cb3b45ba3aff47d4d9163889d2958a4 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Fri, 3 Jul 2015 23:53:47 +0800 Subject: [PATCH 092/175] [FLINK-2240] Add bloom filter to filter probe records during hash join. --- .../flink/configuration/ConfigConstants.java | 13 +- .../operators/hash/MutableHashTable.java | 145 +++++++++- .../runtime/operators/util/BloomFilter.java | 226 +++++++++++++++ .../MutableHashTablePerformanceBenchmark.java | 268 ++++++++++++++++++ .../operators/util/BloomFilterTest.java | 162 +++++++++++ 5 files changed, 806 insertions(+), 8 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index c76741b2de8ee..dad2d99d183fe 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -178,7 +178,7 @@ public final class ConfigConstants { * for hybrid hash joins. */ public static final String DEFAULT_SPILLING_MAX_FAN_KEY = "taskmanager.runtime.max-fan"; - + /** * Key for the default spilling threshold. When more than the threshold memory of the sort buffers is full, the * sorter will start spilling to disk. @@ -190,6 +190,12 @@ public final class ConfigConstants { * A value of 0 indicates infinite waiting. */ public static final String FS_STREAM_OPENING_TIMEOUT_KEY = "taskmanager.runtime.fs_timeout"; + + /** + * While spill probe record to disk during probe phase, whether enable bloom filter to filter the probe records + * to minimize the spilled probe records count. + */ + public static final String HASHJOIN_ENABLE_BLOOMFILTER = "hashjoin.bloomfilter.enabled"; // ------------------------ YARN Configuration ------------------------ @@ -552,6 +558,11 @@ public final class ConfigConstants { * The default timeout for filesystem stream opening: infinite (means max long milliseconds). */ public static final int DEFAULT_FS_STREAM_OPENING_TIMEOUT = 0; + + /** + * Enable bloom filter for hash join as it promote hash join performance most of the time. + */ + public static final boolean DEAFULT_HASHJOIN_ENABLE_BLOOMFILTER = true; // ------------------------ YARN Configuration ------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java index 7f07cfbf2c692..4a57986604a16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java @@ -16,16 +16,16 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.hash; - import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.GlobalConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -42,6 +42,7 @@ import org.apache.flink.runtime.io.disk.iomanager.HeaderlessChannelReaderInputView; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.iterative.io.HashPartitionIterator; +import org.apache.flink.runtime.operators.util.BloomFilter; import org.apache.flink.runtime.util.MathUtils; import org.apache.flink.util.MutableObjectIterator; @@ -194,6 +195,11 @@ public class MutableHashTable implements MemorySegmentSource { * Constant for the bucket status, indicating that the bucket is in memory. */ private static final byte BUCKET_STATUS_IN_MEMORY = 0; + + /** + * Constant for the bucket status, indicating that the bucket has filter. + */ + private static final byte BUCKET_STATUS_IN_FILTER = 1; // ------------------------------------------------------------------------ // Members @@ -348,6 +354,8 @@ public class MutableHashTable implements MemorySegmentSource { private boolean running = true; + private BloomFilter bloomFilter; + // ------------------------------------------------------------------------ // Construction and Teardown // ------------------------------------------------------------------------ @@ -469,12 +477,19 @@ protected boolean processProbeIter() throws IOException{ this.recordComparator.setReference(next); this.bucketIterator.set(bucket, p.overflowSegments, p, hash, bucketInSegmentOffset); return true; - } - else { - p.insertIntoProbeBuffer(next); + } else { + byte status = bucket.get(bucketInSegmentOffset + HEADER_STATUS_OFFSET); + if (status == BUCKET_STATUS_IN_FILTER) { + this.bloomFilter.setBitsLocation(bucket, bucketInSegmentOffset + BUCKET_HEADER_LENGTH); + // Use BloomFilter to filter out all the probe records which would not match any key in spilled build table buckets. + if (this.bloomFilter.testHash(hash)) { + p.insertIntoProbeBuffer(next); + } + } else { + p.insertIntoProbeBuffer(next); + } } } - // -------------- partition done --------------- return false; @@ -710,6 +725,27 @@ protected void buildInitialTable(final MutableObjectIterator input) throws I p.finalizeBuildPhase(this.ioManager, this.currentEnumerator, this.writeBehindBuffers); } } + + private void initBloomFilter(int numBuckets) { + int avgNumRecordsPerBucket = getEstimatedMaxBucketEntries(this.availableMemory.size(), this.segmentSize, + numBuckets, this.avgRecordLen); + // Assign all bucket size to bloom filter except bucket header length. + int byteSize = HASH_BUCKET_SIZE - BUCKET_HEADER_LENGTH; + this.bloomFilter = new BloomFilter(avgNumRecordsPerBucket, byteSize); + if (LOG.isDebugEnabled()) { + double fpp = BloomFilter.estimateFalsePositiveProbability(avgNumRecordsPerBucket, byteSize << 3); + LOG.debug(String.format("Create BloomFilter with average input entries per bucket[%d], bytes size[%d], false positive probability[%f].", + avgNumRecordsPerBucket, byteSize, fpp)); + } + } + + final private int getEstimatedMaxBucketEntries(int numBuffers, int bufferSize, int numBuckets, int recordLenBytes) { + final long totalSize = ((long) bufferSize) * numBuffers; + final long numRecordsStorable = totalSize / (recordLenBytes + RECORD_OVERHEAD_BYTES); + final long maxNumRecordsStorable = (MAX_RECURSION_DEPTH + 1) * numRecordsStorable; + final long maxNumRecordsPerBucket = maxNumRecordsStorable / numBuckets; + return (int) maxNumRecordsPerBucket; + } /** * @param p @@ -816,7 +852,7 @@ protected void buildTableFromSpilledPartition(final HashPartition p) thr final int hashCode = hash(btComparator.hash(rec), nextRecursionLevel); insertIntoTable(rec, hashCode); } - + // finalize the partitions for (int i = 0; i < this.partitionsBeingBuilt.size(); i++) { HashPartition part = this.partitionsBeingBuilt.get(i); @@ -853,6 +889,14 @@ protected final void insertIntoTable(final BT record, final int hashCode) throws if (pointer != -1) { // record was inserted into an in-memory partition. a pointer must be inserted into the buckets insertBucketEntry(p, bucket, bucketInSegmentPos, hashCode, pointer); + } else { + byte status = bucket.get(bucketInSegmentPos + HEADER_STATUS_OFFSET); + if (status == BUCKET_STATUS_IN_FILTER) { + // While partition has been spilled, relocation bloom filter bits for current bucket, + // and build bloom filter with hashcode. + this.bloomFilter.setBitsLocation(bucket, bucketInSegmentPos + BUCKET_HEADER_LENGTH); + this.bloomFilter.addHash(hashCode); + } } } @@ -1047,6 +1091,12 @@ protected void initTable(int numBuckets, byte numPartitions) { } this.buckets = table; this.numBuckets = numBuckets; + + boolean enableBloomFilter = GlobalConfiguration.getBoolean( + ConfigConstants.HASHJOIN_ENABLE_BLOOMFILTER, ConfigConstants.DEAFULT_HASHJOIN_ENABLE_BLOOMFILTER); + if (enableBloomFilter) { + initBloomFilter(numBuckets); + } } /** @@ -1088,6 +1138,12 @@ protected int spillPartition() throws IOException { } final HashPartition p = partitions.get(largestPartNum); + boolean enableBloomFilter = GlobalConfiguration.getBoolean( + ConfigConstants.HASHJOIN_ENABLE_BLOOMFILTER, ConfigConstants.DEAFULT_HASHJOIN_ENABLE_BLOOMFILTER); + if (enableBloomFilter) { + buildBloomFilterForBucketsInPartition(largestPartNum, p); + } + // spill the partition int numBuffersFreed = p.spillPartition(this.availableMemory, this.ioManager, this.currentEnumerator.next(), this.writeBehindBuffers); @@ -1101,6 +1157,81 @@ protected int spillPartition() throws IOException { return largestPartNum; } + final protected void buildBloomFilterForBucketsInPartition(int partNum, HashPartition partition) { + // Find all the buckets which belongs to this partition, and build bloom filter for each bucket(include its overflow buckets). + final int bucketsPerSegment = this.bucketsPerSegmentMask + 1; + for (MemorySegment segment : this.buckets) { + for (int i = 0; i < bucketsPerSegment; i++) { + final int bucketInSegmentOffset = i * HASH_BUCKET_SIZE; + byte partitionNumber = segment.get(bucketInSegmentOffset + HEADER_PARTITION_OFFSET); + if (partitionNumber == partNum) { + byte status = segment.get(bucketInSegmentOffset + HEADER_STATUS_OFFSET); + if (status == BUCKET_STATUS_IN_MEMORY) { + buildBloomFilterForBucket(bucketInSegmentOffset, segment, partition); + } + } + } + } + } + + /** + * Set all the bucket memory except bucket header as the bit set of bloom filter, and use hash code of build records + * to build bloom filter. + * + * @param bucketInSegmentPos + * @param bucket + * @param p + */ + final void buildBloomFilterForBucket(int bucketInSegmentPos, MemorySegment bucket, HashPartition p) { + final int count = bucket.getShort(bucketInSegmentPos + HEADER_COUNT_OFFSET); + int[] hashCodes = new int[count]; + // As the hashcode and bloom filter occupy same bytes, so we read all hashcode out at first and then write back to bloom filter. + for (int i = 0; i < count; i++) { + hashCodes[i] = bucket.getInt(bucketInSegmentPos + BUCKET_HEADER_LENGTH + i * HASH_CODE_LEN); + } + this.bloomFilter.setBitsLocation(bucket, bucketInSegmentPos + BUCKET_HEADER_LENGTH); + for (int hashCode : hashCodes) { + this.bloomFilter.addHash(hashCode); + } + buildBloomFilterForExtraOverflowSegments(bucketInSegmentPos, bucket, p); + } + + final private void buildBloomFilterForExtraOverflowSegments(int bucketInSegmentPos, MemorySegment bucket, HashPartition p) { + int totalCount = 0; + boolean skip = false; + long forwardPointer = bucket.getLong(bucketInSegmentPos + HEADER_FORWARD_OFFSET); + while (forwardPointer != BUCKET_FORWARD_POINTER_NOT_SET) { + final int overflowSegNum = (int) (forwardPointer >>> 32); + if (overflowSegNum < 0 || overflowSegNum >= p.numOverflowSegments) { + skip = true; + break; + } + MemorySegment overflowSegment = p.overflowSegments[overflowSegNum]; + int bucketInOverflowSegmentOffset = (int) (forwardPointer & 0xffffffff); + + final int count = overflowSegment.getShort(bucketInOverflowSegmentOffset + HEADER_COUNT_OFFSET); + totalCount += count; + // The bits size of bloom filter per bucket is 112 * 8, while expected input entries is greater than 2048, the fpp would higher than 0.9, + // which make the bloom filter an overhead instead of optimization. + if (totalCount > 2048) { + skip = true; + break; + } + + for (int i = 0; i < count; i++) { + int hashCode = overflowSegment.getInt(bucketInOverflowSegmentOffset + BUCKET_HEADER_LENGTH + i * HASH_CODE_LEN); + this.bloomFilter.addHash(hashCode); + } + + forwardPointer = overflowSegment.getLong(bucketInOverflowSegmentOffset + HEADER_FORWARD_OFFSET); + + } + + if (!skip) { + bucket.put(bucketInSegmentPos + HEADER_STATUS_OFFSET, BUCKET_STATUS_IN_FILTER); + } + } + /** * This method makes sure that at least a certain number of memory segments is in the list of free segments. * Free memory can be in the list of free segments, or in the return-queue where segments used to write behind are diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java new file mode 100644 index 0000000000000..947a56bd0fd42 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java @@ -0,0 +1,226 @@ +/** + * 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.runtime.operators.util; + +import com.google.common.base.Preconditions; +import org.apache.flink.core.memory.MemorySegment; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * BloomFilter is a probabilistic data structure for set membership check. BloomFilters are + * highly space efficient when compared to using a HashSet. Because of the probabilistic nature of + * bloom filter false positive (element not present in bloom filter but test() says true) are + * possible but false negatives are not possible (if element is present then test() will never + * say false). The false positive probability is configurable depending on which storage requirement + * may increase or decrease. Lower the false positive probability greater is the space requirement. + * Bloom filters are sensitive to number of elements that will be inserted in the bloom filter. + * During the creation of bloom filter expected number of entries must be specified. If the number + * of insertions exceed the specified initial number of entries then false positive probability will + * increase accordingly. + *

    + * Internally, this implementation of bloom filter uses MemorySegment to store BitSet, BloomFilter and + * BitSet are designed to be able to switch between different MemorySegments, so that Flink can share + * the same BloomFilter/BitSet object instance for different bloom filters. + *

    + * Part of this class refers to the implementation from Apache Hive project + * https://github.com/apache/hive/blob/master/common/src/java/org/apache/hive/common/util/BloomFilter.java + */ + +public class BloomFilter { + + protected BitSet bitSet; + protected int expectedEntries; + protected int numHashFunctions; + + public BloomFilter(int expectedEntries, int byteSize) { + checkArgument(expectedEntries > 0, "expectedEntries should be > 0"); + this.expectedEntries = expectedEntries; + this.numHashFunctions = optimalNumOfHashFunctions(expectedEntries, byteSize << 3); + this.bitSet = new BitSet(byteSize); + } + + public void setBitsLocation(MemorySegment memorySegment, int offset) { + this.bitSet.setMemorySegment(memorySegment, offset); + } + + /** + * Compute optimal bits number with given input entries and expected false positive probability. + * + * @param inputEntries + * @param fpp + * @return optimal bits number + */ + public static int optimalNumOfBits(long inputEntries, double fpp) { + int numBits = (int) (-inputEntries * Math.log(fpp) / (Math.log(2) * Math.log(2))); + return numBits; + } + + /** + * Compute the false positive probability based on given input entries and bits size. + * Note: this is just the math expected value, you should not expect the fpp in real case would under the return value for certain. + * + * @param inputEntries + * @param bitSize + * @return + */ + public static double estimateFalsePositiveProbability(long inputEntries, int bitSize) { + int numFunction = optimalNumOfHashFunctions(inputEntries, bitSize); + double p = Math.pow(Math.E, -(double) numFunction * inputEntries / bitSize); + double estimatedFPP = Math.pow(1 - p, numFunction); + return estimatedFPP; + } + + /** + * compute the optimal hash function number with given input entries and bits size, which would + * make the false positive probability lowest. + * + * @param expectEntries + * @param bitSize + * @return hash function number + */ + static int optimalNumOfHashFunctions(long expectEntries, long bitSize) { + return Math.max(1, (int) Math.round((double) bitSize / expectEntries * Math.log(2))); + } + + public void addHash(int hash32) { + int hash1 = hash32; + int hash2 = hash32 >>> 16; + + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = hash1 + (i * hash2); + // hashcode should be positive, flip all the bits if it's negative + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + int pos = combinedHash % bitSet.bitSize(); + bitSet.set(pos); + } + } + + public boolean testHash(int hash32) { + int hash1 = hash32; + int hash2 = hash32 >>> 16; + + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = hash1 + (i * hash2); + // hashcode should be positive, flip all the bits if it's negative + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + int pos = combinedHash % bitSet.bitSize(); + if (!bitSet.get(pos)) { + return false; + } + } + return true; + } + + public void reset() { + this.bitSet.clear(); + } + + @Override + public String toString() { + StringBuilder output = new StringBuilder(); + output.append("BloomFilter:\n"); + output.append("\thash function number:").append(numHashFunctions).append("\n"); + output.append(bitSet); + return output.toString(); + } + + /** + * Bare metal bit set implementation. For performance reasons, this implementation does not check + * for index bounds nor expand the bit set size if the specified index is greater than the size. + */ + public class BitSet { + private MemorySegment memorySegment; + // MemorySegment byte array offset. + private int offset; + // MemorySegment byte size. + private int length; + private final int LONG_POSITION_MASK = 0xffffffc0; + + public BitSet(int byteSize) { + Preconditions.checkArgument(byteSize > 0, "bits size should be greater than 0."); + Preconditions.checkArgument(byteSize << 29 == 0, "bytes size should be integral multiple of long size(8 Bytes)."); + this.length = byteSize; + } + + public void setMemorySegment(MemorySegment memorySegment, int offset) { + this.memorySegment = memorySegment; + this.offset = offset; + } + + /** + * Sets the bit at specified index. + * + * @param index - position + */ + public void set(int index) { + int longIndex = (index & LONG_POSITION_MASK) >>> 3; + long current = memorySegment.getLong(offset + longIndex); + current |= (1L << index); + memorySegment.putLong(offset + longIndex, current); + } + + /** + * Returns true if the bit is set in the specified index. + * + * @param index - position + * @return - value at the bit position + */ + public boolean get(int index) { + int longIndex = (index & LONG_POSITION_MASK) >>> 3; + long current = memorySegment.getLong(offset + longIndex); + return (current & (1L << index)) != 0; + } + + /** + * Number of bits + */ + public int bitSize() { + return length << 3; + } + + public MemorySegment getMemorySegment() { + return this.memorySegment; + } + + /** + * Clear the bit set. + */ + public void clear() { + long zeroValue = 0L; + for (int i = 0; i < (length / 8); i++) { + memorySegment.putLong(offset + i * 8, zeroValue); + } + } + + @Override + public String toString() { + StringBuilder output = new StringBuilder(); + output.append("BitSet:\n"); + output.append("\tMemorySegment:").append(memorySegment.size()).append("\n"); + output.append("\tOffset:").append(offset).append("\n"); + output.append("\tLength:").append(length).append("\n"); + return output.toString(); + } + } +} \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java new file mode 100644 index 0000000000000..452e4c11ca2ed --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java @@ -0,0 +1,268 @@ +/** + * 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.runtime.operators.hash; + +import java.io.IOException; +import java.util.List; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; +import org.apache.flink.runtime.operators.testutils.types.StringPair; +import org.apache.flink.runtime.operators.testutils.types.StringPairComparator; +import org.apache.flink.runtime.operators.testutils.types.StringPairPairComparator; +import org.apache.flink.runtime.operators.testutils.types.StringPairSerializer; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.fail; + +public class MutableHashTablePerformanceBenchmark { + private static final AbstractInvokable MEM_OWNER = new DummyInvokable(); + + private MemoryManager memManager; + private IOManager ioManager; + + private TypeSerializer pairBuildSideAccesssor; + private TypeSerializer pairProbeSideAccesssor; + private TypeComparator pairBuildSideComparator; + private TypeComparator pairProbeSideComparator; + private TypePairComparator pairComparator; + + private static final String COMMENT = "this comments should contains a 96 byte data, 100 plus another integer value and seperator char."; + + + @Before + public void setup() { + this.pairBuildSideAccesssor = new StringPairSerializer(); + this.pairProbeSideAccesssor = new StringPairSerializer(); + this.pairBuildSideComparator = new StringPairComparator(); + this.pairProbeSideComparator = new StringPairComparator(); + this.pairComparator = new StringPairPairComparator(); + + this.memManager = new DefaultMemoryManager(64 * 1024 * 1024, 1); + this.ioManager = new IOManagerAsync(); + } + + @After + public void tearDown() { + // shut down I/O manager and Memory Manager and verify the correct shutdown + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + fail("I/O manager was not property shut down."); + } + if (!this.memManager.verifyEmpty()) { + fail("Not all memory was properly released to the memory manager --> Memory Leak."); + } + } + + @Test + public void compareMutableHashTablePerformance1() throws IOException { + // ----------------------------------------------90% filtered during probe spill phase----------------------------------------- + // create a build input with 1000000 records with key spread between [0 -- 10000000] with step of 10 for nearby records. + int buildSize = 1000000; + int buildStep = 10; + int buildScope = buildStep * buildSize; + // create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records. + int probeSize = 5000000; + int probeStep = 1; + int probeScope = buildSize; + + int expectedResult = 500000; + + long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true); + long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false); + + System.out.println("HybridHashJoin2:"); + System.out.println("Build input size: " + 100 * buildSize); + System.out.println("Probe input size: " + 100 * probeSize); + System.out.println("Available memory: " + this.memManager.getMemorySize()); + System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent."); + System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost)); + } + + @Test + public void compareMutableHashTablePerformance2() throws IOException { + // ----------------------------------------------80% filtered during probe spill phase----------------------------------------- + // create a build input with 1000000 records with key spread between [0 -- 5000000] with step of 5 for nearby records. + int buildSize = 1000000; + int buildStep = 5; + int buildScope = buildStep * buildSize; + // create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records. + int probeSize = 5000000; + int probeStep = 1; + int probeScope = buildSize; + + int expectedResult = 1000000; + + long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true); + long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false); + + System.out.println("HybridHashJoin3:"); + System.out.println("Build input size: " + 100 * buildSize); + System.out.println("Probe input size: " + 100 * probeSize); + System.out.println("Available memory: " + this.memManager.getMemorySize()); + System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent."); + System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost)); + } + + @Test + public void compareMutableHashTablePerformance3() throws IOException { + // ----------------------------------------------50% filtered during probe spill phase------------------------------------------------- + // create a build input with 1000000 records with key spread between [0 -- 2000000] with step of 2 for nearby records. + int buildSize = 1000000; + int buildStep = 2; + int buildScope = buildStep * buildSize; + // create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records. + int probeSize = 5000000; + int probeStep = 1; + int probeScope = buildSize; + + int expectedResult = 2500000; + + long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false); + long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true); + + System.out.println("HybridHashJoin4:"); + System.out.println("Build input size: " + 100 * buildSize); + System.out.println("Probe input size: " + 100 * probeSize); + System.out.println("Available memory: " + this.memManager.getMemorySize()); + System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent."); + System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost)); + } + + @Test + public void compareMutableHashTablePerformance4() throws IOException { + // ----------------------------------------------0% filtered during probe spill phase----------------------------------------- + // create a build input with 1000000 records with key spread between [0 -- 1000000] with step of 1 for nearby records. + int buildSize = 1000000; + int buildStep = 1; + int buildScope = buildStep * buildSize; + // create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records. + int probeSize = 5000000; + int probeStep = 1; + int probeScope = buildSize; + + int expectedResult = probeSize / buildStep; + + long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true); + long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false); + + System.out.println("HybridHashJoin5:"); + System.out.println("Build input size: " + 100 * buildSize); + System.out.println("Probe input size: " + 100 * probeSize); + System.out.println("Available memory: " + this.memManager.getMemorySize()); + System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent."); + System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost)); + } + + private long hybridHashJoin(int buildSize, int buildStep, int buildScope, int probeSize, + int probeStep, int probeScope, int expectedResultSize, boolean enableBloomFilter) throws IOException { + + InputIterator buildIterator = new InputIterator(buildSize, buildStep, buildScope); + InputIterator probeIterator = new InputIterator(probeSize, probeStep, probeScope); + + Configuration conf = new Configuration(); + conf.setBoolean(ConfigConstants.HASHJOIN_ENABLE_BLOOMFILTER, enableBloomFilter); + GlobalConfiguration.includeConfiguration(conf); + + // allocate the memory for the HashTable + List memSegments; + try { + // 33 is minimum number of pages required to perform hash join this inputs + memSegments = this.memManager.allocatePages(MEM_OWNER, (int) (this.memManager.getMemorySize() / this.memManager.getPageSize())); + } catch (MemoryAllocationException maex) { + fail("Memory for the Join could not be provided."); + return -1; + } + + // ---------------------------------------------------------------------------------------- + + long start = System.currentTimeMillis(); + final MutableHashTable join = new MutableHashTable( + this.pairBuildSideAccesssor, this.pairProbeSideAccesssor, + this.pairBuildSideComparator, this.pairProbeSideComparator, this.pairComparator, + memSegments, ioManager); + join.open(buildIterator, probeIterator); + + final StringPair recordReuse = new StringPair(); + int numRecordsInJoinResult = 0; + + while (join.nextRecord()) { + MutableHashTable.HashBucketIterator buildSide = join.getBuildSideIterator(); + while (buildSide.next(recordReuse) != null) { + numRecordsInJoinResult++; + } + } + Assert.assertEquals("Wrong number of records in join result.", expectedResultSize, numRecordsInJoinResult); + + join.close(); + long cost = System.currentTimeMillis() - start; + // ---------------------------------------------------------------------------------------- + + this.memManager.release(join.getFreedMemory()); + return cost; + } + + + static class InputIterator implements MutableObjectIterator { + + private int numLeft; + private int distance; + private int scope; + + public InputIterator(int size, int distance, int scope) { + this.numLeft = size; + this.distance = distance; + this.scope = scope; + } + + @Override + public StringPair next(StringPair reuse) throws IOException { + if (this.numLeft > 0) { + numLeft--; + int currentKey = (numLeft * distance) % scope; + reuse.setKey(Integer.toString(currentKey)); + reuse.setValue(COMMENT); + return reuse; + } else { + return null; + } + } + + @Override + public StringPair next() throws IOException { + return next(new StringPair()); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java new file mode 100644 index 0000000000000..cbbeca017e140 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java @@ -0,0 +1,162 @@ +/** + * 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.runtime.operators.util; + +import org.apache.flink.core.memory.MemorySegment; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class BloomFilterTest { + + private static BloomFilter bloomFilter; + private static final int INPUT_SIZE = 1024; + private static final double FALSE_POSITIVE_PROBABILITY = 0.05; + + @BeforeClass + public static void init() { + int bitsSize = BloomFilter.optimalNumOfBits(INPUT_SIZE, FALSE_POSITIVE_PROBABILITY); + bitsSize = bitsSize + (Long.SIZE - (bitsSize % Long.SIZE)); + int byteSize = bitsSize >>> 3; + MemorySegment memorySegment = new MemorySegment(new byte[byteSize]); + bloomFilter = new BloomFilter(INPUT_SIZE, byteSize); + bloomFilter.setBitsLocation(memorySegment, 0); + } + + @Test(expected = IllegalArgumentException.class) + public void testBloomFilterArguments1() { + new BloomFilter(-1, 128); + } + + @Test(expected = IllegalArgumentException.class) + public void testBloomFilterArguments2() { + new BloomFilter(0, 128); + } + + @Test(expected = IllegalArgumentException.class) + public void testBloomFilterArguments3() { + new BloomFilter(1024, -1); + } + + @Test(expected = IllegalArgumentException.class) + public void testBloomFilterArguments4() { + new BloomFilter(1024, 0); + } + + @Test(expected = IllegalArgumentException.class) + public void testBloomFilterArguments5() { + new BloomFilter(1024, 21); + } + + @Test + public void testBloomNumBits() { + assertEquals(0, BloomFilter.optimalNumOfBits(0, 0)); + assertEquals(0, BloomFilter.optimalNumOfBits(0, 1)); + assertEquals(0, BloomFilter.optimalNumOfBits(1, 1)); + assertEquals(7, BloomFilter.optimalNumOfBits(1, 0.03)); + assertEquals(72, BloomFilter.optimalNumOfBits(10, 0.03)); + assertEquals(729, BloomFilter.optimalNumOfBits(100, 0.03)); + assertEquals(7298, BloomFilter.optimalNumOfBits(1000, 0.03)); + assertEquals(72984, BloomFilter.optimalNumOfBits(10000, 0.03)); + assertEquals(729844, BloomFilter.optimalNumOfBits(100000, 0.03)); + assertEquals(7298440, BloomFilter.optimalNumOfBits(1000000, 0.03)); + assertEquals(6235224, BloomFilter.optimalNumOfBits(1000000, 0.05)); + } + + @Test + public void testBloomFilterNumHashFunctions() { + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(-1, -1)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(0, 0)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(10, 0)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(10, 10)); + assertEquals(7, BloomFilter.optimalNumOfHashFunctions(10, 100)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(100, 100)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(1000, 100)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(10000, 100)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(100000, 100)); + assertEquals(1, BloomFilter.optimalNumOfHashFunctions(1000000, 100)); + } + + @Test + public void testBloomFilterFalsePositiveProbability() { + assertEquals(7298440, BloomFilter.optimalNumOfBits(1000000, 0.03)); + assertEquals(6235224, BloomFilter.optimalNumOfBits(1000000, 0.05)); + assertEquals(4792529, BloomFilter.optimalNumOfBits(1000000, 0.1)); + assertEquals(3349834, BloomFilter.optimalNumOfBits(1000000, 0.2)); + assertEquals(2505911, BloomFilter.optimalNumOfBits(1000000, 0.3)); + assertEquals(1907139, BloomFilter.optimalNumOfBits(1000000, 0.4)); + + // Make sure the estimated fpp error is less than 1%. + assertTrue(Math.abs(BloomFilter.estimateFalsePositiveProbability(1000000, 7298440) - 0.03) < 0.01); + assertTrue(Math.abs(BloomFilter.estimateFalsePositiveProbability(1000000, 6235224) - 0.05) < 0.01); + assertTrue(Math.abs(BloomFilter.estimateFalsePositiveProbability(1000000, 4792529) - 0.1) < 0.01); + assertTrue(Math.abs(BloomFilter.estimateFalsePositiveProbability(1000000, 3349834) - 0.2) < 0.01); + assertTrue(Math.abs(BloomFilter.estimateFalsePositiveProbability(1000000, 2505911) - 0.3) < 0.01); + assertTrue(Math.abs(BloomFilter.estimateFalsePositiveProbability(1000000, 1907139) - 0.4) < 0.01); + } + + @Test + public void testHashcodeInput() { + bloomFilter.reset(); + int val1 = "val1".hashCode(); + int val2 = "val2".hashCode(); + int val3 = "val3".hashCode(); + int val4 = "val4".hashCode(); + int val5 = "val5".hashCode(); + + assertFalse(bloomFilter.testHash(val1)); + assertFalse(bloomFilter.testHash(val2)); + assertFalse(bloomFilter.testHash(val3)); + assertFalse(bloomFilter.testHash(val4)); + assertFalse(bloomFilter.testHash(val5)); + bloomFilter.addHash(val1); + assertTrue(bloomFilter.testHash(val1)); + assertFalse(bloomFilter.testHash(val2)); + assertFalse(bloomFilter.testHash(val3)); + assertFalse(bloomFilter.testHash(val4)); + assertFalse(bloomFilter.testHash(val5)); + bloomFilter.addHash(val2); + assertTrue(bloomFilter.testHash(val1)); + assertTrue(bloomFilter.testHash(val2)); + assertFalse(bloomFilter.testHash(val3)); + assertFalse(bloomFilter.testHash(val4)); + assertFalse(bloomFilter.testHash(val5)); + bloomFilter.addHash(val3); + assertTrue(bloomFilter.testHash(val1)); + assertTrue(bloomFilter.testHash(val2)); + assertTrue(bloomFilter.testHash(val3)); + assertFalse(bloomFilter.testHash(val4)); + assertFalse(bloomFilter.testHash(val5)); + bloomFilter.addHash(val4); + assertTrue(bloomFilter.testHash(val1)); + assertTrue(bloomFilter.testHash(val2)); + assertTrue(bloomFilter.testHash(val3)); + assertTrue(bloomFilter.testHash(val4)); + assertFalse(bloomFilter.testHash(val5)); + bloomFilter.addHash(val5); + assertTrue(bloomFilter.testHash(val1)); + assertTrue(bloomFilter.testHash(val2)); + assertTrue(bloomFilter.testHash(val3)); + assertTrue(bloomFilter.testHash(val4)); + assertTrue(bloomFilter.testHash(val5)); + } +} \ No newline at end of file From 0b73b4387a855627209a4dbaef930321a5090594 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 15:49:07 +0200 Subject: [PATCH 093/175] [FLINK-2240] [runtime] Pass flag to configure use of bloom filters through runtime configuration. Also make sure that most tests run with enabled bloom filters, to increase test coverage. --- docs/setup/config.md | 15 +- .../flink/configuration/ConfigConstants.java | 26 +-- .../flink/runtime/execution/Environment.java | 12 +- .../AbstractCachedBuildSideJoinDriver.java | 20 +- .../flink/runtime/operators/JoinDriver.java | 65 ++++-- .../flink/runtime/operators/PactDriver.java | 10 +- .../runtime/operators/PactTaskContext.java | 9 +- .../runtime/operators/RegularPactTask.java | 24 ++- .../operators/hash/HashMatchIteratorBase.java | 11 +- .../operators/hash/MutableHashTable.java | 192 ++++-------------- ...NonReusingBuildFirstHashMatchIterator.java | 8 +- ...BuildFirstReOpenableHashMatchIterator.java | 26 ++- ...onReusingBuildSecondHashMatchIterator.java | 8 +- ...uildSecondReOpenableHashMatchIterator.java | 23 ++- .../hash/ReOpenableMutableHashTable.java | 18 +- .../ReusingBuildFirstHashMatchIterator.java | 8 +- ...BuildFirstReOpenableHashMatchIterator.java | 22 +- .../ReusingBuildSecondHashMatchIterator.java | 8 +- ...uildSecondReOpenableHashMatchIterator.java | 26 ++- .../taskmanager/RuntimeEnvironment.java | 16 +- .../operators/drivers/TestTaskContext.java | 9 + .../MutableHashTablePerformanceBenchmark.java | 10 +- .../NonReusingHashMatchIteratorITCase.java | 12 +- .../NonReusingReOpenableHashTableITCase.java | 33 ++- .../hash/ReusingHashMatchIteratorITCase.java | 12 +- .../ReusingReOpenableHashTableITCase.java | 33 ++- .../operators/testutils/DriverTestBase.java | 18 +- .../operators/testutils/MockEnvironment.java | 11 +- .../testutils/UnaryOperatorTestBase.java | 12 +- .../util/HashVsSortMiniBenchmark.java | 4 +- .../runtime/tasks/StreamMockEnvironment.java | 14 +- 31 files changed, 338 insertions(+), 377 deletions(-) diff --git a/docs/setup/config.md b/docs/setup/config.md index ba541d4ba5a6c..53b9ae0fb82bd 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -244,11 +244,6 @@ free for objects created by user-defined functions. (DEFAULT: 0.7) This parameter is only evaluated, if `taskmanager.memory.size` is not set. - `jobclient.polling.interval`: The interval (in seconds) in which the client polls the JobManager for the status of its job (DEFAULT: 2). -- `taskmanager.runtime.max-fan`: The maximal fan-in for external merge joins and -fan-out for spilling hash tables. Limits the number of file handles per operator, -but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128). -- `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling -when this fraction of its memory budget is full (DEFAULT: 0.8). - `taskmanager.heartbeat-interval`: The interval in which the TaskManager sends heartbeats to the JobManager. - `jobmanager.max-heartbeat-delay-before-failure.msecs`: The maximum time that a @@ -324,6 +319,16 @@ sample exceeds this value (possible because of misconfiguration of the parser), the sampling aborts. This value can be overridden for a specific input with the input format's parameters (DEFAULT: 2097152 (= 2 MiBytes)). +### Runtime Algorithms + +- `taskmanager.runtime.max-fan`: The maximal fan-in for external merge joins and +fan-out for spilling hash tables. Limits the number of file handles per operator, +but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128). +- `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling +when this fraction of its memory budget is full (DEFAULT: 0.8). +- `taskmanager.runtime.hashjoin-bloom-filters`: If true, the hash join uses bloom filters to pre-filter records against spilled partitions. (DEFAULT: true) + + ## YARN diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index dad2d99d183fe..d145eb24e0abb 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -172,6 +172,8 @@ public final class ConfigConstants { */ public static final String TASK_MANAGER_MAX_REGISTRATION_DURATION = "taskmanager.maxRegistrationDuration"; + // --------------------------- Runtime Algorithms ------------------------------- + /** * Parameter for the maximum fan for out-of-core algorithms. * Corresponds to the maximum fan-in for merge-sorts and the maximum fan-out @@ -184,18 +186,17 @@ public final class ConfigConstants { * sorter will start spilling to disk. */ public static final String DEFAULT_SORT_SPILLING_THRESHOLD_KEY = "taskmanager.runtime.sort-spilling-threshold"; + + /** + * Parameter to switch hash join bloom filters for spilled partitions on and off. + */ + public static final String RUNTIME_HASH_JOIN_BLOOM_FILTERS_KEY = "taskmanager.runtime.hashjoin-bloom-filters"; /** * The config parameter defining the timeout for filesystem stream opening. * A value of 0 indicates infinite waiting. */ public static final String FS_STREAM_OPENING_TIMEOUT_KEY = "taskmanager.runtime.fs_timeout"; - - /** - * While spill probe record to disk during probe phase, whether enable bloom filter to filter the probe records - * to minimize the spilled probe records count. - */ - public static final String HASHJOIN_ENABLE_BLOOMFILTER = "hashjoin.bloomfilter.enabled"; // ------------------------ YARN Configuration ------------------------ @@ -543,6 +544,13 @@ public final class ConfigConstants { * The default task manager's maximum registration duration */ public static final String DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION = "Inf"; + + // ------------------------ Runtime Algorithms ------------------------ + + /** + * Default setting for the switch for hash join bloom filters for spilled partitions. + */ + public static final boolean DEFAULT_RUNTIME_HASH_JOIN_BLOOM_FILTERS = true; /** * The default value for the maximum spilling fan in/out. @@ -558,15 +566,9 @@ public final class ConfigConstants { * The default timeout for filesystem stream opening: infinite (means max long milliseconds). */ public static final int DEFAULT_FS_STREAM_OPENING_TIMEOUT = 0; - - /** - * Enable bloom filter for hash join as it promote hash join performance most of the time. - */ - public static final boolean DEAFULT_HASHJOIN_ENABLE_BLOOMFILTER = true; // ------------------------ YARN Configuration ------------------------ - /** * Minimum amount of Heap memory to subtract from the requested TaskManager size. * We came up with these values experimentally. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index af295600e82c2..c742ce5095ff3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import java.util.Map; import java.util.concurrent.Future; @@ -72,14 +73,11 @@ public interface Environment { Configuration getTaskConfiguration(); /** - * @return The task manager configuration - */ - Configuration getTaskManagerConfiguration(); - - /** - * @return Hostname of the task manager + * Gets the task manager info, with configuration and hostname. + * + * @return The task manager info, with configuration and hostname. */ - String getHostname(); + TaskManagerRuntimeInfo getTaskManagerInfo(); /** * Returns the job-wide configuration object that was attached to the JobGraph. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java index aff8d01c70b0d..4096f0c7c8319 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstReOpenableHashMatchIterator; import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondReOpenableHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstReOpenableHashMatchIterator; @@ -74,7 +75,10 @@ public void initialize() throws Exception { this.taskContext.getTaskConfig().getPairComparatorFactory(this.taskContext.getUserCodeClassLoader()); double availableMemory = config.getRelativeMemoryDriver(); - + boolean hashJoinUseBitMaps = taskContext.getTaskManagerInfo().getConfiguration().getBoolean( + ConfigConstants.RUNTIME_HASH_JOIN_BLOOM_FILTERS_KEY, + ConfigConstants.DEFAULT_RUNTIME_HASH_JOIN_BLOOM_FILTERS); + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); objectReuseEnabled = executionConfig.isObjectReuseEnabled(); @@ -89,7 +93,8 @@ public void initialize() throws Exception { this.taskContext.getMemoryManager(), this.taskContext.getIOManager(), this.taskContext.getOwningNepheleTask(), - availableMemory); + availableMemory, + hashJoinUseBitMaps); } else if (buildSideIndex == 1 && probeSideIndex == 0) { @@ -102,7 +107,8 @@ public void initialize() throws Exception { this.taskContext.getMemoryManager(), this.taskContext.getIOManager(), this.taskContext.getOwningNepheleTask(), - availableMemory); + availableMemory, + hashJoinUseBitMaps); } else { throw new Exception("Error: Inconsistent setup for repeatable hash join driver."); @@ -118,7 +124,8 @@ public void initialize() throws Exception { this.taskContext.getMemoryManager(), this.taskContext.getIOManager(), this.taskContext.getOwningNepheleTask(), - availableMemory); + availableMemory, + hashJoinUseBitMaps); } else if (buildSideIndex == 1 && probeSideIndex == 0) { @@ -131,7 +138,8 @@ public void initialize() throws Exception { this.taskContext.getMemoryManager(), this.taskContext.getIOManager(), this.taskContext.getOwningNepheleTask(), - availableMemory); + availableMemory, + hashJoinUseBitMaps); } else { throw new Exception("Error: Inconsistent setup for repeatable hash join driver."); @@ -148,12 +156,10 @@ public void prepare() throws Exception { @Override public void run() throws Exception { - final FlatJoinFunction matchStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); while (this.running && matchIterator != null && matchIterator.callWithNextKey(matchStub, collector)); - } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java index af3da5590eecd..5df715fd8ac37 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java @@ -19,25 +19,27 @@ package org.apache.flink.runtime.operators; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator; -import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator; -import org.apache.flink.runtime.operators.sort.NonReusingMergeInnerJoinIterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator; +import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; +import org.apache.flink.runtime.operators.sort.NonReusingMergeInnerJoinIterator; import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * The join driver implements the logic of a join operator at runtime. It instantiates either * hash or sort-merge based strategies to find joining pairs of records. @@ -115,19 +117,40 @@ public void prepare() throws Exception{ if (LOG.isDebugEnabled()) { LOG.debug("Join Driver object reuse: " + (objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); } + + boolean hashJoinUseBitMaps = taskContext.getTaskManagerInfo().getConfiguration().getBoolean( + ConfigConstants.RUNTIME_HASH_JOIN_BLOOM_FILTERS_KEY, + ConfigConstants.DEFAULT_RUNTIME_HASH_JOIN_BLOOM_FILTERS); // create and return joining iterator according to provided local strategy. if (objectReuseEnabled) { switch (ls) { case MERGE: - this.joinIterator = new ReusingMergeInnerJoinIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); - + this.joinIterator = new ReusingMergeInnerJoinIterator<>(in1, in2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator12(comparator1, comparator2), + memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: - this.joinIterator = new ReusingBuildFirstHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new ReusingBuildFirstHashMatchIterator<>(in1, in2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator21(comparator1, comparator2), + memoryManager, ioManager, + this.taskContext.getOwningNepheleTask(), + fractionAvailableMemory, + hashJoinUseBitMaps); break; case HYBRIDHASH_BUILD_SECOND: - this.joinIterator = new ReusingBuildSecondHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new ReusingBuildSecondHashMatchIterator<>(in1, in2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator12(comparator1, comparator2), + memoryManager, ioManager, + this.taskContext.getOwningNepheleTask(), + fractionAvailableMemory, + hashJoinUseBitMaps); break; default: throw new Exception("Unsupported driver strategy for join driver: " + ls.name()); @@ -135,14 +158,32 @@ public void prepare() throws Exception{ } else { switch (ls) { case MERGE: - this.joinIterator = new NonReusingMergeInnerJoinIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.joinIterator = new NonReusingMergeInnerJoinIterator<>(in1, in2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator12(comparator1, comparator2), + memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: - this.joinIterator = new NonReusingBuildFirstHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new NonReusingBuildFirstHashMatchIterator<>(in1, in2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator21(comparator1, comparator2), + memoryManager, ioManager, + this.taskContext.getOwningNepheleTask(), + fractionAvailableMemory, + hashJoinUseBitMaps); break; case HYBRIDHASH_BUILD_SECOND: - this.joinIterator = new NonReusingBuildSecondHashMatchIterator<>(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + this.joinIterator = new NonReusingBuildSecondHashMatchIterator<>(in1, in2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator12(comparator1, comparator2), + memoryManager, ioManager, + this.taskContext.getOwningNepheleTask(), + fractionAvailableMemory, + hashJoinUseBitMaps); break; default: throw new Exception("Unsupported driver strategy for join driver: " + ls.name()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java index a53f5bf23826c..288f7ca46b6cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java @@ -16,16 +16,14 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import org.apache.flink.api.common.functions.Function; - /** - * The interface to be implemented by all pact drivers that run alone (or as the primary driver) in a nephele task. - * The driver is the code that deals with everything that specific to a certain PACT. It implements the actual - * map or reduce specific code. + * The interface to be implemented by all drivers that run alone (or as the primary driver) in a task. + * A driver implements the actual code to perform a batch operation, like map(), + * reduce(), join(), or coGroup(). * * @see PactTaskContext * @@ -37,7 +35,7 @@ public interface PactDriver { void setup(PactTaskContext context); /** - * Gets the number of inputs (= Nephele Gates and Readers) that the task has. + * Gets the number of inputs that the task has. * * @return The number of inputs. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java index bc23fa33be910..5c2ed679e0657 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java @@ -26,15 +26,14 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.util.TaskConfig; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; /** - * A runtime task is the task that is executed by the flink engine inside a task vertex. - * It typically has a {@link PactDriver}, and optionally multiple chained drivers. In addition, it - * deals with the runtime setup and teardown and the control-flow logic. The latter appears especially - * in the case of iterations. + * The task context gives a driver (e.g., {@link MapDriver}, or {@link JoinDriver}) access to + * the runtime components and configuration that they can use to fulfil their task. * * @param The UDF type. * @param The produced data type. @@ -44,6 +43,8 @@ public interface PactTaskContext { TaskConfig getTaskConfig(); + + TaskManagerRuntimeInfo getTaskManagerInfo(); ClassLoader getUserCodeClassLoader(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 78bf3836de608..873d9486925eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -60,6 +60,7 @@ import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; @@ -660,7 +661,7 @@ protected S initStub(Class stubSuperClass) throws Exception { */ protected void initInputReaders() throws Exception { final int numInputs = getNumTaskInputs(); - final MutableReader[] inputReaders = new MutableReader[numInputs]; + final MutableReader[] inputReaders = new MutableReader[numInputs]; int currentReaderOffset = 0; @@ -705,7 +706,7 @@ protected void initInputReaders() throws Exception { */ protected void initBroadcastInputReaders() throws Exception { final int numBroadcastInputs = this.config.getNumBroadcastInputs(); - final MutableReader[] broadcastInputReaders = new MutableReader[numBroadcastInputs]; + final MutableReader[] broadcastInputReaders = new MutableReader[numBroadcastInputs]; int currentReaderOffset = config.getNumInputs(); @@ -737,8 +738,8 @@ protected void initBroadcastInputReaders() throws Exception { */ protected void initInputsSerializersAndComparators(int numInputs, int numComparators) throws Exception { this.inputSerializers = new TypeSerializerFactory[numInputs]; - this.inputComparators = numComparators > 0 ? new TypeComparator[numComparators] : null; - this.inputIterators = new MutableObjectIterator[numInputs]; + this.inputComparators = numComparators > 0 ? new TypeComparator[numComparators] : null; + this.inputIterators = new MutableObjectIterator[numInputs]; ClassLoader userCodeClassLoader = getUserCodeClassLoader(); @@ -764,7 +765,7 @@ protected void initInputsSerializersAndComparators(int numInputs, int numCompara * Creates all the serializers and iterators for the broadcast inputs. */ protected void initBroadcastInputsSerializers(int numBroadcastInputs) throws Exception { - this.broadcastInputSerializers = new TypeSerializerFactory[numBroadcastInputs]; + this.broadcastInputSerializers = new TypeSerializerFactory[numBroadcastInputs]; ClassLoader userCodeClassLoader = getUserCodeClassLoader(); @@ -787,8 +788,8 @@ protected void initLocalStrategies(int numInputs) throws Exception { final MemoryManager memMan = getMemoryManager(); final IOManager ioMan = getIOManager(); - this.localStrategies = new CloseableInputProvider[numInputs]; - this.inputs = new MutableObjectIterator[numInputs]; + this.localStrategies = new CloseableInputProvider[numInputs]; + this.inputs = new MutableObjectIterator[numInputs]; this.excludeFromReset = new boolean[numInputs]; this.inputIsCached = new boolean[numInputs]; this.inputIsAsyncMaterialized = new boolean[numInputs]; @@ -807,8 +808,8 @@ protected void initLocalStrategies(int numInputs) throws Exception { // acts as a pipeline breaker. this one should only be there, if a pipeline breaker is needed. // the second variant spills to the side and will not read unless the result is also consumed // in a pipelined fashion. - this.resettableInputs = new SpillingResettableMutableObjectIterator[numInputs]; - this.tempBarriers = new TempBarrier[numInputs]; + this.resettableInputs = new SpillingResettableMutableObjectIterator[numInputs]; + this.tempBarriers = new TempBarrier[numInputs]; for (int i = 0; i < numInputs; i++) { final int memoryPages; @@ -1043,6 +1044,11 @@ public TaskConfig getTaskConfig() { return this.config; } + @Override + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return getEnvironment().getTaskManagerInfo(); + } + @Override public MemoryManager getMemoryManager() { return getEnvironment().getMemoryManager(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java index 4e0112ab806ef..08b619103b6a6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java @@ -32,6 +32,7 @@ * Common methods for all Hash Join Iterators. */ public class HashMatchIteratorBase { + public MutableHashTable getHashJoin( TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, @@ -41,11 +42,15 @@ public MutableHashTable getHashJoin( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) throws MemoryAllocationException { + double memoryFraction, + boolean useBloomFilters) throws MemoryAllocationException { final int numPages = memManager.computeNumberOfPages(memoryFraction); final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new MutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + + return new MutableHashTable(buildSideSerializer, probeSideSerializer, + buildSideComparator, probeSideComparator, pairComparator, + memorySegments, ioManager, + useBloomFilters); } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java index 4a57986604a16..b0042fcb10010 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java @@ -24,10 +24,9 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.GlobalConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -46,22 +45,16 @@ import org.apache.flink.runtime.util.MathUtils; import org.apache.flink.util.MutableObjectIterator; - /** * An implementation of a Hybrid Hash Join. The join starts operating in memory and gradually starts * spilling contents to disk, when the memory is not sufficient. It does not need to know a priori * how large the input will be. - *

    - * The design of this class follows on many parts the design presented in - * "Hash joins and hash teams in Microsoft SQL Server", by Goetz Graefe et al. In its current state, the - * implementation lacks features like dynamic role reversal, partition tuning, or histogram guided partitioning. - *

    - * - * - *


    * - * The layout of the buckets inside a memory segment is as follows: + *

    The design of this class follows on many parts the design presented in + * "Hash joins and hash teams in Microsoft SQL Server", by Goetz Graefe et al. In its current state, the + * implementation lacks features like dynamic role reversal, partition tuning, or histogram guided partitioning.

    * + *

    The layout of the buckets inside a memory segment is as follows:

    *
      * +----------------------------- Bucket x ----------------------------
      * |Partition (1 byte) | Status (1 byte) | element count (2 bytes) |
    @@ -189,8 +182,6 @@ public class MutableHashTable implements MemorySegmentSource {
     	 */
     	private static final long BUCKET_FORWARD_POINTER_NOT_SET = ~0x0L;
     	
    -//	private static final byte BUCKET_STATUS_SPILLED = 1;
    -	
     	/**
     	 * Constant for the bucket status, indicating that the bucket is in memory.
     	 */
    @@ -274,11 +265,14 @@ public class MutableHashTable implements MemorySegmentSource {
     	 */
     	protected final int bucketsPerSegmentBits;
     	
    -	/**
    +	/** 
     	 * An estimate for the average record length.
     	 */
     	private final int avgRecordLen;
     	
    +	/** Flag to enable/disable bloom filters for spilled partitions */
    +	private final boolean useBloomFilters;
    +	
     	// ------------------------------------------------------------------------
     	
     	/**
    @@ -296,8 +290,6 @@ public class MutableHashTable implements MemorySegmentSource {
     	 */
     	private HashBucketIterator bucketIterator;
     	
    -//	private LazyHashBucketIterator lazyBucketIterator;
    -	
     	/**
     	 * Iterator over the elements from the probe side.
     	 */
    @@ -319,6 +311,10 @@ public class MutableHashTable implements MemorySegmentSource {
     	 * of hash-codes and pointers to the elements.
     	 */
     	protected MemorySegment[] buckets;
    +
    +	/** The bloom filter utility used to transform hash buckets of spilled partitions into a
    +	 * probabilistic filter */
    +	private BloomFilter bloomFilter;
     	
     	/**
     	 * The number of buckets in the current table. The bucket array is not necessarily fully
    @@ -353,25 +349,35 @@ public class MutableHashTable implements MemorySegmentSource {
     	protected boolean furtherPartitioning = false;
     	
     	private boolean running = true;
    -
    -	private BloomFilter bloomFilter;
     	
     	// ------------------------------------------------------------------------
     	//                         Construction and Teardown
     	// ------------------------------------------------------------------------
    +
    +	public MutableHashTable(TypeSerializer buildSideSerializer, TypeSerializer probeSideSerializer,
    +							TypeComparator buildSideComparator, TypeComparator probeSideComparator,
    +							TypePairComparator comparator,
    +							List memorySegments, IOManager ioManager)
    +	{
    +		this(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, comparator,
    +				memorySegments, ioManager, true);
    +	}
     	
     	public MutableHashTable(TypeSerializer buildSideSerializer, TypeSerializer probeSideSerializer,
     			TypeComparator buildSideComparator, TypeComparator probeSideComparator,
    -			TypePairComparator comparator, List memorySegments, IOManager ioManager)
    +			TypePairComparator comparator,
    +			List memorySegments,
    +			IOManager ioManager,
    +			boolean useBloomFilters)
     	{
     		this(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, comparator,
    -			memorySegments, ioManager, DEFAULT_RECORD_LEN);
    +			memorySegments, ioManager, DEFAULT_RECORD_LEN, useBloomFilters);
     	}
     	
     	public MutableHashTable(TypeSerializer buildSideSerializer, TypeSerializer probeSideSerializer,
     			TypeComparator buildSideComparator, TypeComparator probeSideComparator,
     			TypePairComparator comparator, List memorySegments,
    -			IOManager ioManager, int avgRecordLen)
    +			IOManager ioManager, int avgRecordLen, boolean useBloomFilters)
     	{
     		// some sanity checks first
     		if (memorySegments == null) {
    @@ -390,6 +396,7 @@ public MutableHashTable(TypeSerializer buildSideSerializer, TypeSerializer

    0 ? avgRecordLen : buildSideSerializer.getLength() == -1 ? DEFAULT_RECORD_LEN : buildSideSerializer.getLength(); @@ -551,16 +558,12 @@ protected boolean prepareNextPartition() throws IOException { } public boolean nextRecord() throws IOException { - + final boolean probeProcessing = processProbeIter(); - if(probeProcessing) { - return true; - } - return prepareNextPartition(); + return probeProcessing || prepareNextPartition(); } - public HashBucketIterator getMatchesFor(PT record) throws IOException - { + public HashBucketIterator getMatchesFor(PT record) throws IOException { final TypeComparator probeAccessors = this.probeSideComparator; final int hash = hash(probeAccessors.hash(record), this.currentRecursionDepth); final int posHashCode = hash % this.numBuckets; @@ -585,32 +588,6 @@ public HashBucketIterator getMatchesFor(PT record) throws IOException } } -// public LazyHashBucketIterator getLazyMatchesFor(PT record) throws IOException -// { -// final TypeComparator probeAccessors = this.probeSideComparator; -// final int hash = hash(probeAccessors.hash(record), this.currentRecursionDepth); -// final int posHashCode = hash % this.numBuckets; -// -// // get the bucket for the given hash code -// final int bucketArrayPos = posHashCode >> this.bucketsPerSegmentBits; -// final int bucketInSegmentOffset = (posHashCode & this.bucketsPerSegmentMask) << NUM_INTRA_BUCKET_BITS; -// final MemorySegment bucket = this.buckets[bucketArrayPos]; -// -// // get the basic characteristics of the bucket -// final int partitionNumber = bucket.get(bucketInSegmentOffset + HEADER_PARTITION_OFFSET); -// final HashPartition p = this.partitionsBeingBuilt.get(partitionNumber); -// -// // for an in-memory partition, process set the return iterators, else spill the probe records -// if (p.isInMemory()) { -// this.recordComparator.setReference(record); -// this.lazyBucketIterator.set(bucket, p.overflowSegments, p, hash, bucketInSegmentOffset); -// return this.lazyBucketIterator; -// } -// else { -// throw new IllegalStateException("Method is not applicable to partially spilled hash tables."); -// } -// } - public PT getCurrentProbeRecord() { return this.probeIterator.getCurrent(); } @@ -739,7 +716,7 @@ private void initBloomFilter(int numBuckets) { } } - final private int getEstimatedMaxBucketEntries(int numBuffers, int bufferSize, int numBuckets, int recordLenBytes) { + private int getEstimatedMaxBucketEntries(int numBuffers, int bufferSize, int numBuckets, int recordLenBytes) { final long totalSize = ((long) bufferSize) * numBuffers; final long numRecordsStorable = totalSize / (recordLenBytes + RECORD_OVERHEAD_BYTES); final long maxNumRecordsStorable = (MAX_RECURSION_DEPTH + 1) * numRecordsStorable; @@ -1092,9 +1069,7 @@ protected void initTable(int numBuckets, byte numPartitions) { this.buckets = table; this.numBuckets = numBuckets; - boolean enableBloomFilter = GlobalConfiguration.getBoolean( - ConfigConstants.HASHJOIN_ENABLE_BLOOMFILTER, ConfigConstants.DEAFULT_HASHJOIN_ENABLE_BLOOMFILTER); - if (enableBloomFilter) { + if (useBloomFilters) { initBloomFilter(numBuckets); } } @@ -1107,8 +1082,8 @@ protected void releaseTable() { this.numBuckets = 0; if (this.buckets != null) { - for (int i = 0; i < this.buckets.length; i++) { - this.availableMemory.add(this.buckets[i]); + for (MemorySegment bucket : this.buckets) { + this.availableMemory.add(bucket); } this.buckets = null; } @@ -1138,9 +1113,7 @@ protected int spillPartition() throws IOException { } final HashPartition p = partitions.get(largestPartNum); - boolean enableBloomFilter = GlobalConfiguration.getBoolean( - ConfigConstants.HASHJOIN_ENABLE_BLOOMFILTER, ConfigConstants.DEAFULT_HASHJOIN_ENABLE_BLOOMFILTER); - if (enableBloomFilter) { + if (useBloomFilters) { buildBloomFilterForBucketsInPartition(largestPartNum, p); } @@ -1196,7 +1169,7 @@ final void buildBloomFilterForBucket(int bucketInSegmentPos, MemorySegment bucke buildBloomFilterForExtraOverflowSegments(bucketInSegmentPos, bucket, p); } - final private void buildBloomFilterForExtraOverflowSegments(int bucketInSegmentPos, MemorySegment bucket, HashPartition p) { + private void buildBloomFilterForExtraOverflowSegments(int bucketInSegmentPos, MemorySegment bucket, HashPartition p) { int totalCount = 0; boolean skip = false; long forwardPointer = bucket.getLong(bucketInSegmentPos + HEADER_FORWARD_OFFSET); @@ -1207,7 +1180,7 @@ final private void buildBloomFilterForExtraOverflowSegments(int bucketInSegmentP break; } MemorySegment overflowSegment = p.overflowSegments[overflowSegNum]; - int bucketInOverflowSegmentOffset = (int) (forwardPointer & 0xffffffff); + int bucketInOverflowSegmentOffset = (int) forwardPointer; final int count = overflowSegment.getShort(bucketInOverflowSegmentOffset + HEADER_COUNT_OFFSET); totalCount += count; @@ -1587,93 +1560,6 @@ public void reset() { } } // end HashBucketIterator - - - // ====================================================================================================== - -// public static final class LazyHashBucketIterator { -// -// private final TypePairComparator comparator; -// -// private MemorySegment bucket; -// -// private MemorySegment[] overflowSegments; -// -// private HashPartition partition; -// -// private int bucketInSegmentOffset; -// -// private int searchHashCode; -// -// private int posInSegment; -// -// private int countInSegment; -// -// private int numInSegment; -// -// private LazyHashBucketIterator(TypePairComparator comparator) { -// this.comparator = comparator; -// } -// -// -// void set(MemorySegment bucket, MemorySegment[] overflowSegments, HashPartition partition, -// int searchHashCode, int bucketInSegmentOffset) { -// -// this.bucket = bucket; -// this.overflowSegments = overflowSegments; -// this.partition = partition; -// this.searchHashCode = searchHashCode; -// this.bucketInSegmentOffset = bucketInSegmentOffset; -// -// this.posInSegment = this.bucketInSegmentOffset + BUCKET_HEADER_LENGTH; -// this.countInSegment = bucket.getShort(bucketInSegmentOffset + HEADER_COUNT_OFFSET); -// this.numInSegment = 0; -// } -// -// public boolean next(BT target) { -// // loop over all segments that are involved in the bucket (original bucket plus overflow buckets) -// while (true) { -// -// while (this.numInSegment < this.countInSegment) { -// -// final int thisCode = this.bucket.getInt(this.posInSegment); -// this.posInSegment += HASH_CODE_LEN; -// -// // check if the hash code matches -// if (thisCode == this.searchHashCode) { -// // get the pointer to the pair -// final long pointer = this.bucket.getLong(this.bucketInSegmentOffset + -// BUCKET_POINTER_START_OFFSET + (this.numInSegment * POINTER_LEN)); -// this.numInSegment++; -// -// // check whether it is really equal, or whether we had only a hash collision -// LazyDeSerializable lds = (LazyDeSerializable) target; -// lds.setDeSerializer(this.partition, this.partition.getWriteView(), pointer); -// if (this.comparator.equalToReference(target)) { -// return true; -// } -// } -// else { -// this.numInSegment++; -// } -// } -// -// // this segment is done. check if there is another chained bucket -// final long forwardPointer = this.bucket.getLong(this.bucketInSegmentOffset + HEADER_FORWARD_OFFSET); -// if (forwardPointer == BUCKET_FORWARD_POINTER_NOT_SET) { -// return false; -// } -// -// final int overflowSegNum = (int) (forwardPointer >>> 32); -// this.bucket = this.overflowSegments[overflowSegNum]; -// this.bucketInSegmentOffset = (int) (forwardPointer & 0xffffffff); -// this.countInSegment = this.bucket.getShort(this.bucketInSegmentOffset + HEADER_COUNT_OFFSET); -// this.posInSegment = this.bucketInSegmentOffset + BUCKET_HEADER_LENGTH; -// this.numInSegment = 0; -// } -// } -// } - // ====================================================================================================== diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java index c2d780500d1ff..5000dab591008 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java @@ -67,16 +67,16 @@ public NonReusingBuildFirstHashMatchIterator( TypePairComparator pairComparator, MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + this.memManager = memManager; this.firstInput = firstInput; this.secondInput = secondInput; this.probeSideSerializer = serializer2; this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, comparator2, - pairComparator, memManager, ioManager, ownerTask, memoryFraction); + pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java index ee870a67619b3..af1626a376821 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java @@ -48,25 +48,32 @@ public NonReusingBuildFirstReOpenableHashMatchIterator( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + super(firstInput, secondInput, serializer1, comparator1, serializer2, comparator2, pairComparator, memManager, ioManager, ownerTask, - memoryFraction); + memoryFraction, useBitmapFilters); + reopenHashTable = (ReOpenableMutableHashTable) hashJoin; } @Override - public MutableHashTable getHashJoin(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, + public MutableHashTable getHashJoin( + TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) - throws MemoryAllocationException - { + MemoryManager memManager, IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + final int numPages = memManager.computeNumberOfPages(memoryFraction); final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + + return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, + buildSideComparator, probeSideComparator, pairComparator, + memorySegments, ioManager, useBitmapFilters); } /** @@ -76,5 +83,4 @@ public MutableHashTable getHashJoin(TypeSerializer buildSid public void reopenProbe(MutableObjectIterator probeInput) throws IOException { reopenHashTable.reopenProbe(probeInput); } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java index 6099ac76295ae..83952c93b41f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java @@ -66,16 +66,16 @@ public NonReusingBuildSecondHashMatchIterator( TypePairComparator pairComparator, MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + this.memManager = memManager; this.firstInput = firstInput; this.secondInput = secondInput; this.probeSideSerializer = serializer1; this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, - comparator1, pairComparator, memManager, ioManager, ownerTask, memoryFraction); + comparator1, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java index bc7e65b09b225..029be5ffd4838 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java @@ -48,11 +48,12 @@ public NonReusingBuildSecondReOpenableHashMatchIterator( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + super(firstInput, secondInput, serializer1, comparator1, serializer2, - comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction); + comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters); + reopenHashTable = (ReOpenableMutableHashTable) hashJoin; } @@ -62,12 +63,17 @@ public MutableHashTable getHashJoin( TypeComparator buildSideComparator, TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) - throws MemoryAllocationException - { + MemoryManager memManager, IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + final int numPages = memManager.computeNumberOfPages(memoryFraction); final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + + return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, + buildSideComparator, probeSideComparator, pairComparator, + memorySegments, ioManager, useBitmapFilters); } /** @@ -77,5 +83,4 @@ public MutableHashTable getHashJoin( public void reopenProbe(MutableObjectIterator probeInput) throws IOException { reopenHashTable.reopenProbe(probeInput); } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableMutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableMutableHashTable.java index 68199243208c4..fd5fcde377fb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableMutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableMutableHashTable.java @@ -33,17 +33,12 @@ public class ReOpenableMutableHashTable extends MutableHashTable { - /** - * Channel for the spilled partitions - */ + /** Channel for the spilled partitions */ private final FileIOChannel.Enumerator spilledInMemoryPartitions; - /** - * Stores the initial partitions and a list of the files that contain the spilled contents - */ + /** Stores the initial partitions and a list of the files that contain the spilled contents */ private List> initialPartitions; - /** * The values of these variables are stored here after the initial open() * Required to restore the initial state before each additional probe phase. @@ -58,16 +53,17 @@ public ReOpenableMutableHashTable(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, TypeComparator probeSideComparator, TypePairComparator comparator, - List memorySegments, IOManager ioManager) { + List memorySegments, IOManager ioManager, + boolean useBitmapFilters) { + super(buildSideSerializer, probeSideSerializer, buildSideComparator, - probeSideComparator, comparator, memorySegments, ioManager); + probeSideComparator, comparator, memorySegments, ioManager, useBitmapFilters); keepBuildSidePartitions = true; spilledInMemoryPartitions = ioManager.createChannelEnumerator(); } @Override - public void open(MutableObjectIterator buildSide, - MutableObjectIterator probeSide) throws IOException { + public void open(MutableObjectIterator buildSide, MutableObjectIterator probeSide) throws IOException { super.open(buildSide, probeSide); initialPartitions = new ArrayList>( partitionsBeingBuilt ); initialPartitionFanOut = (byte) partitionsBeingBuilt.size(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java index da76045016149..b4aaa95369489 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java @@ -71,9 +71,9 @@ public ReusingBuildFirstHashMatchIterator( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + this.memManager = memManager; this.firstInput = firstInput; this.secondInput = secondInput; @@ -83,7 +83,7 @@ public ReusingBuildFirstHashMatchIterator( this.tempBuildSideRecord = serializer1.createInstance(); this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, - comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction); + comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java index 55012712307b2..714a1f57a8094 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java @@ -48,25 +48,32 @@ public ReusingBuildFirstReOpenableHashMatchIterator( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { super(firstInput, secondInput, serializer1, comparator1, serializer2, comparator2, pairComparator, memManager, ioManager, ownerTask, - memoryFraction); + memoryFraction, useBitmapFilters); reopenHashTable = (ReOpenableMutableHashTable) hashJoin; } @Override - public MutableHashTable getHashJoin(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, + public MutableHashTable getHashJoin( + TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) - throws MemoryAllocationException - { + MemoryManager memManager, IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + final int numPages = memManager.computeNumberOfPages(memoryFraction); final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + + return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, + buildSideComparator, probeSideComparator, pairComparator, + memorySegments, ioManager, useBitmapFilters); } /** @@ -76,5 +83,4 @@ public MutableHashTable getHashJoin(TypeSerializer buildSid public void reopenProbe(MutableObjectIterator probeInput) throws IOException { reopenHashTable.reopenProbe(probeInput); } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java index a9435ef17c1ff..b7c3e29662bed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java @@ -71,9 +71,9 @@ public ReusingBuildSecondHashMatchIterator( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + this.memManager = memManager; this.firstInput = firstInput; this.secondInput = secondInput; @@ -83,7 +83,7 @@ public ReusingBuildSecondHashMatchIterator( this.tempBuildSideRecord = serializer2.createInstance(); this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, comparator1, pairComparator, - memManager, ioManager, ownerTask, memoryFraction); + memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java index 559d20a31bb3a..4b4cdf57d800d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java @@ -48,24 +48,31 @@ public ReusingBuildSecondReOpenableHashMatchIterator( MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, - double memoryFraction) - throws MemoryAllocationException - { + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + super(firstInput, secondInput, serializer1, comparator1, serializer2, - comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction); + comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters); + reopenHashTable = (ReOpenableMutableHashTable) hashJoin; } @Override - public MutableHashTable getHashJoin(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, + public MutableHashTable getHashJoin( + TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) - throws MemoryAllocationException - { + MemoryManager memManager, IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction, + boolean useBitmapFilters) throws MemoryAllocationException { + final int numPages = memManager.computeNumberOfPages(memoryFraction); final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + + return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, + buildSideComparator, probeSideComparator, pairComparator, + memorySegments, ioManager, useBitmapFilters); } /** @@ -75,5 +82,4 @@ public MutableHashTable getHashJoin(TypeSerializer buildSid public void reopenProbe(MutableObjectIterator probeInput) throws IOException { reopenHashTable.reopenProbe(probeInput); } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index cd6dbd633f76b..8cfc1c367f7a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -75,9 +75,7 @@ public class RuntimeEnvironment implements Environment { private final AccumulatorRegistry accumulatorRegistry; - private final Configuration taskManagerConfiguration; - - private final String hostname; + private final TaskManagerRuntimeInfo taskManagerInfo; // ------------------------------------------------------------------------ @@ -124,8 +122,7 @@ public RuntimeEnvironment( this.writers = checkNotNull(writers); this.inputGates = checkNotNull(inputGates); this.jobManager = checkNotNull(jobManager); - this.taskManagerConfiguration = checkNotNull(taskManagerInfo).getConfiguration(); - this.hostname = taskManagerInfo.getHostname(); + this.taskManagerInfo = checkNotNull(taskManagerInfo); } // ------------------------------------------------------------------------ @@ -176,13 +173,8 @@ public Configuration getTaskConfiguration() { } @Override - public Configuration getTaskManagerConfiguration(){ - return taskManagerConfiguration; - } - - @Override - public String getHostname(){ - return hostname; + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return taskManagerInfo; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index b1466c90ec634..30e417b87c71e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.operators.PactTaskContext; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.util.TaskConfig; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -62,6 +63,8 @@ public class TestTaskContext implements PactTaskContext { private ExecutionConfig executionConfig = new ExecutionConfig(); + private TaskManagerRuntimeInfo taskManageInfo; + // -------------------------------------------------------------------------------------------- // Constructors // -------------------------------------------------------------------------------------------- @@ -70,6 +73,7 @@ public TestTaskContext() {} public TestTaskContext(long memoryInBytes) { this.memoryManager = new DefaultMemoryManager(memoryInBytes,1 ,32 * 1024, true); + this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); } // -------------------------------------------------------------------------------------------- @@ -155,6 +159,11 @@ public IOManager getIOManager() { return null; } + @Override + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return this.taskManageInfo; + } + @Override @SuppressWarnings("unchecked") public MutableObjectIterator getInput(int index) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java index 452e4c11ca2ed..c0f8f5960f05c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java @@ -24,9 +24,6 @@ import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; @@ -40,6 +37,7 @@ import org.apache.flink.runtime.operators.testutils.types.StringPairPairComparator; import org.apache.flink.runtime.operators.testutils.types.StringPairSerializer; import org.apache.flink.util.MutableObjectIterator; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -192,10 +190,6 @@ private long hybridHashJoin(int buildSize, int buildStep, int buildScope, int pr InputIterator buildIterator = new InputIterator(buildSize, buildStep, buildScope); InputIterator probeIterator = new InputIterator(probeSize, probeStep, probeScope); - Configuration conf = new Configuration(); - conf.setBoolean(ConfigConstants.HASHJOIN_ENABLE_BLOOMFILTER, enableBloomFilter); - GlobalConfiguration.includeConfiguration(conf); - // allocate the memory for the HashTable List memSegments; try { @@ -212,7 +206,7 @@ private long hybridHashJoin(int buildSize, int buildStep, int buildScope, int pr final MutableHashTable join = new MutableHashTable( this.pairBuildSideAccesssor, this.pairProbeSideAccesssor, this.pairBuildSideComparator, this.pairProbeSideComparator, this.pairComparator, - memSegments, ioManager); + memSegments, ioManager, enableBloomFilter); join.open(buildIterator, probeIterator); final StringPair recordReuse = new StringPair(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java index f4d225184ab59..0d5a26ef1f6b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java @@ -155,7 +155,7 @@ public void testBuildFirst() { new NonReusingBuildFirstHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -242,7 +242,7 @@ public void testBuildFirstWithHighNumberOfCommonKeys() new NonReusingBuildFirstHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -291,7 +291,7 @@ public void testBuildSecond() { new NonReusingBuildSecondHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -378,7 +378,7 @@ public void testBuildSecondWithHighNumberOfCommonKeys() new NonReusingBuildSecondHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -425,7 +425,7 @@ public void testBuildFirstWithMixedDataTypes() { new NonReusingBuildSecondHashMatchIterator( input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator, - this.memoryManager, this.ioManager, this.parentTask, 1.0); + this.memoryManager, this.ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -472,7 +472,7 @@ public void testBuildSecondWithMixedDataTypes() { new NonReusingBuildFirstHashMatchIterator( input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.recordPairPairComparator, - this.memoryManager, this.ioManager, this.parentTask, 1.0); + this.memoryManager, this.ioManager, this.parentTask, 1.0, true); iterator.open(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java index f5105bbb99b64..306a370c63449 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java @@ -239,7 +239,7 @@ private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIte new NonReusingBuildFirstReOpenableHashMatchIterator( buildInput, probeInput, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); // do first join with both inputs @@ -277,7 +277,7 @@ private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIte // // - private final MutableObjectIterator getProbeInput(final int numKeys, + private MutableObjectIterator getProbeInput(final int numKeys, final int probeValsPerKey, final int repeatedValue1, final int repeatedValue2) { MutableObjectIterator probe1 = new UniformRecordGenerator(numKeys, probeValsPerKey, true); MutableObjectIterator probe2 = new ConstantsKeyValuePairsIterator(repeatedValue1, 17, 5); @@ -334,9 +334,9 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException final ReOpenableMutableHashTable join = new ReOpenableMutableHashTable( this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, - memSegments, ioManager); + memSegments, ioManager, true); - for(int probe = 0; probe < NUM_PROBES; probe++) { + for (int probe = 0; probe < NUM_PROBES; probe++) { // create a probe input that gives 10 million pairs with 10 values sharing a key MutableObjectIterator probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2); if(probe == 0) { @@ -348,9 +348,8 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException Record record; final Record recordReuse = new Record(); - while (join.nextRecord()) - { - int numBuildValues = 0; + while (join.nextRecord()) { + long numBuildValues = 0; final Record probeRec = join.getCurrentProbeRecord(); int key = probeRec.getField(0, IntValue.class).getValue(); @@ -370,10 +369,10 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException Long contained = map.get(key); if (contained == null) { - contained = Long.valueOf(numBuildValues); + contained = numBuildValues; } else { - contained = Long.valueOf(contained.longValue() + numBuildValues); + contained = contained + numBuildValues; } map.put(key, contained); @@ -450,11 +449,12 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException final ReOpenableMutableHashTable join = new ReOpenableMutableHashTable( this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, - memSegments, ioManager); - for(int probe = 0; probe < NUM_PROBES; probe++) { + memSegments, ioManager, true); + + for (int probe = 0; probe < NUM_PROBES; probe++) { // create a probe input that gives 10 million pairs with 10 values sharing a key MutableObjectIterator probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2); - if(probe == 0) { + if (probe == 0) { join.open(buildInput, probeInput); } else { join.reopenProbe(probeInput); @@ -462,9 +462,8 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException Record record; final Record recordReuse = new Record(); - while (join.nextRecord()) - { - int numBuildValues = 0; + while (join.nextRecord()) { + long numBuildValues = 0; final Record probeRec = join.getCurrentProbeRecord(); int key = probeRec.getField(0, IntValue.class).getValue(); @@ -484,10 +483,10 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException Long contained = map.get(key); if (contained == null) { - contained = Long.valueOf(numBuildValues); + contained = numBuildValues; } else { - contained = Long.valueOf(contained.longValue() + numBuildValues); + contained = contained + numBuildValues; } map.put(key, contained); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java index 18cd8d0da6825..f770ca440eb91 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java @@ -155,7 +155,7 @@ public void testBuildFirst() { new ReusingBuildFirstHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -242,7 +242,7 @@ public void testBuildFirstWithHighNumberOfCommonKeys() new ReusingBuildFirstHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -291,7 +291,7 @@ public void testBuildSecond() { new ReusingBuildSecondHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -378,7 +378,7 @@ public void testBuildSecondWithHighNumberOfCommonKeys() new ReusingBuildSecondHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -425,7 +425,7 @@ public void testBuildFirstWithMixedDataTypes() { new ReusingBuildSecondHashMatchIterator( input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator, - this.memoryManager, this.ioManager, this.parentTask, 1.0); + this.memoryManager, this.ioManager, this.parentTask, 1.0, true); iterator.open(); @@ -472,7 +472,7 @@ public void testBuildSecondWithMixedDataTypes() { new ReusingBuildFirstHashMatchIterator( input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.recordPairPairComparator, - this.memoryManager, this.ioManager, this.parentTask, 1.0); + this.memoryManager, this.ioManager, this.parentTask, 1.0, true); iterator.open(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java index 717288751435a..d302487451531 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java @@ -238,7 +238,7 @@ private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIte new ReusingBuildFirstReOpenableHashMatchIterator( buildInput, probeInput, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, - this.memoryManager, ioManager, this.parentTask, 1.0); + this.memoryManager, ioManager, this.parentTask, 1.0, true); iterator.open(); // do first join with both inputs @@ -276,7 +276,7 @@ private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIte // // - private final MutableObjectIterator getProbeInput(final int numKeys, + private MutableObjectIterator getProbeInput(final int numKeys, final int probeValsPerKey, final int repeatedValue1, final int repeatedValue2) { MutableObjectIterator probe1 = new UniformRecordGenerator(numKeys, probeValsPerKey, true); MutableObjectIterator probe2 = new ConstantsKeyValuePairsIterator(repeatedValue1, 17, 5); @@ -289,8 +289,7 @@ private final MutableObjectIterator getProbeInput(final int numKeys, } @Test - public void testSpillingHashJoinWithMassiveCollisions() throws IOException - { + public void testSpillingHashJoinWithMassiveCollisions() throws IOException { // the following two values are known to have a hash-code collision on the initial level. // we use them to make sure one partition grows over-proportionally large final int REPEATED_VALUE_1 = 40559; @@ -311,9 +310,6 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException builds.add(build2); builds.add(build3); MutableObjectIterator buildInput = new UnionIterator(builds); - - - // allocate the memory for the HashTable List memSegments; @@ -333,7 +329,7 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException final ReOpenableMutableHashTable join = new ReOpenableMutableHashTable( this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, - memSegments, ioManager); + memSegments, ioManager, true); for(int probe = 0; probe < NUM_PROBES; probe++) { // create a probe input that gives 10 million pairs with 10 values sharing a key @@ -347,9 +343,8 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException Record record; final Record recordReuse = new Record(); - while (join.nextRecord()) - { - int numBuildValues = 0; + while (join.nextRecord()) { + long numBuildValues = 0; final Record probeRec = join.getCurrentProbeRecord(); int key = probeRec.getField(0, IntValue.class).getValue(); @@ -369,10 +364,10 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException Long contained = map.get(key); if (contained == null) { - contained = Long.valueOf(numBuildValues); + contained = numBuildValues; } else { - contained = Long.valueOf(contained.longValue() + numBuildValues); + contained = contained + numBuildValues; } map.put(key, contained); @@ -449,8 +444,9 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException final ReOpenableMutableHashTable join = new ReOpenableMutableHashTable( this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, - memSegments, ioManager); - for(int probe = 0; probe < NUM_PROBES; probe++) { + memSegments, ioManager, true); + + for (int probe = 0; probe < NUM_PROBES; probe++) { // create a probe input that gives 10 million pairs with 10 values sharing a key MutableObjectIterator probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2); if(probe == 0) { @@ -463,7 +459,7 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException while (join.nextRecord()) { - int numBuildValues = 0; + long numBuildValues = 0; final Record probeRec = join.getCurrentProbeRecord(); int key = probeRec.getField(0, IntValue.class).getValue(); @@ -483,10 +479,10 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException Long contained = map.get(key); if (contained == null) { - contained = Long.valueOf(numBuildValues); + contained = numBuildValues; } else { - contained = Long.valueOf(contained.longValue() + numBuildValues); + contained = contained + numBuildValues; } map.put(key, contained); @@ -526,5 +522,4 @@ static Map> deepCopy(Map implements PactTaskContext> sorters; private final AbstractInvokable owner; - - private final Configuration config; - + private final TaskConfig taskConfig; + private final TaskManagerRuntimeInfo taskManageInfo; + protected final long perSortMem; protected final double perSortFractionMem; @@ -111,11 +112,9 @@ protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNu this.sorters = new ArrayList>(); this.owner = new DummyInvokable(); - - this.config = new Configuration(); - this.taskConfig = new TaskConfig(this.config); - + this.taskConfig = new TaskConfig(new Configuration()); this.executionConfig = executionConfig; + this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); } @Parameterized.Parameters @@ -279,7 +278,10 @@ public TaskConfig getTaskConfig() { return this.taskConfig; } - + @Override + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return this.taskManageInfo; + } @Override public ExecutionConfig getExecutionConfig() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index b71b01e49aa8c..51c7f93c6244b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.operators.testutils; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; @@ -43,6 +42,7 @@ import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.types.Record; import org.apache.flink.util.MutableObjectIterator; import org.mockito.invocation.InvocationOnMock; @@ -193,13 +193,8 @@ public Configuration getJobConfiguration() { } @Override - public Configuration getTaskManagerConfiguration(){ - return new UnmodifiableConfiguration(new Configuration()); - } - - @Override - public String getHostname(){ - return "localhost"; + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 1e25bab9d6e91..20edc20a2065b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.operators.ResettablePactDriver; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.util.TaskConfig; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -54,7 +55,9 @@ public class UnaryOperatorTestBase implements PactT protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024; - protected static final int PAGE_SIZE = 32 * 1024; + protected static final int PAGE_SIZE = 32 * 1024; + + private final TaskManagerRuntimeInfo taskManageInfo; private final IOManager ioManager; @@ -110,6 +113,8 @@ protected UnaryOperatorTestBase(ExecutionConfig executionConfig, long memory, in this.executionConfig = executionConfig; this.comparators = new ArrayList>(2); + + this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); } @Parameterized.Parameters @@ -291,6 +296,11 @@ public MemoryManager getMemoryManager() { return this.memManager; } + @Override + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return this.taskManageInfo; + } + @Override public MutableObjectIterator getInput(int index) { MutableObjectIterator in = this.input; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java index 38d999285c572..7debb081c0889 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java @@ -187,7 +187,7 @@ public void testBuildFirst() { new ReusingBuildFirstHashMatchIterator( input1, input2, this.serializer1.getSerializer(), this.comparator1, this.serializer2.getSerializer(), this.comparator2, this.pairComparator11, - this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE); + this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE, true); iterator.open(); @@ -226,7 +226,7 @@ public void testBuildSecond() { new ReusingBuildSecondHashMatchIterator( input1, input2, this.serializer1.getSerializer(), this.comparator1, this.serializer2.getSerializer(), this.comparator2, this.pairComparator11, - this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE); + this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE, true); iterator.open(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index 44013ef7e8d4e..9091fa744cc38 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.JobID; @@ -45,6 +46,7 @@ import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -86,7 +88,8 @@ public class StreamMockEnvironment implements Environment { private final int bufferSize; - public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) { + public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, long memorySize, + MockInputSplitProvider inputSplitProvider, int bufferSize) { this.jobConfiguration = jobConfig; this.taskConfiguration = taskConfig; this.inputs = new LinkedList(); @@ -293,13 +296,8 @@ public void acknowledgeCheckpoint(long checkpointId, StateHandle state) { } @Override - public Configuration getTaskManagerConfiguration(){ - return new UnmodifiableConfiguration(new Configuration()); - } - - @Override - public String getHostname(){ - return "localhost"; + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); } } From 441ebf1ff381a6b4d9b6b289766d814195e560cb Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 19:31:14 +0200 Subject: [PATCH 094/175] [hotfix] Fix TEZ task contect to reflect updated interface. --- .../java/org/apache/flink/tez/runtime/TezTask.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java index 90df992c15608..47fbad716b954 100644 --- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java +++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java @@ -21,11 +21,13 @@ import com.google.common.base.Preconditions; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; + import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.distributions.DataDistribution; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; +import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -39,10 +41,10 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; -import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.runtime.operators.util.CloseableInputProvider; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.runtime.operators.util.TaskConfig; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.tez.runtime.input.TezReaderIterator; import org.apache.flink.tez.runtime.output.TezChannelSelector; import org.apache.flink.tez.runtime.output.TezOutputEmitter; @@ -51,6 +53,7 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.MutableObjectIterator; + import org.apache.tez.runtime.library.api.KeyValueReader; import org.apache.tez.runtime.library.api.KeyValueWriter; @@ -269,6 +272,11 @@ public IOManager getIOManager() { return runtimeEnvironment.getIOManager(); } + @Override + public TaskManagerRuntimeInfo getTaskManagerInfo() { + return new TaskManagerRuntimeInfo("localhost", new Configuration()); + } + @Override public MutableObjectIterator getInput(int index) { if (index < 0 || index > this.driver.getNumberOfInputs()) { From f1dd914de21313a90c3799438f1318349dd5d6df Mon Sep 17 00:00:00 2001 From: vasia Date: Fri, 31 Jul 2015 22:12:18 +0200 Subject: [PATCH 095/175] [FLINK-2452] [Gelly] adds a playcount threshold to the MusicProfiles example This closes #968 --- .../flink/graph/example/MusicProfiles.java | 41 ++++++++++++------- .../test/example/MusicProfilesITCase.java | 2 +- 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java index a5352162719d7..0fc45bdc9120b 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java @@ -46,15 +46,17 @@ public class MusicProfiles implements ProgramDescription { /** - * This example demonstrates how to mix the "record" Flink API with the - * graph API. The input is a set triplets and - * a set of bad records,i.e. song ids that should not be trusted. Initially, - * we use the record API to filter out the bad records. Then, we use the - * graph API to create a user -> song weighted bipartite graph and compute - * the top song (most listened) per user. Then, we use the record API again, - * to create a user-user similarity graph, based on common songs, where two - * users that listen to the same song are connected. Finally, we use the - * graph API to run the label propagation community detection algorithm on + * This example demonstrates how to mix the DataSet Flink API with the Gelly API. + * The input is a set triplets and + * a set of bad records, i.e. song ids that should not be trusted. + * Initially, we use the DataSet API to filter out the bad records. + * Then, we use Gelly to create a user -> song weighted bipartite graph and compute + * the top song (most listened) per user. + * Then, we use the DataSet API again, to create a user-user similarity graph, + * based on common songs, where users that are listeners of the same song + * are connected. A user-defined threshold on the playcount value + * defines when a user is considered to be a listener of a song. + * Finally, we use the graph API to run the label propagation community detection algorithm on * the similarity graph. * * The triplets input is expected to be given as one triplet per line, @@ -116,7 +118,13 @@ public static void main(String[] args) throws Exception { * create an edge between each pair of its in-neighbors. */ DataSet> similarUsers = userSongGraph - .getEdges().groupBy(1) + .getEdges() + // filter out user-song edges that are below the playcount threshold + .filter(new FilterFunction>() { + public boolean filter(Edge edge) { + return (edge.getValue() > playcountThreshold); + } + }).groupBy(1) .reduceGroup(new CreateSimilarUserEdges()).distinct(); Graph similarUsersGraph = Graph.fromDataSet(similarUsers, @@ -241,6 +249,8 @@ public String getDescription() { private static String topTracksOutputPath = null; + private static int playcountThreshold = 0; + private static String communitiesOutputPath = null; private static int maxIterations = 10; @@ -248,10 +258,10 @@ public String getDescription() { private static boolean parseParameters(String[] args) { if(args.length > 0) { - if(args.length != 5) { + if(args.length != 6) { System.err.println("Usage: MusicProfiles " + " " - + " "); + + " "); return false; } @@ -259,15 +269,16 @@ private static boolean parseParameters(String[] args) { userSongTripletsInputPath = args[0]; mismatchesInputPath = args[1]; topTracksOutputPath = args[2]; - communitiesOutputPath = args[3]; - maxIterations = Integer.parseInt(args[4]); + playcountThreshold = Integer.parseInt(args[3]); + communitiesOutputPath = args[4]; + maxIterations = Integer.parseInt(args[5]); } else { System.out.println("Executing Music Profiles example with default parameters and built-in default data."); System.out.println(" Provide parameters to read input data from files."); System.out.println(" See the documentation for the correct format of input files."); System.out.println("Usage: MusicProfiles " + " " - + " "); + + " "); } return true; } diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java index 0410d417472e7..5aa9f26f3785a 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java @@ -73,7 +73,7 @@ public void before() throws Exception { @Test public void testMusicProfilesExample() throws Exception { - MusicProfiles.main(new String[]{tripletsPath, mismatchesPath, topSongsResultPath, communitiesResultPath, + MusicProfiles.main(new String[]{tripletsPath, mismatchesPath, topSongsResultPath, "0", communitiesResultPath, MusicProfilesData.MAX_ITERATIONS + ""}); expectedTopSongs = MusicProfilesData.TOP_SONGS_RESULT; } From 6d9eeb559a895d92bb0a71c6535c55dfb49a16cb Mon Sep 17 00:00:00 2001 From: szape Date: Fri, 19 Jun 2015 10:22:04 +0200 Subject: [PATCH 096/175] [FLINK-2243] [storm-compat] Added finite spout functionality to Storm compatibility layer --- .../api/FlinkTopologyBuilder.java | 13 ++- .../wrappers/AbstractStormSpoutWrapper.java | 1 - .../wrappers/FiniteStormSpout.java | 37 ++++++++ .../wrappers/FiniteStormSpoutWrapper.java | 87 +++++++++++++++++++ .../wrappers/FiniteStormSpoutWrapperTest.java | 69 +++++++++++++++ 5 files changed, 205 insertions(+), 2 deletions(-) create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java index 4ecf4a6acdc67..d1462506e21a7 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java @@ -33,6 +33,9 @@ import backtype.storm.tuple.Fields; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.stormcompatibility.wrappers.AbstractStormSpoutWrapper; +import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; +import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper; import org.apache.flink.streaming.api.datastream.DataStream; @@ -93,7 +96,15 @@ public FlinkTopology createTopology() { * -> add an additional output attribute tagging the output stream, and use .split() and .select() to split * the streams */ - final DataStreamSource source = env.addSource(new StormSpoutWrapper(userSpout), declarer.getOutputType()); + AbstractStormSpoutWrapper spoutWrapper; + + if (userSpout instanceof FiniteStormSpout) { + spoutWrapper = new FiniteStormSpoutWrapper((FiniteStormSpout) userSpout); + } else { + spoutWrapper = new StormSpoutWrapper(userSpout); + } + + final DataStreamSource source = env.addSource(spoutWrapper, declarer.getOutputType()); availableOperators.put(spoutId, source); int dop = 1; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java index 3021bcb338144..4e43a8a55d55a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java @@ -19,7 +19,6 @@ import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.topology.IRichSpout; - import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java new file mode 100644 index 0000000000000..58a4f7a11d246 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java @@ -0,0 +1,37 @@ +/* + * 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.stormcompatibility.wrappers; + +import backtype.storm.topology.IRichSpout; + +/** + * This interface represents a Storm spout that emits a finite number of records. Common Storm + * spouts emit infinite streams by default. To change this behaviour and take advantage of + * Flink's finite-source capabilities, the spout should implement this interface. To wrap + * {@link FiniteStormSpout} separately, use {@link FiniteStormSpoutWrapper}. + */ +public interface FiniteStormSpout extends IRichSpout { + + /** + * When returns true, the spout has reached the end of the stream. + * + * @return true, if the spout's stream reached its end, false otherwise + */ + public boolean reachedEnd(); + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java new file mode 100644 index 0000000000000..79135109b6e39 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java @@ -0,0 +1,87 @@ +/* + * 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.stormcompatibility.wrappers; + +/** + * A {@link FiniteStormSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls the wrapped + * {@link FiniteStormSpout}'s {@link FiniteStormSpout#nextTuple()} method until {@link + * FiniteStormSpout#reachedEnd()} is true. + */ +public class FiniteStormSpoutWrapper extends AbstractStormSpoutWrapper { + private static final long serialVersionUID = -218340336648247605L; + + private FiniteStormSpout finiteSpout; + + /** + * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link + * FiniteStormSpout spout} such that it can be used within a Flink streaming program. The + * output + * type will be one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * + * @param spout + * The Storm {@link FiniteStormSpout spout} to be used. @throws + * IllegalArgumentException If + * the number of declared output attributes is not with range [1;25]. + */ + public FiniteStormSpoutWrapper(FiniteStormSpout spout) + throws IllegalArgumentException { + super(spout); + this.finiteSpout = spout; + } + + /** + * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link + * FiniteStormSpout spout} such that it can be used within a Flink streaming program. The + * output + * type can be any type if parameter {@code rawOutput} is {@code true} and the spout's + * number of + * declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be + * one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared number of + * attributes. + * + * @param spout + * The Storm {@link FiniteStormSpout spout} to be used. + * @param rawOutput + * Set to {@code true} if a single attribute output stream, should not be of type {@link + * Tuple1} but be of a raw type. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is + * not 1 + * or if {@code rawOuput} is {@code false} and the number of declared output attributes + * is not + * with range [1;25]. + */ + public FiniteStormSpoutWrapper(final FiniteStormSpout spout, final boolean rawOutput) + throws IllegalArgumentException { + super(spout, rawOutput); + this.finiteSpout = spout; + } + + /** + * Calls the {@link FiniteStormSpout#nextTuple()} method until {@link + * FiniteStormSpout#reachedEnd()} is true or {@link FiniteStormSpout#cancel()} is called. + */ + @Override + protected void execute() { + while (super.isRunning && !finiteSpout.reachedEnd()) { + finiteSpout.nextTuple(); + } + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java new file mode 100644 index 0000000000000..776e65df27cdc --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java @@ -0,0 +1,69 @@ +/* + * 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.stormcompatibility.wrappers; + +import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(StormWrapperSetupHelper.class) +public class FiniteStormSpoutWrapperTest { + + @SuppressWarnings("unchecked") + @Test + public void runAndExecuteTest1() throws Exception { + + FiniteStormSpout stormSpout = + mock(FiniteStormSpout.class); + when(stormSpout.reachedEnd()).thenReturn(false, false, false, true, false, false, true); + + FiniteStormSpoutWrapper wrapper = + new FiniteStormSpoutWrapper(stormSpout); + wrapper.setRuntimeContext(mock(StreamingRuntimeContext.class)); + + wrapper.run(mock(SourceContext.class)); + verify(stormSpout, times(3)).nextTuple(); + } + + @SuppressWarnings("unchecked") + @Test + public void runAndExecuteTest2() throws Exception { + + FiniteStormSpout stormSpout = + mock(FiniteStormSpout.class); + when(stormSpout.reachedEnd()).thenReturn(true, false, true, false, true, false, true); + + FiniteStormSpoutWrapper wrapper = + new FiniteStormSpoutWrapper(stormSpout); + wrapper.setRuntimeContext(mock(StreamingRuntimeContext.class)); + + wrapper.run(mock(SourceContext.class)); + verify(stormSpout, never()).nextTuple(); + } + +} From dba2946f465a72f18b6452e7ab34f9198b71a908 Mon Sep 17 00:00:00 2001 From: szape Date: Fri, 19 Jun 2015 10:43:10 +0200 Subject: [PATCH 097/175] [FLINK-2243] [storm-compat] Demonstrating finite Storm spout functionality on exclamation example -minor renaming -improving JavaDocs Closes #853 --- docs/apis/storm_compatibility.md | 51 ++++++++++ .../excamation/ExclamationTopology.java | 47 ++++++--- ...ion.java => ExclamationWithStormBolt.java} | 26 ++++- ...on.java => ExclamationWithStormSpout.java} | 42 ++++++-- .../excamation/StormExclamationLocal.java | 25 ++++- .../StormExclamationRemoteByClient.java | 22 +++++ .../StormExclamationRemoteBySubmitter.java | 21 ++++ .../util/FiniteStormFileSpout.java | 97 +++++++++++++++++++ ...ter.java => FiniteStormInMemorySpout.java} | 28 ++++-- .../util/OutputFormatter.java | 11 ++- .../util/SimpleOutputFormatter.java | 15 ++- .../wordcount/BoltTokenizerWordCount.java | 2 +- .../wordcount/SpoutSourceWordCount.java | 4 +- .../wordcount/StormWordCountLocal.java | 2 +- .../StormWordCountRemoteByClient.java | 2 +- .../StormWordCountRemoteBySubmitter.java | 2 +- .../wordcount/WordCountTopology.java | 2 +- ...va => ExclamationWithStormBoltITCase.java} | 6 +- ...a => ExclamationWithStormSpoutITCase.java} | 6 +- 19 files changed, 358 insertions(+), 53 deletions(-) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/{StormBoltExclamation.java => ExclamationWithStormBolt.java} (78%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/{StormSpoutExclamation.java => ExclamationWithStormSpout.java} (69%) create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/{RawOutputFormatter.java => FiniteStormInMemorySpout.java} (55%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/{StormSpoutExclamationITCase.java => ExclamationWithStormBoltITCase.java} (86%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/{StormBoltExclamationITCase.java => ExclamationWithStormSpoutITCase.java} (86%) diff --git a/docs/apis/storm_compatibility.md b/docs/apis/storm_compatibility.md index b8fe66ebc2a4b..1390b923586b1 100644 --- a/docs/apis/storm_compatibility.md +++ b/docs/apis/storm_compatibility.md @@ -167,6 +167,57 @@ The input type is `Tuple1` and `Fields("sentence")` specify that `input. See [BoltTokenizerWordCountPojo](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java) and [BoltTokenizerWordCountWithNames](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java) for examples. +# Flink Extensions + +## Finite Storm Spouts + +In Flink streaming, sources can be finite - i.e. emit a finite number of records and stop after emitting the last record -, however, Storm spouts always emit infinite streams. +The bridge between the two approach is the `FiniteStormSpout` interface which, in addition to `IRichSpout`, contains a `reachedEnd()` method, where the user can specify a stopping-condition. +The user can create a finite Storm spout by implementing this interface instead of `IRichSpout`, and implementing the `reachedEnd()`method in addition. +When used as part of a Flink topology, a `FiniteStormSpout` should be wrapped in a `FiniteStormSpoutWrapper` class. + +Although finite Storm spouts are not necessary to embed Storm spouts into a Flink streaming program or to submit a whole Storm topology to Flink, there are cases where they may come in handy: + + * to achieve that a native Storm spout behaves the same way as a finite Flink source with minimal modifications + * the user wants to process a stream only for some time; after that, the spout can stop automatically + * reading a file into a stream + * for testing purposes + +A `FiniteStormSpout` can be still used as a normal, infinite Storm spout by changing its wrapper class to `StormSpoutWraper` in the Flink topology. + +An example of a finite Storm spout that emits records for 10 seconds only: +
    +
    +~~~java +public class TimedFiniteStormSpout extends AbstractStormSpout implements FiniteStormSpout { + [...] + private long starttime = System.currentTimeMillis(); + + public boolean reachedEnd() { + return System.currentTimeMillis() - starttime > 10000l; + } + [...] +} +~~~ +
    +
    + +Using a `FiniteStormSpout` in a Flink topology: +
    +
    +~~~java +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +DataStream rawInput = env.addSource( + new FiniteStormSpoutWrapper(new TimedFiniteStormSpout(), true) + TypeExtractor.getForClass(String.class)); + +// process data stream +[...] +~~~ +
    +
    + # Storm Compatibility Examples You can find more examples in Maven module `flink-storm-compatibilty-examples`. diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java index a5bb5711d31e7..b7c98a8e1c7dd 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java @@ -20,15 +20,32 @@ import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; import org.apache.flink.stormcompatibility.excamation.stormoperators.ExclamationBolt; +import org.apache.flink.stormcompatibility.util.FiniteStormFileSpout; +import org.apache.flink.stormcompatibility.util.FiniteStormInMemorySpout; import org.apache.flink.stormcompatibility.util.OutputFormatter; -import org.apache.flink.stormcompatibility.util.RawOutputFormatter; +import org.apache.flink.stormcompatibility.util.SimpleOutputFormatter; import org.apache.flink.stormcompatibility.util.StormBoltFileSink; import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; -import org.apache.flink.stormcompatibility.util.StormFileSpout; -import org.apache.flink.stormcompatibility.util.StormInMemorySpout; /** - * This is a basic example of a Storm topology. + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text + * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. + *

    + *

    + * The input is a plain text file with lines separated by newline characters. + *

    + *

    + * Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] <text path> + * <result path>
    + * If no parameters are provided, the program is run with default data from + * {@link WordCountData}. + *

    + *

    + * This example shows how to: + *

      + *
    • construct a regular Storm topology as Flink program
    • + *
    • make use of the FiniteStormSpout interface
    • + *
    */ public class ExclamationTopology { @@ -36,7 +53,7 @@ public class ExclamationTopology { public final static String firstBoltId = "exclamation1"; public final static String secondBoltId = "exclamation2"; public final static String sinkId = "sink"; - private final static OutputFormatter formatter = new RawOutputFormatter(); + private final static OutputFormatter formatter = new SimpleOutputFormatter(); public static FlinkTopologyBuilder buildTopology() { final FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); @@ -46,9 +63,9 @@ public static FlinkTopologyBuilder buildTopology() { // read the text file from given input path final String[] tokens = textPath.split(":"); final String inputFile = tokens[tokens.length - 1]; - builder.setSpout(spoutId, new StormFileSpout(inputFile)); + builder.setSpout(spoutId, new FiniteStormFileSpout(inputFile)); } else { - builder.setSpout(spoutId, new StormInMemorySpout(WordCountData.WORDS)); + builder.setSpout(spoutId, new FiniteStormInMemorySpout(WordCountData.WORDS)); } builder.setBolt(firstBoltId, new ExclamationBolt(), 3).shuffleGrouping(spoutId); @@ -59,9 +76,11 @@ public static FlinkTopologyBuilder buildTopology() { // read the text file from given input path final String[] tokens = outputPath.split(":"); final String outputFile = tokens[tokens.length - 1]; - builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)).shuffleGrouping(secondBoltId); + builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) + .shuffleGrouping(secondBoltId); } else { - builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4).shuffleGrouping(secondBoltId); + builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) + .shuffleGrouping(secondBoltId); } return builder; @@ -84,13 +103,17 @@ static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: StormExclamation* "); + System.err.println( + "Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] "); return false; } } else { - System.out.println("Executing StormExclamation* example with built-in default data"); + System.out.println("Executing StormExclamation example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: StormExclamation* "); + System.out.println( + " Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] " + + " "); } return true; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormBoltExclamation.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java similarity index 78% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormBoltExclamation.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java index 52e740c27ef3e..7bcb7f9816b98 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormBoltExclamation.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java @@ -26,7 +26,25 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -public class StormBoltExclamation { +/** + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text + * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. + *

    + *

    + * The input is a plain text file with lines separated by newline characters. + *

    + *

    + * Usage: StormExclamationWithStormBolt <text path> <result path>
    + * If no parameters are provided, the program is run with default data from + * {@link WordCountData}. + *

    + *

    + * This example shows how to: + *

      + *
    • use a Storm bolt within a Flink Streaming program
    • + *
    + */ +public class ExclamationWithStormBolt { // ************************************************************************* // PROGRAM @@ -90,13 +108,13 @@ private static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: StormBoltExclamation "); + System.err.println("Usage: ExclamationWithStormBolt "); return false; } } else { - System.out.println("Executing StormBoltExclamation example with built-in default data"); + System.out.println("Executing ExclamationWithStormBolt example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: StormBoltExclamation "); + System.out.println(" Usage: ExclamationWithStormBolt "); } return true; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormSpoutExclamation.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java similarity index 69% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormSpoutExclamation.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java index 2569fa74bc046..f027eae165ce2 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormSpoutExclamation.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java @@ -21,13 +21,32 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.util.StormFileSpout; -import org.apache.flink.stormcompatibility.util.StormInMemorySpout; -import org.apache.flink.stormcompatibility.wrappers.StormFiniteSpoutWrapper; +import org.apache.flink.stormcompatibility.util.FiniteStormFileSpout; +import org.apache.flink.stormcompatibility.util.FiniteStormInMemorySpout; +import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -public class StormSpoutExclamation { +/** + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text + * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. + *

    + *

    + * The input is a plain text file with lines separated by newline characters. + *

    + *

    + * Usage: StormExclamationWithStormSpout <text path> <result path>
    + * If no parameters are provided, the program is run with default data from + * {@link WordCountData}. + *

    + *

    + * This example shows how to: + *

      + *
    • use a Storm spout within a Flink Streaming program
    • + *
    • make use of the FiniteStormSpout interface
    • + *
    + */ +public class ExclamationWithStormSpout { // ************************************************************************* // PROGRAM @@ -89,13 +108,14 @@ private static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: StormSpoutExclamation "); + System.err.println("Usage: ExclamationWithStormSpout "); return false; } } else { - System.out.println("Executing StormSpoutExclamation example with built-in default data"); + System.out.println("Executing ExclamationWithStormSpout example with built-in default " + + "data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: StormSpoutExclamation "); + System.out.println(" Usage: ExclamationWithStormSpout "); } return true; } @@ -106,12 +126,14 @@ private static DataStream getTextDataStream(final StreamExecutionEnviron final String[] tokens = textPath.split(":"); final String localFile = tokens[tokens.length - 1]; return env.addSource( - new StormFiniteSpoutWrapper(new StormFileSpout(localFile), true), + new FiniteStormSpoutWrapper(new FiniteStormFileSpout(localFile), true), TypeExtractor.getForClass(String.class)).setParallelism(1); } - return env.addSource(new StormFiniteSpoutWrapper(new StormInMemorySpout(WordCountData.WORDS), true), - TypeExtractor.getForClass(String.class)); + return env.addSource( + new FiniteStormSpoutWrapper( + new FiniteStormInMemorySpout(WordCountData.WORDS), true), + TypeExtractor.getForClass(String.class)).setParallelism(1); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java index a25e5e0a1f718..5941ff0c839fe 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java @@ -21,6 +21,27 @@ import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +/** + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text + * files in a streaming fashion. The program is constructed as a regular {@link StormTopology} and + * submitted to Flink for execution in the same way as to a Storm {@link LocalCluster}. + *

    + * This example shows how to run program directly within Java, thus it cannot be used to submit a + * {@link StormTopology} via Flink command line clients (ie, bin/flink). + *

    + *

    + * The input is a plain text file with lines separated by newline characters. + *

    + *

    + * Usage: StormExclamationLocal <text path> <result path>
    + * If no parameters are provided, the program is run with default data from {@link WordCountData}. + *

    + *

    + * This example shows how to: + *

      + *
    • run a regular Storm program locally on Flink
    • + *
    + */ public class StormExclamationLocal { public final static String topologyId = "Streaming Exclamation"; @@ -43,10 +64,6 @@ public static void main(final String[] args) throws Exception { cluster.submitTopology(topologyId, null, builder.createTopology()); Utils.sleep(10 * 1000); - - // TODO kill does no do anything so far - cluster.killTopology(topologyId); - cluster.shutdown(); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java index 3f55316f7e72e..0f64301087df6 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java @@ -25,6 +25,28 @@ import org.apache.flink.stormcompatibility.api.FlinkClient; import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +/** + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text + * files in a streaming fashion. The program is constructed as a regular {@link StormTopology} and + * submitted to Flink for execution in the same way as to a Storm cluster similar to + * {@link NimbusClient}. The Flink cluster can be local or remote. + *

    + * This example shows how to submit the program via Java, thus it cannot be used to submit a + * {@link StormTopology} via Flink command line clients (ie, bin/flink). + *

    + *

    + * The input is a plain text file with lines separated by newline characters. + *

    + *

    + * Usage: StormExclamationRemoteByClient <text path> <result path>
    + * If no parameters are provided, the program is run with default data from {@link WordCountData}. + *

    + *

    + * This example shows how to: + *

      + *
    • submit a regular Storm program to a local or remote Flink cluster.
    • + *
    + */ public class StormExclamationRemoteByClient { public final static String topologyId = "Streaming Exclamation"; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java index 728c5c79ac60f..d58052003597e 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java @@ -22,6 +22,27 @@ import org.apache.flink.stormcompatibility.api.FlinkSubmitter; import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +/** + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text + * files in a streaming fashion. The program is constructed as a regular {@link StormTopology} and + * submitted to Flink for execution in the same way as to a Storm cluster similar to + * {@link StormSubmitter}. The Flink cluster can be local or remote. + *

    + * This example shows how to submit the program via Java as well as Flink's command line client (ie, bin/flink). + *

    + *

    + * The input is a plain text file with lines separated by newline characters. + *

    + *

    + * Usage: StormExclamationRemoteByClient <text path> <result path>
    + * If no parameters are provided, the program is run with default data from {@link WordCountData}. + *

    + *

    + * This example shows how to: + *

      + *
    • submit a regular Storm program to a local or remote Flink cluster.
    • + *
    + */ public class StormExclamationRemoteBySubmitter { public final static String topologyId = "Streaming Exclamation"; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java new file mode 100644 index 0000000000000..d45ad7603ba85 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java @@ -0,0 +1,97 @@ +/* + * 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.stormcompatibility.util; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Values; +import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; + +import java.io.BufferedReader; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.IOException; +import java.util.Map; + +/** + * Implements a Storm Spout that reads data from a given local file. The spout stops automatically + * when it reached the end of the file. + */ +public class FiniteStormFileSpout extends AbstractStormSpout implements FiniteStormSpout { + private static final long serialVersionUID = -6996907090003590436L; + + private final String path; + private BufferedReader reader; + private String line; + private boolean newLineRead; + + public FiniteStormFileSpout(final String path) { + this.path = path; + } + + @SuppressWarnings("rawtypes") + @Override + public void open(final Map conf, final TopologyContext context, + final SpoutOutputCollector collector) { + super.open(conf, context, collector); + try { + this.reader = new BufferedReader(new FileReader(this.path)); + } catch (final FileNotFoundException e) { + throw new RuntimeException(e); + } + newLineRead = false; + } + + @Override + public void close() { + if (this.reader != null) { + try { + this.reader.close(); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } + } + + @Override + public void nextTuple() { + this.collector.emit(new Values(line)); + newLineRead = false; + } + + /** + * Can be called before nextTuple() any times including 0. + */ + public boolean reachedEnd() { + try { + readLine(); + } catch (IOException e) { + throw new RuntimeException("Exception occured while reading file " + path); + } + return line == null; + } + + private void readLine() throws IOException { + if (!newLineRead) { + line = reader.readLine(); + newLineRead = true; + } + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/RawOutputFormatter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java similarity index 55% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/RawOutputFormatter.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java index 7faf6cd3e8941..899c569db2a9c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/RawOutputFormatter.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java @@ -18,15 +18,31 @@ package org.apache.flink.stormcompatibility.util; -import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; -public class RawOutputFormatter implements OutputFormatter { - private static final long serialVersionUID = 8685668993521259832L; +/** + * Implements a Storm Spout that reads String[] data stored in the memory. The spout stops + * automatically when it emitted all of the data. + */ +public class FiniteStormInMemorySpout extends AbstractStormSpout implements FiniteStormSpout { + + private static final long serialVersionUID = -4008858647468647019L; + + private String[] source; + private int counter = 0; + + public FiniteStormInMemorySpout(String[] source) { + this.source = source; + } @Override - public String format(final Tuple input) { - assert (input.size() == 1); - return input.getValue(0).toString(); + public void nextTuple() { + this.collector.emit(new Values(source[this.counter++])); + } + + public boolean reachedEnd() { + return counter >= source.length; } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java index bfc3135db65f3..ec9adfe841ec7 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java @@ -18,12 +18,19 @@ package org.apache.flink.stormcompatibility.util; -import java.io.Serializable; - import backtype.storm.tuple.Tuple; +import java.io.Serializable; + public interface OutputFormatter extends Serializable { + /** + * Converts a Storm {@link Tuple} to a string. This method is used for formatting the output + * tuples before writing them out to a file or to the consol. + * + * @param input The tuple to be formatted + * @return The string result of the formatting + */ public String format(Tuple input); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java index ccb617baa9fd4..0702e947ea083 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java @@ -23,9 +23,20 @@ public class SimpleOutputFormatter implements OutputFormatter { private static final long serialVersionUID = 6349573860144270338L; + /** + * Converts a Storm {@link Tuple} with 1 field to a string by retrieving the value of that + * field. This method is used for formatting raw outputs wrapped in tuples, before writing them + * out to a file or to the consol. + * + * @param input + * The tuple to be formatted + * @return The string result of the formatting + */ @Override public String format(final Tuple input) { - return input.getValues().toString(); + if (input.getValues().size() != 1) { + throw new RuntimeException("The output is not raw"); + } + return input.getValue(0).toString(); } - } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java index 8f4503f8de2c0..eab58f58c8e9c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java @@ -40,7 +40,7 @@ *

    * This example shows how to: *

      - *
    • use a Storm bolt within a Flink Streaming program. + *
    • use a Storm bolt within a Flink Streaming program.
    • *
    */ public class BoltTokenizerWordCount { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java index 361d83af25663..4c012d80e683a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java @@ -43,7 +43,7 @@ *

    * This example shows how to: *

      - *
    • use a Storm bolt within a Flink Streaming program. + *
    • use a Storm spout within a Flink Streaming program.
    • *
    */ public class SpoutSourceWordCount { @@ -145,7 +145,7 @@ private static DataStream getTextDataStream(final StreamExecutionEnviron } return env.addSource(new StormFiniteSpoutWrapper(new StormInMemorySpout(WordCountData.WORDS), true), - TypeExtractor.getForClass(String.class)); + TypeExtractor.getForClass(String.class)).setParallelism(1); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java index 3fbd5b730430c..836c8e945c13c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java @@ -42,7 +42,7 @@ *

    * This example shows how to: *

      - *
    • run a regular Storm program locally on Flink + *
    • run a regular Storm program locally on Flink
    • *
    */ public class StormWordCountLocal { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java index 9e56c14357861..0bbe11b28e004 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java @@ -46,7 +46,7 @@ *

    * This example shows how to: *

      - *
    • submit a regular Storm program to a local or remote Flink cluster. + *
    • submit a regular Storm program to a local or remote Flink cluster.
    • *
    */ public class StormWordCountRemoteByClient { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java index a1fb79d00e656..264dc415b0ae6 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java @@ -42,7 +42,7 @@ *

    * This example shows how to: *

      - *
    • submit a regular Storm program to a local or remote Flink cluster. + *
    • submit a regular Storm program to a local or remote Flink cluster.
    • *
    */ public class StormWordCountRemoteBySubmitter { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java index f028266585c3c..367ca9e660ec0 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java @@ -47,7 +47,7 @@ *

    * This example shows how to: *

      - *
    • how to construct a regular Storm topology as Flink program + *
    • how to construct a regular Storm topology as Flink program
    • *
    */ public class WordCountTopology { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormSpoutExclamationITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java similarity index 86% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormSpoutExclamationITCase.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java index 2b08b4b8bfb56..930f87bced77c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormSpoutExclamationITCase.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java @@ -18,12 +18,12 @@ package org.apache.flink.stormcompatibility.exclamation; -import org.apache.flink.stormcompatibility.excamation.StormSpoutExclamation; +import org.apache.flink.stormcompatibility.excamation.ExclamationWithStormBolt; import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData; import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.apache.flink.test.testdata.WordCountData; -public class StormSpoutExclamationITCase extends StreamingProgramTestBase { +public class ExclamationWithStormBoltITCase extends StreamingProgramTestBase { protected String textPath; protected String resultPath; @@ -41,7 +41,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - StormSpoutExclamation.main(new String[]{this.textPath, this.resultPath}); + ExclamationWithStormBolt.main(new String[]{this.textPath, this.resultPath}); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormBoltExclamationITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java similarity index 86% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormBoltExclamationITCase.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java index 75dd5fc6a7877..4c515ce147dd0 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormBoltExclamationITCase.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java @@ -18,12 +18,12 @@ package org.apache.flink.stormcompatibility.exclamation; -import org.apache.flink.stormcompatibility.excamation.StormBoltExclamation; +import org.apache.flink.stormcompatibility.excamation.ExclamationWithStormSpout; import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData; import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.apache.flink.test.testdata.WordCountData; -public class StormBoltExclamationITCase extends StreamingProgramTestBase { +public class ExclamationWithStormSpoutITCase extends StreamingProgramTestBase { protected String textPath; protected String resultPath; @@ -41,7 +41,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - StormBoltExclamation.main(new String[]{this.textPath, this.resultPath}); + ExclamationWithStormSpout.main(new String[]{this.textPath, this.resultPath}); } } From fdebcb83d0b19aad4f37da14116c08972af06ff1 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 21 Jul 2015 16:54:26 +0200 Subject: [PATCH 098/175] [FLINK-2387] add streaming test case for live accumulators This closes #926. --- .../source/FromElementsFunction.java | 2 +- .../accumulators/AccumulatorLiveITCase.java | 175 +++++++++++++----- 2 files changed, 133 insertions(+), 44 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java index 28544ee216a3a..af47f5991fe2b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java @@ -36,7 +36,7 @@ * *

    Upon construction, this source function serializes the elements using Flink's type information. * That way, any object transport using Java serialization will not be affected by the serializability - * if the elements.

    + * of the elements.

    * * @param The type of elements returned by this function. */ diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 51d3eb995bdd1..020919e7a496f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -24,6 +24,7 @@ import akka.pattern.Patterns; import akka.testkit.JavaTestKit; import akka.util.Timeout; +import org.apache.flink.api.common.ExecutionMode; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; @@ -52,6 +53,8 @@ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; import org.junit.After; import org.junit.Before; @@ -60,7 +63,6 @@ import org.slf4j.LoggerFactory; import scala.concurrent.Await; import scala.concurrent.Future; -import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; import java.io.IOException; @@ -75,6 +77,17 @@ /** * Tests the availability of accumulator results during runtime. The test case tests a user-defined * accumulator and Flink's internal accumulators for two consecutive tasks. + * + * CHAINED[Source -> Map] -> Sink + * + * Checks are performed as the elements arrive at the operators. Checks consist of a message sent by + * the task to the task manager which notifies the job manager and sends the current accumulators. + * The task blocks until the test has been notified about the current accumulator values. + * + * A barrier between the operators ensures that that pipelining is disabled for the streaming test. + * The batch job reads the records one at a time. The streaming code buffers the records beforehand; + * that's why exact guarantees about the number of records read are very hard to make. Thus, why we + * check for an upper bound of the elements read. */ public class AccumulatorLiveITCase { @@ -83,15 +96,17 @@ public class AccumulatorLiveITCase { private static ActorSystem system; private static ActorGateway jobManagerGateway; private static ActorRef taskManager; + private static JobID jobID; + private static JobGraph jobGraph; // name of user accumulator - private static String NAME = "test"; + private static String ACCUMULATOR_NAME = "test"; // number of heartbeat intervals to check private static final int NUM_ITERATIONS = 5; - private static List inputData = new ArrayList(NUM_ITERATIONS); + private static List inputData = new ArrayList<>(NUM_ITERATIONS); private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS); @@ -113,29 +128,60 @@ public void before() throws Exception { for (int i=0; i < NUM_ITERATIONS; i++) { inputData.add(i, String.valueOf(i+1)); } + + NotifyingMapper.finished = false; } @After public void after() throws Exception { JavaTestKit.shutdownActorSystem(system); + inputData.clear(); } @Test - public void testProgram() throws Exception { + public void testBatch() throws Exception { - new JavaTestKit(system) {{ + /** The program **/ + ExecutionEnvironment env = new BatchPlanExtractor(); + env.setParallelism(1); + + DataSet input = env.fromCollection(inputData); + input + .flatMap(new NotifyingMapper()) + .output(new NotifyingOutputFormat()); + + env.execute(); + + // Extract job graph and set job id for the task to notify of accumulator changes. + jobGraph = getOptimizedPlan(((BatchPlanExtractor) env).plan); + jobID = jobGraph.getJobID(); + + verifyResults(); + } + + + @Test + public void testStreaming() throws Exception { + + StreamExecutionEnvironment env = new StreamJobExtractor(); + env.setParallelism(1); + + DataStream input = env.fromCollection(inputData); + input + .flatMap(new NotifyingMapper()) + .write(new NotifyingOutputFormat(), 1000).disableChaining(); - /** The program **/ - ExecutionEnvironment env = new PlanExtractor(); - DataSet input = env.fromCollection(inputData); - input - .flatMap(new WaitingUDF()) - .output(new WaitingOutputFormat()); - env.execute(); + env.execute(); - // Extract job graph and set job id for the task to notify of accumulator changes. - JobGraph jobGraph = getOptimizedPlan(((PlanExtractor) env).plan); - jobID = jobGraph.getJobID(); + jobGraph = ((StreamJobExtractor) env).graph; + jobID = jobGraph.getJobID(); + + verifyResults(); + } + + + private static void verifyResults() { + new JavaTestKit(system) {{ ActorGateway selfGateway = new AkkaActorGateway(getRef(), jobManagerGateway.leaderSessionID()); @@ -149,12 +195,12 @@ public void testProgram() throws Exception { expectMsgClass(TIMEOUT, Status.Success.class); - ExecutionAttemptID mapperTaskID = null; - TestingJobManagerMessages.UpdatedAccumulators msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); Map>> flinkAccumulators = msg.flinkAccumulators(); Map> userAccumulators = msg.userAccumulators(); + ExecutionAttemptID mapperTaskID = null; + // find out the first task's execution attempt id for (Map.Entry entry : flinkAccumulators.entrySet()) { if (entry.getValue() != null) { @@ -163,8 +209,19 @@ public void testProgram() throws Exception { } } + ExecutionAttemptID sinkTaskID = null; + + // find the second's task id + for (ExecutionAttemptID key : flinkAccumulators.keySet()) { + if (key != mapperTaskID) { + sinkTaskID = key; + break; + } + } + /* Check for accumulator values */ - if(checkUserAccumulators(0, userAccumulators) && checkFlinkAccumulators(mapperTaskID, 0, 0, 0, 0, flinkAccumulators)) { + if(checkUserAccumulators(0, userAccumulators) && + checkFlinkAccumulators(mapperTaskID, 0, 0, 0, 0, flinkAccumulators)) { LOG.info("Passed initial check for map task."); } else { fail("Wrong accumulator results when map task begins execution."); @@ -172,7 +229,6 @@ public void testProgram() throws Exception { int expectedAccVal = 0; - ExecutionAttemptID sinkTaskID = null; /* for mapper task */ for (int i = 1; i <= NUM_ITERATIONS; i++) { @@ -186,8 +242,16 @@ public void testProgram() throws Exception { LOG.info("{}", flinkAccumulators); LOG.info("{}", userAccumulators); - if (checkUserAccumulators(expectedAccVal, userAccumulators) && checkFlinkAccumulators(mapperTaskID, 0, i, 0, i * 4, flinkAccumulators)) { - LOG.info("Passed round " + i); + if (checkUserAccumulators(expectedAccVal, userAccumulators) && + checkFlinkAccumulators(mapperTaskID, 0, i, 0, i * 4, flinkAccumulators)) { + LOG.info("Passed round #" + i); + } else if (checkUserAccumulators(expectedAccVal, userAccumulators) && + checkFlinkAccumulators(sinkTaskID, 0, i, 0, i * 4, flinkAccumulators)) { + // we determined the wrong task id and need to switch the two here + ExecutionAttemptID temp = mapperTaskID; + mapperTaskID = sinkTaskID; + sinkTaskID = temp; + LOG.info("Passed round #" + i); } else { fail("Failed in round #" + i); } @@ -197,15 +261,8 @@ public void testProgram() throws Exception { flinkAccumulators = msg.flinkAccumulators(); userAccumulators = msg.userAccumulators(); - // find the second's task id - for (ExecutionAttemptID key : flinkAccumulators.keySet()) { - if (key != mapperTaskID) { - sinkTaskID = key; - break; - } - } - - if(checkUserAccumulators(expectedAccVal, userAccumulators) && checkFlinkAccumulators(sinkTaskID, 0, 0, 0, 0, flinkAccumulators)) { + if(checkUserAccumulators(expectedAccVal, userAccumulators) && + checkFlinkAccumulators(sinkTaskID, 0, 0, 0, 0, flinkAccumulators)) { LOG.info("Passed initial check for sink task."); } else { fail("Wrong accumulator results when sink task begins execution."); @@ -223,8 +280,9 @@ public void testProgram() throws Exception { LOG.info("{}", flinkAccumulators); LOG.info("{}", userAccumulators); - if (checkUserAccumulators(expectedAccVal, userAccumulators) && checkFlinkAccumulators(sinkTaskID, i, 0, i*4, 0, flinkAccumulators)) { - LOG.info("Passed round " + i); + if (checkUserAccumulators(expectedAccVal, userAccumulators) && + checkFlinkAccumulators(sinkTaskID, i, 0, i * 4, 0, flinkAccumulators)) { + LOG.info("Passed round #" + i); } else { fail("Failed in round #" + i); } @@ -235,9 +293,10 @@ public void testProgram() throws Exception { }}; } + private static boolean checkUserAccumulators(int expected, Map> accumulatorMap) { LOG.info("checking user accumulators"); - return accumulatorMap.containsKey(NAME) && expected == ((IntCounter)accumulatorMap.get(NAME)).getLocalValue(); + return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter)accumulatorMap.get(ACCUMULATOR_NAME)).getLocalValue(); } private static boolean checkFlinkAccumulators(ExecutionAttemptID taskKey, int expectedRecordsIn, int expectedRecordsOut, int expectedBytesIn, int expectedBytesOut, @@ -253,12 +312,12 @@ private static boolean checkFlinkAccumulators(ExecutionAttemptID taskKey, int ex * The following two cases are for the DataSource and Map task */ case NUM_RECORDS_OUT: - if(((LongCounter) entry.getValue()).getLocalValue() != expectedRecordsOut) { + if(((LongCounter) entry.getValue()).getLocalValue() < expectedRecordsOut) { return false; } break; case NUM_BYTES_OUT: - if (((LongCounter) entry.getValue()).getLocalValue() != expectedBytesOut) { + if (((LongCounter) entry.getValue()).getLocalValue() < expectedBytesOut) { return false; } break; @@ -266,12 +325,12 @@ private static boolean checkFlinkAccumulators(ExecutionAttemptID taskKey, int ex * The following two cases are for the DataSink task */ case NUM_RECORDS_IN: - if (((LongCounter) entry.getValue()).getLocalValue() != expectedRecordsIn) { + if (((LongCounter) entry.getValue()).getLocalValue() < expectedRecordsIn) { return false; } break; case NUM_BYTES_IN: - if (((LongCounter) entry.getValue()).getLocalValue() != expectedBytesIn) { + if (((LongCounter) entry.getValue()).getLocalValue() < expectedBytesIn) { return false; } break; @@ -284,15 +343,17 @@ private static boolean checkFlinkAccumulators(ExecutionAttemptID taskKey, int ex /** - * UDF that waits for at least the heartbeat interval's duration. + * UDF that notifies when it changes the accumulator values */ - private static class WaitingUDF extends RichFlatMapFunction { + private static class NotifyingMapper extends RichFlatMapFunction { private IntCounter counter = new IntCounter(); + private static boolean finished = false; + @Override public void open(Configuration parameters) throws Exception { - getRuntimeContext().addAccumulator(NAME, counter); + getRuntimeContext().addAccumulator(ACCUMULATOR_NAME, counter); notifyTaskManagerOfAccumulatorUpdate(); } @@ -305,9 +366,16 @@ public void flatMap(String value, Collector out) throws Exception { notifyTaskManagerOfAccumulatorUpdate(); } + @Override + public void close() throws Exception { + finished = true; + } } - private static class WaitingOutputFormat implements OutputFormat { + /** + * Outputs format which notifies of accumulator changes and waits for the previous mapper. + */ + private static class NotifyingOutputFormat implements OutputFormat { @Override public void configure(Configuration parameters) { @@ -315,6 +383,11 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { + while (!NotifyingMapper.finished) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) {} + } notifyTaskManagerOfAccumulatorUpdate(); } @@ -334,7 +407,7 @@ public void close() throws IOException { */ public static void notifyTaskManagerOfAccumulatorUpdate() { new JavaTestKit(system) {{ - Timeout timeout = new Timeout(Duration.create(5, "seconds")); + Timeout timeout = new Timeout(TIMEOUT); Future ask = Patterns.ask(taskManager, new TestingTaskManagerMessages.AccumulatorsChanged(jobID), timeout); try { Await.result(ask, timeout.duration()); @@ -354,7 +427,7 @@ private static JobGraph getOptimizedPlan(Plan plan) { return jgg.compileJobGraph(op); } - private static class PlanExtractor extends LocalEnvironment { + private static class BatchPlanExtractor extends LocalEnvironment { private Plan plan = null; @@ -363,6 +436,22 @@ public JobExecutionResult execute(String jobName) throws Exception { plan = createProgramPlan(); return new JobExecutionResult(new JobID(), -1, null); } + } + + + private static class StreamJobExtractor extends StreamExecutionEnvironment { + + private JobGraph graph = null; + @Override + public JobExecutionResult execute() throws Exception { + return execute("default"); + } + + @Override + public JobExecutionResult execute(String jobName) throws Exception { + graph = this.streamGraph.getJobGraph(); + return new JobExecutionResult(new JobID(), -1, null); + } } } From 368d7aee1ce3a4a2fd458628852a96db098ae635 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 19:42:13 +0200 Subject: [PATCH 099/175] [FLINK-705] [license] Add license reference for TPC-H sample data in tests. --- .../test/recordJobTests/TPCHQuery3ITCase.java | 14 ++++++++++++++ .../recordJobTests/TPCHQuery3WithUnionITCase.java | 14 +++++++++++++- .../test/recordJobTests/TPCHQuery9ITCase.java | 14 +++++++++++++- .../recordJobTests/TPCHQueryAsterixITCase.java | 14 +++++++++++++- 4 files changed, 53 insertions(+), 3 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java index a0236c2767410..159c93180aa1b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java @@ -24,10 +24,24 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.test.recordJobs.relational.TPCHQuery3; import org.apache.flink.test.util.RecordAPITestBase; + import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +// ----------------------------------------------------------------------------- +// --- NOTE --- +// +// This class contains test data generated by tools from by the +// Transaction Processing Council (TPC), specifically the TPC-H benchmark's +// data generator. +// +// Any form of use and redistribution must happen in accordance with the TPC-H +// Software License Agreement. +// +// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf +// ----------------------------------------------------------------------------- + @RunWith(Parameterized.class) public class TPCHQuery3ITCase extends RecordAPITestBase { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java index 3ade96438ff29..4c51ac7c0b03a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java @@ -16,13 +16,25 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobTests; import org.apache.flink.api.common.Plan; import org.apache.flink.test.recordJobs.relational.TPCHQuery3Unioned; import org.apache.flink.test.util.RecordAPITestBase; +// ----------------------------------------------------------------------------- +// --- NOTE --- +// +// This class contains test data generated by tools from by the +// Transaction Processing Council (TPC), specifically the TPC-H benchmark's +// data generator. +// +// Any form of use and redistribution must happen in accordance with the TPC-H +// Software License Agreement. +// +// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf +// ----------------------------------------------------------------------------- + public class TPCHQuery3WithUnionITCase extends RecordAPITestBase { private String orders1Path = null; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java index f09240088ba75..ab99e22eb2f74 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java @@ -16,13 +16,25 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobTests; import org.apache.flink.api.common.Plan; import org.apache.flink.test.recordJobs.relational.TPCHQuery9; import org.apache.flink.test.util.RecordAPITestBase; +// ----------------------------------------------------------------------------- +// --- NOTE --- +// +// This class contains test data generated by tools from by the +// Transaction Processing Council (TPC), specifically the TPC-H benchmark's +// data generator. +// +// Any form of use and redistribution must happen in accordance with the TPC-H +// Software License Agreement. +// +// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf +// ----------------------------------------------------------------------------- + public class TPCHQuery9ITCase extends RecordAPITestBase { private String partInputPath; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java index 2c53ee28a8cfd..881bd2c9f5a22 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java @@ -16,13 +16,25 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobTests; import org.apache.flink.api.common.Plan; import org.apache.flink.test.recordJobs.relational.TPCHQueryAsterix; import org.apache.flink.test.util.RecordAPITestBase; +// ----------------------------------------------------------------------------- +// --- NOTE --- +// +// This class contains test data generated by tools from by the +// Transaction Processing Council (TPC), specifically the TPC-H benchmark's +// data generator. +// +// Any form of use and redistribution must happen in accordance with the TPC-H +// Software License Agreement. +// +// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf +// ----------------------------------------------------------------------------- + public class TPCHQueryAsterixITCase extends RecordAPITestBase { private String ordersPath; From e503ebcfc2330544c7bebcdaead6b1c5292409a9 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 20:29:24 +0200 Subject: [PATCH 100/175] [FLINK-2453] [docs] Update README and setup docs to reflect requirement for Java 7+ --- README.md | 2 +- docs/setup/cluster_setup.md | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index d9250b8c1b529..3cf08c77803aa 100644 --- a/README.md +++ b/README.md @@ -35,7 +35,7 @@ Prerequisites for building Flink: * Unix-like environment (We use Linux, Mac OS X, Cygwin) * git * Maven (at least version 3.0.4) -* Java 6, 7 or 8 (Note that Oracle's JDK 6 library will fail to build Flink, but is able to run a pre-compiled package without problem) +* Java 7 or 8 ``` git clone https://github.com/apache/flink.git diff --git a/docs/setup/cluster_setup.md b/docs/setup/cluster_setup.md index 232708b5569e4..474aace711c82 100644 --- a/docs/setup/cluster_setup.md +++ b/docs/setup/cluster_setup.md @@ -40,7 +40,7 @@ and **Cygwin** (for Windows) and expects the cluster to consist of **one master node** and **one or more worker nodes**. Before you start to setup the system, make sure you have the following software installed **on each node**: -- **Java 1.6.x** or higher, +- **Java 1.7.x** or higher, - **ssh** (sshd must be running to use the Flink scripts that manage remote components) @@ -65,9 +65,9 @@ The command should output something comparable to the following on every node of your cluster (depending on your Java version, there may be small differences): ~~~bash -java version "1.6.0_22" -Java(TM) SE Runtime Environment (build 1.6.0_22-b04) -Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode) +java version "1.7.0_55" +Java(TM) SE Runtime Environment (build 1.7.0_55-b13) +Java HotSpot(TM) 64-Bit Server VM (build 24.55-b03, mixed mode) ~~~ To make sure the ssh daemon is running properly, you can use the command From 27044b4107eac80a1f24afaf4b1a4a79af423d76 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 6 Aug 2015 20:39:34 +0200 Subject: [PATCH 101/175] [FLINK-1177] [docs] Mark HDFS setup instructions as option in cluster setup docs --- docs/setup/cluster_setup.md | 200 +++++++++++++++++++----------------- 1 file changed, 103 insertions(+), 97 deletions(-) diff --git a/docs/setup/cluster_setup.md b/docs/setup/cluster_setup.md index 474aace711c82..29df3de3c7f74 100644 --- a/docs/setup/cluster_setup.md +++ b/docs/setup/cluster_setup.md @@ -158,14 +158,111 @@ echo "PermitUserEnvironment yes" >> /etc/ssh/sshd_config /etc/init.d/sshd restart ~~~ -## Hadoop Distributed Filesystem (HDFS) Setup -The Flink system currently uses the Hadoop Distributed Filesystem (HDFS) -to read and write data in a distributed fashion. It is possible to use -Flink without HDFS or other distributed file systems. +## Flink Setup + +Go to the [downloads page]({{site.baseurl}}/downloads.html) and get the ready to run +package. Make sure to pick the Flink package **matching your Hadoop +version**. + +After downloading the latest release, copy the archive to your master node and +extract it: + +~~~bash +tar xzf flink-*.tgz +cd flink-* +~~~ + +### Configuring the Cluster + +After having extracted the system files, you need to configure Flink for +the cluster by editing *conf/flink-conf.yaml*. + +Set the `jobmanager.rpc.address` key to point to your master node. Furthermode +define the maximum amount of main memory the JVM is allowed to allocate on each +node by setting the `jobmanager.heap.mb` and `taskmanager.heap.mb` keys. + +The value is given in MB. If some worker nodes have more main memory which you +want to allocate to the Flink system you can overwrite the default value +by setting an environment variable `FLINK_TM_HEAP` on the respective +node. + +Finally you must provide a list of all nodes in your cluster which shall be used +as worker nodes. Therefore, similar to the HDFS configuration, edit the file +*conf/slaves* and enter the IP/host name of each worker node. Each worker node +will later run a TaskManager. + +Each entry must be separated by a new line, as in the following example: + +~~~ +192.168.0.100 +192.168.0.101 +. +. +. +192.168.0.150 +~~~ + +The Flink directory must be available on every worker under the same +path. Similarly as for HDFS, you can use a shared NSF directory, or copy the +entire Flink directory to every worker node. + +Please see the [configuration page](config.html) for details and additional +configuration options. + +In particular, + + * the amount of available memory per TaskManager (`taskmanager.heap.mb`), + * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`), + * the total number of CPUs in the cluster (`parallelism.default`) and + * the temporary directories (`taskmanager.tmp.dirs`) + +are very important configuration values. + + +### Starting Flink + +The following script starts a JobManager on the local node and connects via +SSH to all worker nodes listed in the *slaves* file to start the +TaskManager on each node. Now your Flink system is up and +running. The JobManager running on the local node will now accept jobs +at the configured RPC port. + +Assuming that you are on the master node and inside the Flink directory: + +~~~bash +bin/start-cluster.sh +~~~ + +To stop Flink, there is also a `stop-cluster.sh` script. + + +### Starting Flink in the streaming mode + +~~~bash +bin/start-cluster-streaming.sh +~~~ + +The streaming mode changes the startup behavior of Flink: The system is not +bringing up the managed memory services with preallocated memory at the beginning. +Flink streaming is not using the managed memory employed by the batch operators. +By not starting these services with preallocated memory, streaming jobs can benefit +from more heap space being available. + +Note that you can still start batch jobs in the streaming mode. The memory manager +will then allocate memory segments from the Java heap as needed. + + +## Optional: Hadoop Distributed Filesystem (HDFS) Setup -Make sure to have a running HDFS installation. The following instructions are -just a general overview of some required settings. Please consult one of the +**NOTE** Flink does not require HDFS to run; HDFS is simply a typical choice of a distributed data +store to read data from (in parallel) and write results to. +If HDFS is already available on the cluster, or Flink is used purely with different storage +techniques (e.g., Apache Kafka, JDBC, Rabbit MQ, or other storage or message queues), this +setup step is not needed. + + +The following instructions are a general overview of usual required settings. Please consult one of the many installation guides available online for more detailed instructions. __Note that the following instructions are based on Hadoop 1.2 and might differ @@ -271,95 +368,4 @@ like to point you to the [Hadoop Quick Start](http://wiki.apache.org/hadoop/QuickStart) guide. -## Flink Setup - -Go to the [downloads page]({{site.baseurl}}/downloads.html) and get the ready to run -package. Make sure to pick the Flink package **matching your Hadoop -version**. - -After downloading the latest release, copy the archive to your master node and -extract it: - -~~~bash -tar xzf flink-*.tgz -cd flink-* -~~~ - -### Configuring the Cluster - -After having extracted the system files, you need to configure Flink for -the cluster by editing *conf/flink-conf.yaml*. - -Set the `jobmanager.rpc.address` key to point to your master node. Furthermode -define the maximum amount of main memory the JVM is allowed to allocate on each -node by setting the `jobmanager.heap.mb` and `taskmanager.heap.mb` keys. -The value is given in MB. If some worker nodes have more main memory which you -want to allocate to the Flink system you can overwrite the default value -by setting an environment variable `FLINK_TM_HEAP` on the respective -node. - -Finally you must provide a list of all nodes in your cluster which shall be used -as worker nodes. Therefore, similar to the HDFS configuration, edit the file -*conf/slaves* and enter the IP/host name of each worker node. Each worker node -will later run a TaskManager. - -Each entry must be separated by a new line, as in the following example: - -~~~ -192.168.0.100 -192.168.0.101 -. -. -. -192.168.0.150 -~~~ - -The Flink directory must be available on every worker under the same -path. Similarly as for HDFS, you can use a shared NSF directory, or copy the -entire Flink directory to every worker node. - -Please see the [configuration page](config.html) for details and additional -configuration options. - -In particular, - - * the amount of available memory per TaskManager (`taskmanager.heap.mb`), - * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`), - * the total number of CPUs in the cluster (`parallelism.default`) and - * the temporary directories (`taskmanager.tmp.dirs`) - -are very important configuration values. - - -### Starting Flink - -The following script starts a JobManager on the local node and connects via -SSH to all worker nodes listed in the *slaves* file to start the -TaskManager on each node. Now your Flink system is up and -running. The JobManager running on the local node will now accept jobs -at the configured RPC port. - -Assuming that you are on the master node and inside the Flink directory: - -~~~bash -bin/start-cluster.sh -~~~ - -To stop Flink, there is also a `stop-cluster.sh` script. - - -### Starting Flink in the streaming mode - -~~~bash -bin/start-cluster-streaming.sh -~~~ - -The streaming mode changes the startup behavior of Flink: The system is not -bringing up the managed memory services with preallocated memory at the beginning. -Flink streaming is not using the managed memory employed by the batch operators. -By not starting these services with preallocated memory, streaming jobs can benefit -from more heap space being available. - -Note that you can still start batch jobs in the streaming mode. The memory manager -will then allocate memory segments from the Java heap as needed. From 0ce757fd72446a60d5b79ea970dfb545406e3544 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 10 Aug 2015 15:37:33 +0200 Subject: [PATCH 102/175] [hotfix] Fix repository language statistics by adjusting .gitattributes --- .gitattributes | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitattributes b/.gitattributes index 69b47b5ade88f..b68afa618ec90 100644 --- a/.gitattributes +++ b/.gitattributes @@ -1 +1,3 @@ *.bat text eol=crlf +flink-runtime-web/web-dashboard/web/* linguist-vendored + From ceb5c5e2f442bfecd5a5dbbc6feb5cfb58cf267b Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 10 Aug 2015 18:33:17 +0200 Subject: [PATCH 103/175] [FLINK-2498] [tests] Clean up GroupReduceITCase and make it more robust by avoiding temp files --- .../scala/operators/GroupReduceITCase.scala | 821 ++++++++++-------- 1 file changed, 450 insertions(+), 371 deletions(-) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala index 09f0c5168461b..559d1d1e3460e 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala @@ -21,71 +21,53 @@ import java.lang.Iterable import org.apache.flink.api.common.functions._ import org.apache.flink.api.common.operators.Order -import org.apache.flink.api.scala.ExecutionEnvironment +import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.util.CollectionDataSets.{CrazyNested, POJO, MutableTuple3, -CustomType} +import org.apache.flink.api.scala.util.CollectionDataSets.{MutableTuple3, CustomType} import org.apache.flink.optimizer.Optimizer import org.apache.flink.configuration.Configuration -import org.apache.flink.core.fs.FileSystem.WriteMode import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{TestBaseUtils, MultipleProgramsTestBase} +import org.apache.flink.test.util.MultipleProgramsTestBase import org.apache.flink.util.Collector -import org.hamcrest.core.{IsNot, IsEqual} -import org.junit._ -import org.junit.rules.TemporaryFolder + +import org.junit.Test +import org.junit.Assert._ +import org.junit.Assume._ import org.junit.runner.RunWith import org.junit.runners.Parameterized import scala.collection.JavaConverters._ -import org.apache.flink.api.scala._ - @RunWith(classOf[Parameterized]) class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - if(expected != null) { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } - } - + + /** + * check correctness of groupReduce on tuples with key field selector + */ @Test def testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelector(): Unit = { - /* - * check correctness of groupReduce on tuples with key field selector - */ + val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(1).reduceGroup { in => in.map(t => (t._1, t._2)).reduce((l, r) => (l._1 + r._1, l._2)) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1\n" + "5,2\n" + "15,3\n" + "34,4\n" + "65,5\n" + "111,6\n" + val result: Seq[(Int, Long)] = reduceDs.collect().sortBy(_._1) + + val expected = Seq[(Int, Long)]( (1,1), (5,2), (15,3), (34,4), (65,5), (111,6) ) + + assertEquals(expected, result) } + /** + * check correctness of groupReduce on tuples with multiple key field selector + */ @Test def testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelector(): Unit = { - /* - * check correctness of groupReduce on tuples with multiple key field selector - */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets - .get5TupleDataSet(env) + val ds = CollectionDataSets.get5TupleDataSet(env) + val reduceDs = ds.groupBy(4, 0).reduceGroup { in => val (i, l, l2) = in @@ -93,18 +75,29 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas .reduce((l, r) => (l._1, l._2 + r._2, l._3)) (i, l, 0, "P-)", l2) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,0,P-),1\n" + "2,3,0,P-),1\n" + "2,2,0,P-),2\n" + "3,9,0,P-),2\n" + "3,6,0," + - "P-),3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,0,P-),1\n" + "5,29,0,P-)," + - "2\n" + "5,25,0,P-),3\n" - } - + + val result: Seq[(Int, Long, Int, String, Long)] = reduceDs.collect().sortBy( t=> (t._1, t._5)) + + val expected = Seq[(Int, Long, Int, String, Long)]( + (1,1,0,"P-)",1), + (2,3,0,"P-)",1), + (2,2,0,"P-)",2), + (3,9,0,"P-)",2), + (3,6,0,"P-)",3), + (4,17,0,"P-)",1), + (4,17,0,"P-)",2), + (5,11,0,"P-)",1), + (5,29,0,"P-)",2), + (5,25,0,"P-)",3) ) + + assertEquals(expected, result) + } + + /** + * check correctness of groupReduce on tuples with key field selector and group sorting + */ @Test def testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSorting(): Unit = { - /* - * check correctness of groupReduce on tuples with key field selector and group sorting - */ val env = ExecutionEnvironment.getExecutionEnvironment env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) @@ -112,37 +105,45 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas in => in.reduce((l, r) => (l._1 + r._1, l._2, l._3 + "-" + r._3)) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + - "5,2,Hello-Hello world\n" + - "15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" + - "34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" + - "65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" + - "111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n" - } - + + val result: Seq[(Int, Long, String)] = reduceDs.collect().sortBy(_._1) + + val expected = Seq[(Int, Long, String)] ( + (1,1,"Hi"), + (5,2,"Hello-Hello world"), + (15,3,"Hello world, how are you?-I am fine.-Luke Skywalker"), + (34,4,"Comment#1-Comment#2-Comment#3-Comment#4"), + (65,5,"Comment#5-Comment#6-Comment#7-Comment#8-Comment#9"), + (111,6,"Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15") ) + + assertEquals(expected, result) + } + + /** + * check correctness of groupReduce on tuples with key extractor + */ @Test def testCorrectnessOfGroupReduceOnTuplesWithKeyExtractor(): Unit = { - /* - * check correctness of groupReduce on tuples with key extractor - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(_._2).reduceGroup { in => in.map(t => (t._1, t._2)).reduce((l, r) => (l._1 + r._1, l._2)) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1\n" + "5,2\n" + "15,3\n" + "34,4\n" + "65,5\n" + "111,6\n" + + val result: Seq[(Int, Long)] = reduceDs.collect().sortBy(_._1) + + val expected = Seq[(Int, Long)]( (1,1), (5,2), (15,3), (34,4), (65,5), (111,6) ) + + assertEquals(expected, result) } + /** + * check correctness of groupReduce on custom type with type extractor + */ @Test def testCorrectnessOfGroupReduceOnCustomTypeWithTypeExtractor(): Unit = { - /* - * check correctness of groupReduce on custom type with type extractor - */ + val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getCustomTypeDataSet(env) val reduceDs = ds.groupBy(_.myInt).reduceGroup { @@ -161,17 +162,20 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas } o } - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "1,0,Hello!\n" + "2,3,Hello!\n" + "3,12,Hello!\n" + "4,30,Hello!\n" + "5,60," + - "Hello!\n" + "6,105,Hello!\n" + + val result: Seq[String] = reduceDs.map(_.toString).collect().sorted + + val expected = Seq[String]( "1,0,Hello!", "2,3,Hello!", "3,12,Hello!", + "4,30,Hello!", "5,60,Hello!", "6,105,Hello!") + + assertEquals(expected, result) } + /** + * check correctness of all-groupreduce for tuples + */ @Test def testCorrectnessOfAllGroupReduceForTuples(): Unit = { - /* - * check correctness of all-groupreduce for tuples - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.reduceGroup { @@ -184,16 +188,17 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas } (i, l, "Hello World") } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "231,91,Hello World\n" + + val result: (Int, Long, String) = reduceDs.collect().head + val expected: (Int, Long, String) = (231,91,"Hello World") + assertEquals(expected, result) } + /** + * check correctness of all-groupreduce for custom types + */ @Test def testCorrectnessOfAllGroupReduceForCustomTypes(): Unit = { - /* - * check correctness of all-groupreduce for custom types - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getCustomTypeDataSet(env) val reduceDs = ds.reduceGroup { @@ -205,16 +210,17 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas } o } - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "91,210,Hello!" + val result : String = reduceDs.collect().head.toString() + val expected = "91,210,Hello!" + assertEquals(expected, result) } + /** + * check correctness of groupReduce with broadcast set + */ @Test def testCorrectnessOfGroupReduceWithBroadcastSet(): Unit = { - /* - * check correctness of groupReduce with broadcast set - */ + val env = ExecutionEnvironment.getExecutionEnvironment val intDs = CollectionDataSets.getIntDataSet(env) val ds = CollectionDataSets.get3TupleDataSet(env) @@ -239,20 +245,30 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas out.collect((i, l, f2Replace)) } }).withBroadcastSet(intDs, "ints") - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,55\n" + "5,2,55\n" + "15,3,55\n" + "34,4,55\n" + "65,5,55\n" + "111,6,55\n" - } - + + + val result: Seq[(Int, Long, String)] = reduceDs.collect().sortBy(_._1) + val expected = Seq[(Int, Long, String)]( + (1,1,"55"), + (5,2,"55"), + (15,3,"55"), + (34,4,"55"), + (65,5,"55"), + (111,6,"55") ) + + assertEquals(expected, result) + } + + /** + * check correctness of groupReduce if UDF returns input objects multiple times and + * changes it in between + */ @Test def testCorrectnessOfGroupReduceIfUDFReturnsInputObjectMultipleTimesWhileChangingIt(): Unit = { - /* - * check correctness of groupReduce if UDF returns input objects multiple times and - * changes it in between - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) .map( t => MutableTuple3(t._1, t._2, t._3) ) + val reduceDs = ds.groupBy(1).reduceGroup { (in, out: Collector[MutableTuple3[Int, Long, String]]) => for (t <- in) { @@ -266,59 +282,75 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas } } } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "11,1,Hi!\n" + "21,1,Hi again!\n" + "12,2,Hi!\n" + "22,2,Hi again!\n" + "13,2," + - "Hi!\n" + "23,2,Hi again!\n" - } - + val result: Seq[String] = reduceDs.collect().map(x => s"${x._1},${x._2},${x._3}").sorted + + val expected = Seq[String]( + "11,1,Hi!", "21,1,Hi again!", + "12,2,Hi!", "22,2,Hi again!", + "13,2,Hi!", "23,2,Hi again!").sorted + + assertEquals(expected, result) + } + + /** + * check correctness of groupReduce on custom type with key extractor and combine + */ @Test def testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine(): Unit = { - /* - * check correctness of groupReduce on custom type with key extractor and combine - */ - org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(TestExecutionMode.COLLECTION))) + + org.junit.Assume.assumeFalse(mode == TestExecutionMode.COLLECTION) val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getCustomTypeDataSet(env) - val reduceDs = ds.groupBy(_.myInt).reduceGroup(new CustomTypeGroupReduceWithCombine) - - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = - "1,0,test1\n" + "2,3,test2\n" + "3,12,test3\n" + "4,30,test4\n" + "5,60," + - "test5\n" + "6,105,test6\n" + val reduceDs = ds.groupBy(_.myInt).reduceGroup(new CustomTypeGroupReduceWithCombine()) + + val result: Seq[String] = reduceDs.collect().map(_.toString()).sorted + + val expected = Seq[String]( + "1,0,test1", + "2,3,test2", + "3,12,test3", + "4,30,test4", + "5,60,test5", + "6,105,test6") + assertEquals(expected, result) } + /** + * check correctness of groupReduce on tuples with combine + */ @Test def testCorrectnessOfGroupReduceOnTuplesWithCombine(): Unit = { - /* - * check correctness of groupReduce on tuples with combine - */ - org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(TestExecutionMode.COLLECTION))) + + org.junit.Assume.assumeFalse(mode == TestExecutionMode.COLLECTION) val env = ExecutionEnvironment.getExecutionEnvironment // important because it determines how often the combiner is called env.setParallelism(2) val ds = CollectionDataSets.get3TupleDataSet(env) - val reduceDs = ds.groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine) - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = - "1,test1\n" + "5,test2\n" + "15,test3\n" + "34,test4\n" + "65,test5\n" + "111," + - "test6\n" - - } - + val reduceDs = ds.groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine()) + + val result: Seq[(Int, String)] = reduceDs.collect().sortBy(_._1) + val expected = Seq[(Int, String)]( + (1,"test1"), + (5,"test2"), + (15,"test3"), + (34,"test4"), + (65,"test5"), + (111,"test6") ) + assertEquals(expected, result) + } + + /** + * check correctness of all-groupreduce for tuples with combine + */ @Test def testCorrectnessOfAllGroupReduceForTuplesWithCombine(): Unit = { - /* - * check correctness of all-groupreduce for tuples with combine - */ - org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(TestExecutionMode.COLLECTION))) + + org.junit.Assume.assumeFalse(mode == TestExecutionMode.COLLECTION) val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env).map(t => t).setParallelism(4) @@ -328,42 +360,45 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas val reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine).withParameters(cfg) - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "322," + - "testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n" - + val result: (Int, String) = reduceDs.collect().head + val expected = (322, + "testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest") + assertEquals(expected, result) } + /** + * check correctness of groupReduce with descending group sort + */ @Test def testCorrectnessOfGroupReduceWithDescendingGroupSort(): Unit = { - /* - * check correctness of groupReduce with descending group sort - */ + val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup { in => in.reduce((l, r) => (l._1 + r._1, l._2, l._3 + "-" + r._3)) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "5,2,Hello world-Hello\n" + "15,3,Luke Skywalker-I am fine.-Hello " + - "world, how are you?\n" + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + "65,5," + - "Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,6," + - "Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n" + val result: Seq[(Int, Long, String)] = reduceDs.collect().sortBy(_._1) + val expected = Seq[(Int, Long, String)]( + (1,1,"Hi"), + (5,2,"Hello world-Hello"), + (15,3,"Luke Skywalker-I am fine.-Hello world, how are you?"), + (34,4,"Comment#4-Comment#3-Comment#2-Comment#1"), + (65,5,"Comment#9-Comment#8-Comment#7-Comment#6-Comment#5"), + (111,6,"Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10") ) + + assertEquals(expected, result) } + /** + * check correctness of groupReduce on tuples with tuple-returning key selector + */ @Test def testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector(): Unit = { - /* - * check correctness of groupReduce on tuples with tuple-returning key selector - */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets - .get5TupleDataSet(env) + val ds = CollectionDataSets.get5TupleDataSet(env) + val reduceDs = ds.groupBy( t => (t._1, t._5)).reduceGroup { in => val (i, l, l2) = in @@ -372,42 +407,59 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas (i, l, 0, "P-)", l2) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,0,P-),1\n" + "2,3,0,P-),1\n" + "2,2,0,P-),2\n" + "3,9,0,P-),2\n" + "3,6,0," + - "P-),3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,0,P-),1\n" + "5,29,0,P-)," + - "2\n" + "5,25,0,P-),3\n" - } - + val result: Seq[(Int, Long, Int, String, Long)] = reduceDs.collect().sortBy(x => (x._1, x._5)) + + val expected = Seq[(Int, Long, Int, String, Long)]( + (1,1,0,"P-)",1), + (2,3,0,"P-)",1), + (2,2,0,"P-)",2), + (3,9,0,"P-)",2), + (3,6,0,"P-)",3), + (4,17,0,"P-)",1), + (4,17,0,"P-)",2), + (5,11,0,"P-)",1), + (5,29,0,"P-)",2), + (5,25,0,"P-)",3) ) + + assertEquals(expected, result) + } + + /** + * check that input of combiner is also sorted for combinable groupReduce with group + * sorting + */ @Test def testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting(): Unit = { - /* - * check that input of combiner is also sorted for combinable groupReduce with group - * sorting - */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) - val ds = CollectionDataSets.get3TupleDataSet(env).map { t => + val ds = CollectionDataSets.get3TupleDataSet(env).map { t => MutableTuple3(t._1, t._2, t._3) } - val reduceDs = ds.groupBy(1) - .sortGroup(0, Order.ASCENDING).reduceGroup(new OrderCheckingCombinableReduce) - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "7,4," + - "Comment#1\n" + "11,5,Comment#5\n" + "16,6,Comment#10\n" - } - + val reduceDs = ds.groupBy(1).sortGroup(0, Order.ASCENDING) + .reduceGroup(new OrderCheckingCombinableReduce()) + + + val result: Seq[String] = reduceDs.collect().sortBy(_._1).map(x => s"${x._1},${x._2},${x._3}") + + val expected = Seq[String] ( + "1,1,Hi", + "2,2,Hello", + "4,3,Hello world, how are you?", + "7,4,Comment#1", + "11,5,Comment#5", + "16,6,Comment#10") + assertEquals(expected, result) + } + + /** + * Deep nesting test + * + null value in pojo + */ @Test def testDeepNestingAndNullValueInPojo(): Unit = { - /* - * Deep nesting test - * + null value in pojo - */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets - .getCrazyNestedDataSet(env) + val ds = CollectionDataSets.getCrazyNestedDataSet(env) + val reduceDs = ds.groupBy("nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal") .reduceGroup { in => @@ -419,211 +471,236 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas } (n, c) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "aa,1\nbb,2\ncc,3\n" + + val result: Seq[(String, Int)] = reduceDs.collect().sortBy(_._1) + val expected = Seq[(String, Int)]( + ("aa",1), + ("bb",2), + ("cc",3) ) + assertEquals(expected, result) } + /** + * Test Pojo containing a Writable and Tuples + */ @Test def testPojoContainigAWritableAndTuples(): Unit = { - /* - * Test Pojo containing a Writable and Tuples - */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets - .getPojoContainingTupleAndWritable(env) - val reduceDs = ds.groupBy("hadoopFan", "theTuple.*").reduceGroup(new - GroupReduceFunction[CollectionDataSets.PojoContainingTupleAndWritable, Integer] { - def reduce( - values: Iterable[CollectionDataSets.PojoContainingTupleAndWritable], - out: Collector[Integer]) { + val ds = CollectionDataSets.getPojoContainingTupleAndWritable(env) + + val reduceDs = ds.groupBy("hadoopFan", "theTuple.*").reduceGroup { + (values, out: Collector[Int]) => { var c: Int = 0 - for (v <- values.asScala) { + for (v <- values) { c += 1 } out.collect(c) } - }) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "1\n5\n" + } + + val result: Seq[Int] = reduceDs.collect().sorted + val expected = Seq[Int](1, 5) + assertEquals(expected, result) } + /** + * Test Tuple containing pojos and regular fields + */ @Test def testTupleContainingPojosAndRegularFields(): Unit ={ - /* - * Test Tuple containing pojos and regular fields - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getTupleContainingPojos(env) - val reduceDs = ds.groupBy("_1", "_2.*").reduceGroup( - new GroupReduceFunction[(Int, CrazyNested, POJO), Int] { - def reduce(values: Iterable[(Int, CrazyNested, POJO)], out: Collector[Int]) { - var c: Int = 0 - for (v <- values.asScala) { - c += 1 - } - out.collect(c) - } - }) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "3\n1\n" + + val reduceDs = ds.groupBy("_1", "_2.*").reduceGroup { + (values, out: Collector[Int]) => { + out.collect(values.size) + } + } + + val result: Seq[Int] = reduceDs.collect().sorted + val expected = Seq[Int](1, 3) + assertEquals(expected, result) } + /** + * Test string-based definition on group sort, based on test: + * check correctness of groupReduce with descending group sort + */ @Test def testStringBasedDefinitionOnGroupSort(): Unit = { - /* - * Test string-based definition on group sort, based on test: - * check correctness of groupReduce with descending group sort - */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) + val reduceDs = ds.groupBy(1) .sortGroup("_3", Order.DESCENDING) .reduceGroup { in => in.reduce((l, r) => (l._1 + r._1, l._2, l._3 + "-" + r._3)) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "5,2,Hello world-Hello\n" + "15,3,Luke Skywalker-I am fine.-Hello " + - "world, how are you?\n" + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + "65,5," + - "Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,6," + - "Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n" - } - + + val result: Seq[(Int, Long, String)] = reduceDs.collect().sortBy(_._1) + val expected = Seq[(Int, Long, String)]( + (1,1,"Hi"), + (5,2,"Hello world-Hello"), + (15,3,"Luke Skywalker-I am fine.-Hello world, how are you?"), + (34,4,"Comment#4-Comment#3-Comment#2-Comment#1"), + (65,5,"Comment#9-Comment#8-Comment#7-Comment#6-Comment#5"), + (111,6,"Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10") ) + assertEquals(expected, result) + } + + /** + * Test int-based definition on group sort, for (full) nested Tuple + */ @Test def testIntBasedDefinitionOnGroupSortForFullNestedTuple(): Unit = { - /* - * Test int-based definition on group sort, for (full) nested Tuple - */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) + val reduceDs = ds.groupBy("_2").sortGroup(0, Order.DESCENDING) - .reduceGroup(new NestedTupleReducer) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n" + "c--(4,9)-(3,6)-(3,3)-\n" - } - + .reduceGroup(new NestedTupleReducer()) + + val result: Seq[String] = reduceDs.map(_.toString()).collect().sorted + val expected = Seq[String]( + "a--(2,1)-(1,3)-(1,2)-", + "b--(2,2)-", + "c--(4,9)-(3,6)-(3,3)-") + assertEquals(expected, result) + } + + /** + * Test int-based definition on group sort, for (partial) nested Tuple ASC + */ @Test def testIntBasedDefinitionOnGroupSortForPartialNestedTuple(): Unit = { - /* - * Test int-based definition on group sort, for (partial) nested Tuple ASC - */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) + val reduceDs = ds.groupBy("_2") .sortGroup("_1._1", Order.ASCENDING) .sortGroup("_1._2", Order.ASCENDING) .reduceGroup(new NestedTupleReducer) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "a--(1,2)-(1,3)-(2,1)-\n" + "b--(2,2)-\n" + "c--(3,3)-(3,6)-(4,9)-\n" + + val result: Seq[String] = reduceDs.map(_.toString).collect().sorted + val expected = Seq[String]( + "a--(1,2)-(1,3)-(2,1)-", + "b--(2,2)-", + "c--(3,3)-(3,6)-(4,9)-") + assertEquals(expected, result) } + /** + * Test string-based definition on group sort, for (partial) nested Tuple DESC + */ @Test def testStringBasedDefinitionOnGroupSortForPartialNestedTuple(): Unit = { - /* - * Test string-based definition on group sort, for (partial) nested Tuple DESC - */ + val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) + val reduceDs = ds.groupBy("_2") .sortGroup("_1._1", Order.DESCENDING) .sortGroup("_1._2", Order.ASCENDING) .reduceGroup(new NestedTupleReducer) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "a--(2,1)-(1,2)-(1,3)-\n" + "b--(2,2)-\n" + "c--(4,9)-(3,3)-(3,6)-\n" + + val result: Seq[String] = reduceDs.map(_.toString()).collect().sorted + val expected = Seq[String]( + "a--(2,1)-(1,2)-(1,3)-", + "b--(2,2)-", + "c--(4,9)-(3,3)-(3,6)-") + assertEquals(expected, result) } + /** + * Test string-based definition on group sort, for two grouping keys + */ @Test def testStringBasedDefinitionOnGroupSortForTwoGroupingKeys(): Unit = { - /* - * Test string-based definition on group sort, for two grouping keys - */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) + val reduceDs = ds.groupBy("_2") .sortGroup("_1._1", Order.DESCENDING) .sortGroup("_1._2", Order.DESCENDING) - .reduceGroup(new NestedTupleReducer) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n" + "c--(4,9)-(3,6)-(3,3)-\n" - } - + .reduceGroup(new NestedTupleReducer()) + + val result: Seq[String] = reduceDs.map(_.toString()).collect().sorted + val expected = Seq[String] ( + "a--(2,1)-(1,3)-(1,2)-", + "b--(2,2)-", + "c--(4,9)-(3,6)-(3,3)-") + assertEquals(expected, result) + } + + /** + * Test string-based definition on group sort, for two grouping keys with Pojos + */ @Test def testStringBasedDefinitionOnGroupSortForTwoGroupingKeysWithPojos(): Unit = { - /* - * Test string-based definition on group sort, for two grouping keys with Pojos - */ val env = ExecutionEnvironment.getExecutionEnvironment env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env) val reduceDs = ds.groupBy("hadoopFan") .sortGroup("theTuple._1", Order.DESCENDING) .sortGroup("theTuple._2", Order.DESCENDING) - .reduceGroup( - new GroupReduceFunction[CollectionDataSets.PojoContainingTupleAndWritable, String] { - def reduce( - values: Iterable[CollectionDataSets.PojoContainingTupleAndWritable], - out: Collector[String]) { - var once: Boolean = false - val concat: StringBuilder = new StringBuilder - for (value <- values.asScala) { - if (!once) { - concat.append(value.hadoopFan.get) - concat.append("---") - once = true - } - concat.append(value.theTuple) - concat.append("-") + .reduceGroup { + (values, out: Collector[String]) => { + var once: Boolean = false + val concat: StringBuilder = new StringBuilder + for (value <- values) { + if (!once) { + concat.append(value.hadoopFan.get) + concat.append("---") + once = true } - out.collect(concat.toString()) + concat.append(value.theTuple) + concat.append("-") } - }) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "1---(10,100)-\n" + "2---(30,600)-(30,400)-(30,200)-(20,201)-(20,200)-\n" + out.collect(concat.toString()) + } + } + + val result: Seq[String] = reduceDs.map(_.toString()).collect().sorted + val expected = Seq[String]( + "1---(10,100)-", + "2---(30,600)-(30,400)-(30,200)-(20,201)-(20,200)-") + assertEquals(expected, result) } + /** + * check correctness of sorted groupReduce on tuples with keyselector sorting + */ @Test - def testTupleKeySelectorGroupSort: Unit = { - /* - * check correctness of sorted groupReduce on tuples with keyselector sorting - */ + def testTupleKeySelectorGroupSort(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) + val reduceDs = ds.groupBy(_._2).sortGroup(_._3, Order.DESCENDING).reduceGroup { in => in.reduce((l, r) => (l._1 + r._1, l._2, l._3 + "-" + r._3)) } - reduceDs.writeAsCsv(resultPath) - env.execute() - expected = "1,1,Hi\n" + - "5,2,Hello world-Hello\n" + - "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + - "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + - "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + - "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n" - } - - @Test - def testPojoKeySelectorGroupSort: Unit = { - /* + + val result: Seq[(Int, Long, String)] = reduceDs.collect().sortBy(_._1) + + val expected = Seq[(Int, Long, String)]( + (1,1,"Hi"), + (5,2,"Hello world-Hello"), + (15,3,"Luke Skywalker-I am fine.-Hello world, how are you?"), + (34,4,"Comment#4-Comment#3-Comment#2-Comment#1"), + (65,5,"Comment#9-Comment#8-Comment#7-Comment#6-Comment#5"), + (111,6,"Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10") ) + + assertEquals(expected, result) + } + + /** * check correctness of sorted groupReduce on custom type with keyselector sorting */ + @Test + def testPojoKeySelectorGroupSort(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getCustomTypeDataSet(env) + val reduceDs = ds.groupBy(_.myInt).sortGroup(_.myString, Order.DESCENDING).reduceGroup { in => val iter = in.toIterator @@ -642,48 +719,51 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas o.myString = concat.toString() o } - reduceDs.writeAsText(resultPath) - env.execute() - expected = "1,0,Hi\n" + - "2,3,Hello world-Hello\n" + - "3,12,Luke Skywalker-I am fine.-Hello world, how are you?\n" + - "4,30,Comment#4-Comment#3-Comment#2-Comment#1\n" + - "5,60,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + - "6,105,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n" - } - + + val result: Seq[String] = reduceDs.map(_.toString()).collect().sorted + + val expected = Seq[String]( "1,0,Hi", + "2,3,Hello world-Hello", + "3,12,Luke Skywalker-I am fine.-Hello world, how are you?", + "4,30,Comment#4-Comment#3-Comment#2-Comment#1", + "5,60,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5", + "6,105,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10" ) + + assertEquals(expected, result) + } + + /** + * check correctness of sorted groupReduce with combine on tuples with keyselector sorting + */ @Test - def testTupleKeySelectorSortWithCombine: Unit = { - /* - * check correctness of sorted groupReduce with combine on tuples with keyselector sorting - */ + def testTupleKeySelectorSortWithCombine(): Unit = { + + assumeTrue(mode != TestExecutionMode.COLLECTION) + val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(_._2).sortGroup(_._3, Order.DESCENDING) - .reduceGroup(new Tuple3SortedGroupReduceWithCombine) - reduceDs.writeAsCsv(resultPath) - env.execute() - if (mode == TestExecutionMode.COLLECTION) { - expected = null - } else { - expected = "1,Hi\n" + - "5,Hello world-Hello\n" + - "15,Luke Skywalker-I am fine.-Hello world, how are you?\n" + - "34,Comment#4-Comment#3-Comment#2-Comment#1\n" + - "65,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + - "111,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n" - } - } - + .reduceGroup(new Tuple3SortedGroupReduceWithCombine()) + + val result : Seq[(Int, String)] = reduceDs.collect().sortBy(_._1) + + val expected = Seq[(Int, String)]( (1,"Hi"), + (5,"Hello world-Hello"), + (15,"Luke Skywalker-I am fine.-Hello world, how are you?"), + (34,"Comment#4-Comment#3-Comment#2-Comment#1"), + (65,"Comment#9-Comment#8-Comment#7-Comment#6-Comment#5"), + (111,"Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10") ) + + assertEquals(expected, result) + } + + /** + * check correctness of sorted groupReduceon with Tuple2 keyselector sorting + */ @Test - def testTupleKeySelectorSortCombineOnTuple: Unit = { - /* - * check correctness of sorted groupReduceon with Tuple2 keyselector sorting - */ + def testTupleKeySelectorSortCombineOnTuple(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.get5TupleDataSet(env) val reduceDs = ds.groupBy(_._1).sortGroup(t => (t._5, t._3), Order.DESCENDING).reduceGroup{ @@ -706,58 +786,57 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas (key, sum, 0, concat.toString(), s) // in.reduce((l, r) => (l._1, l._2 + r._2, 0, l._4 + "-" + r._4, l._5)) } - reduceDs.writeAsCsv(resultPath) - env.execute() - expected = "1,1,0,Hallo,1\n" + - "2,5,0,Hallo Welt-Hallo Welt wie,1\n" + - "3,15,0,BCD-ABC-Hallo Welt wie gehts?,2\n" + - "4,34,0,FGH-CDE-EFG-DEF,1\n" + - "5,65,0,IJK-HIJ-KLM-JKL-GHI,1\n" - } - - + + val result: Seq[(Int, Long, Int, String, Long)] = reduceDs.collect().sortBy(_._1) + val expected = Seq[(Int, Long, Int, String, Long)]( + (1,1,0,"Hallo",1), + (2,5,0,"Hallo Welt-Hallo Welt wie",1), + (3,15,0,"BCD-ABC-Hallo Welt wie gehts?",2), + (4,34,0,"FGH-CDE-EFG-DEF",1), + (5,65,0,"IJK-HIJ-KLM-JKL-GHI",1) ) + assertEquals(expected, result) + } + + /** + * Test grouping with pojo containing multiple pojos (was a bug) + */ @Test - def testGroupingWithPojoContainingMultiplePojos: Unit = { - /* - * Test grouping with pojo containing multiple pojos (was a bug) - */ + def testGroupingWithPojoContainingMultiplePojos(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) val ds = CollectionDataSets.getPojoWithMultiplePojos(env) - val reduceDs = ds.groupBy("p2.a2") - .reduceGroup { - new GroupReduceFunction[CollectionDataSets.PojoWithMultiplePojos, String] { - def reduce( - values: Iterable[CollectionDataSets.PojoWithMultiplePojos], - out: Collector[String]) { - val concat: StringBuilder = new StringBuilder - for (value <- values.asScala) { - concat.append(value.p2.a2) - } - out.collect(concat.toString()) + + val reduceDs = ds.groupBy("p2.a2").reduceGroup { + (values, out: Collector[String]) => { + val concat: StringBuilder = new StringBuilder() + for (value <- values) { + concat.append(value.p2.a2) } + out.collect(concat.toString()) } } - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "b\nccc\nee\n" + + val result : Seq[String] = reduceDs.map(_.toString()).collect().sorted + val expected = Seq[String]("b", "ccc", "ee") + assertEquals(expected, result) } @Test - def testWithAtomic1: Unit = { + def testWithAtomic1(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment val ds = env.fromElements(0, 1, 1, 2) val reduceDs = ds.groupBy("*").reduceGroup((ints: Iterator[Int]) => ints.next()) - reduceDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "0\n1\n2" + + val result: Seq[Int] = reduceDs.collect().sorted + val expected = Seq[Int](0, 1, 2) + assertEquals(expected, result) } } @RichGroupReduceFunction.Combinable -class OrderCheckingCombinableReduce - extends RichGroupReduceFunction[MutableTuple3[Int, Long, String], - MutableTuple3[Int, Long, String]] { +class OrderCheckingCombinableReduce extends +RichGroupReduceFunction[MutableTuple3[Int, Long, String], MutableTuple3[Int, Long, String]] { + def reduce( values: Iterable[MutableTuple3[Int, Long, String]], out: Collector[MutableTuple3[Int, Long, String]]) { From b19e5e3bb39fad27962415ae432d616aa98fc298 Mon Sep 17 00:00:00 2001 From: ffbin <869218239@qq.com> Date: Tue, 4 Aug 2015 20:12:46 +0800 Subject: [PATCH 104/175] [FLINK-2483] [runtime] Add default branch of switch(scheduleMode) in scheduleForExecution function This closes #984 --- .../org/apache/flink/runtime/executiongraph/ExecutionGraph.java | 2 ++ 1 file changed, 2 insertions(+) mode change 100644 => 100755 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java old mode 100644 new mode 100755 index be92bd4b79b95..b52a4e814229d --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -666,6 +666,8 @@ public void scheduleForExecution(Scheduler scheduler) throws JobException { case BACKTRACKING: // go back from vertices that need computation to the ones we need to run throw new JobException("BACKTRACKING is currently not supported as schedule mode."); + default: + throw new JobException("Schedule mode is invalid."); } } else { From 4ce9475cd2ba32817f74dc2883bbd2e9dea12db1 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 11 Aug 2015 11:07:31 +0200 Subject: [PATCH 105/175] [travis tools] Adds $HOME/bin/artifacts to $PATH of watchdog script --- tools/travis_mvn_watchdog.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 8878b651dd315..08cf6efa415d9 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -81,7 +81,7 @@ upload_artifacts_s3() { # Install artifacts tool curl -sL https://raw.githubusercontent.com/travis-ci/artifacts/master/install | bash - PATH=$HOME/bin:$PATH + PATH=$HOME/bin/artifacts:$HOME/bin:$PATH echo "UPLOADING build artifacts." From d09fc7663542f04b198ddb2bc3eb51d5786d56fa Mon Sep 17 00:00:00 2001 From: Enrique Bautista Date: Mon, 10 Aug 2015 22:19:39 +0200 Subject: [PATCH 106/175] [FLINK-2357] [web dashboard] Update Node.js installation instructions This closes #1006 --- flink-runtime-web/README.md | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/flink-runtime-web/README.md b/flink-runtime-web/README.md index 2e8c23e2c1c84..4eb32a1e1488f 100644 --- a/flink-runtime-web/README.md +++ b/flink-runtime-web/README.md @@ -60,14 +60,10 @@ The dashboard files are all pre-built, so one can try it out without building it Depending on your version of Linux or MacOS, you may need to manually install *node.js* and *bower*. -#### Ubuntu Linux (12.04 and 14.04) +#### Ubuntu Linux + +Install *node.js* by following [these instructions](https://github.com/joyent/node/wiki/installing-node.js-via-package-manager). -Install *node.js* via -``` -sudo add-apt-repository ppa:chris-lea/node.js -sudo apt-get update -sudo apt-get -y install nodejs -``` Verify that the installed version is at least *2.11.3*, via `npm -version`. From 5bb855bac7441701495ce47db7ba03ab0e0c6963 Mon Sep 17 00:00:00 2001 From: mjsax Date: Sun, 9 Aug 2015 14:22:56 +0200 Subject: [PATCH 107/175] [FLINK-2502] [storm compatibility] Fix FiniteStormSpout documenation rendering - added missing empty lines - additional minor improvements This closes #1002 --- docs/apis/storm_compatibility.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/apis/storm_compatibility.md b/docs/apis/storm_compatibility.md index 1390b923586b1..b38667bfc2e7e 100644 --- a/docs/apis/storm_compatibility.md +++ b/docs/apis/storm_compatibility.md @@ -171,10 +171,10 @@ See [BoltTokenizerWordCountPojo](https://github.com/apache/flink/tree/master/fli ## Finite Storm Spouts -In Flink streaming, sources can be finite - i.e. emit a finite number of records and stop after emitting the last record -, however, Storm spouts always emit infinite streams. +In Flink streaming, sources can be finite – i.e., emit a finite number of records and stop after emitting the last record –, however, Storm spouts always emit infinite streams. The bridge between the two approach is the `FiniteStormSpout` interface which, in addition to `IRichSpout`, contains a `reachedEnd()` method, where the user can specify a stopping-condition. The user can create a finite Storm spout by implementing this interface instead of `IRichSpout`, and implementing the `reachedEnd()`method in addition. -When used as part of a Flink topology, a `FiniteStormSpout` should be wrapped in a `FiniteStormSpoutWrapper` class. +When used as part of a Flink topology, a `FiniteStormSpout` should be wrapped by `FiniteStormSpoutWrapper`. Although finite Storm spouts are not necessary to embed Storm spouts into a Flink streaming program or to submit a whole Storm topology to Flink, there are cases where they may come in handy: @@ -186,6 +186,7 @@ Although finite Storm spouts are not necessary to embed Storm spouts into a Flin A `FiniteStormSpout` can be still used as a normal, infinite Storm spout by changing its wrapper class to `StormSpoutWraper` in the Flink topology. An example of a finite Storm spout that emits records for 10 seconds only: +
    ~~~java @@ -203,6 +204,7 @@ public class TimedFiniteStormSpout extends AbstractStormSpout implements FiniteS
    Using a `FiniteStormSpout` in a Flink topology: +
    ~~~java From b42fbf7a81c5b57dcf9760825edb175ffd944fb2 Mon Sep 17 00:00:00 2001 From: HuangWHWHW <404823056@qq.com> Date: Sat, 8 Aug 2015 14:46:46 +0800 Subject: [PATCH 108/175] [FLINK-2500] [streaming] Code cleanup in DataStream This closes #1001 --- .../apache/flink/streaming/api/datastream/DataStream.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 5bd3fb8756585..10ed5e3ab46a3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -151,10 +151,10 @@ public DataStream(DataStream dataStream) { this.iterationWaitTime = dataStream.iterationWaitTime; this.unionedStreams = new ArrayList>(); this.unionedStreams.add(this); - if (dataStream.unionedStreams.size() > 1) { - for (int i = 1; i < dataStream.unionedStreams.size(); i++) { - this.unionedStreams.add(new DataStream(dataStream.unionedStreams.get(i))); - } + + int size = dataStream.unionedStreams.size(); + for (int i = 1; i < size; i++) { + this.unionedStreams.add(new DataStream(dataStream.unionedStreams.get(i))); } } From f50ae26a2fb4a0c7f5b390e2f0f5528be9f61730 Mon Sep 17 00:00:00 2001 From: Pieter-Jan Van Aeken Date: Mon, 10 Aug 2015 15:16:08 +0200 Subject: [PATCH 109/175] [FLINK-2277] [scala api] Add flag to set delta iteration solution set to unmanaged This closes #1005 --- .../org/apache/flink/api/scala/DataSet.scala | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala index 167aa265dd4f0..207bc5d47c347 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala @@ -1067,6 +1067,36 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { wrap(result) } + /** + * Creates a new DataSet by performing delta (or workset) iterations using the given step + * function. At the beginning `this` DataSet is the solution set and `workset` is the Workset. + * The iteration step function gets the current solution set and workset and must output the + * delta for the solution set and the workset for the next iteration. + * + * Note: The syntax of delta iterations are very likely going to change soon. + */ + def iterateDelta[R: ClassTag](workset: DataSet[R], maxIterations: Int, keyFields: Array[Int], + solutionSetUnManaged: Boolean)( + stepFunction: (DataSet[T], DataSet[R]) => (DataSet[T], DataSet[R])) = { + val key = new ExpressionKeys[T](keyFields, javaSet.getType, false) + + val iterativeSet = new DeltaIteration[T, R]( + javaSet.getExecutionEnvironment, + javaSet.getType, + javaSet, + workset.javaSet, + key, + maxIterations) + + iterativeSet.setSolutionSetUnManaged(solutionSetUnManaged) + + val (newSolution, newWorkset) = stepFunction( + wrap(iterativeSet.getSolutionSet), + wrap(iterativeSet.getWorkset)) + val result = iterativeSet.closeWith(newSolution.javaSet, newWorkset.javaSet) + wrap(result) + } + /** * Creates a new DataSet by performing delta (or workset) iterations using the given step * function. At the beginning `this` DataSet is the solution set and `workset` is the Workset. @@ -1094,6 +1124,36 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { wrap(result) } + /** + * Creates a new DataSet by performing delta (or workset) iterations using the given step + * function. At the beginning `this` DataSet is the solution set and `workset` is the Workset. + * The iteration step function gets the current solution set and workset and must output the + * delta for the solution set and the workset for the next iteration. + * + * Note: The syntax of delta iterations are very likely going to change soon. + */ + def iterateDelta[R: ClassTag](workset: DataSet[R], maxIterations: Int, keyFields: Array[String], + solutionSetUnManaged: Boolean)( + stepFunction: (DataSet[T], DataSet[R]) => (DataSet[T], DataSet[R])) = { + + val key = new ExpressionKeys[T](keyFields, javaSet.getType) + val iterativeSet = new DeltaIteration[T, R]( + javaSet.getExecutionEnvironment, + javaSet.getType, + javaSet, + workset.javaSet, + key, + maxIterations) + + iterativeSet.setSolutionSetUnManaged(solutionSetUnManaged) + + val (newSolution, newWorkset) = stepFunction( + wrap(iterativeSet.getSolutionSet), + wrap(iterativeSet.getWorkset)) + val result = iterativeSet.closeWith(newSolution.javaSet, newWorkset.javaSet) + wrap(result) + } + // ------------------------------------------------------------------------------------------- // Custom Operators // ------------------------------------------------------------------------------------------- From 4ca7df592538edb6fd69159c7b955673ed3af820 Mon Sep 17 00:00:00 2001 From: mbalassi Date: Tue, 4 Aug 2015 10:11:10 +0200 Subject: [PATCH 110/175] [streaming] [tests] Checkpointing tests refactor & cleanup --- docs/apis/streaming_guide.md | 2 +- .../CoStreamCheckpointingITCase.java | 215 ++++------- .../PartitionedStateCheckpointingITCase.java | 92 ++--- .../checkpointing/StateCheckpoinedITCase.java | 158 +++----- .../StreamCheckpointNotifierITCase.java | 341 ++++++++++++++++++ .../StreamCheckpointingITCase.java | 224 ++++-------- .../StreamFaultToleranceTestBase.java | 150 ++++++++ 7 files changed, 693 insertions(+), 489 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index 080272b52f842..b20482e5b84ea 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -1355,7 +1355,7 @@ public static class CounterSource implements RichParallelSourceFunction { } {% endhighlight %} -Some operators might need the information when a checkpoint is fully acknowledged by Flink to communicate that with the outside world. In this case see the `flink.streaming.api.checkpoint.CheckpointComitter` interface. +Some operators might need the information when a checkpoint is fully acknowledged by Flink to communicate that with the outside world. In this case see the `flink.streaming.api.checkpoint.CheckpointNotifier` interface. ### Checkpointed interface diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java index 619709260b0f0..258bee839aad6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java @@ -22,28 +22,22 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.state.OperatorState; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.Collector; -import org.junit.AfterClass; import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import java.util.Random; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * A simple test that runs a streaming topology with checkpointing enabled. This differs from @@ -64,44 +58,9 @@ * state reflects the "exactly once" semantics. */ @SuppressWarnings("serial") -public class CoStreamCheckpointingITCase { - - private static final int NUM_TASK_MANAGERS = 2; - private static final int NUM_TASK_SLOTS = 3; - private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; - - private static ForkableFlinkMiniCluster cluster; - - @BeforeClass - public static void startCluster() { - try { - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); - config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - - cluster = new ForkableFlinkMiniCluster(config, false); - } - catch (Exception e) { - e.printStackTrace(); - fail("Failed to start test cluster: " + e.getMessage()); - } - } - - @AfterClass - public static void shutdownCluster() { - try { - cluster.shutdown(); - cluster = null; - } - catch (Exception e) { - e.printStackTrace(); - fail("Failed to stop test cluster: " + e.getMessage()); - } - } - +public class CoStreamCheckpointingITCase extends StreamFaultToleranceTestBase { + final long NUM_STRINGS = 10_000_000L; /** * Runs the following program: @@ -110,102 +69,76 @@ public static void shutdownCluster() { * [ (source)->(filter)->(map) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ] * */ - @Test - public void runCheckpointedProgram() { + @Override + public void testProgram(StreamExecutionEnvironment env) { - final long NUM_STRINGS = 10000000L; assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( - "localhost", cluster.getJobManagerRPCPort()); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(500); - env.getConfig().disableSysoutLogging(); - - DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); - - stream - // -------------- first vertex, chained to the source ---------------- - .filter(new StringRichFilterFunction()) - - // -------------- second vertex - the stateful one that also fails ---------------- - .connect(stream).flatMap(new LeftIdentityCoRichFlatMapFunction()) - - // -------------- third vertex - the stateful one that also fails ---------------- - .map(new StringPrefixCountRichMapFunction()) - .startNewChain() - .map(new StatefulCounterFunction()) - - // -------------- fourth vertex - reducer and the sink ---------------- - .groupBy("prefix") - .reduce(new OnceFailingReducer(NUM_STRINGS)) - .addSink(new RichSinkFunction() { - - private Map counts = new HashMap(); - - @Override - public void invoke(PrefixCount value) { - Character first = value.prefix.charAt(0); - Long previous = counts.get(first); - if (previous == null) { - counts.put(first, value.count); - } else { - counts.put(first, Math.max(previous, value.count)); - } - } - -// @Override -// public void close() { -// for (Long count : counts.values()) { -// assertEquals(NUM_STRINGS / 40, count.longValue()); -// } -// } - }); - - env.execute(); - - long filterSum = 0; - for (long l : StringRichFilterFunction.counts) { - filterSum += l; - } + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); - long coMapSum = 0; - for (long l : LeftIdentityCoRichFlatMapFunction.counts) { - coMapSum += l; - } + stream + // -------------- first vertex, chained to the source ---------------- + .filter(new StringRichFilterFunction()) - long mapSum = 0; - for (long l : StringPrefixCountRichMapFunction.counts) { - mapSum += l; - } + // -------------- second vertex - the stateful one that also fails ---------------- + .connect(stream).flatMap(new LeftIdentityCoRichFlatMapFunction()) - long countSum = 0; - for (long l : StatefulCounterFunction.counts) { - countSum += l; - } + // -------------- third vertex - the stateful one that also fails ---------------- + .map(new StringPrefixCountRichMapFunction()) + .startNewChain() + .map(new StatefulCounterFunction()) - if (!StringPrefixCountRichMapFunction.restoreCalledAtLeastOnce) { - Assert.fail("Restore was never called on counting Map function."); - } + // -------------- fourth vertex - reducer and the sink ---------------- + .groupBy("prefix") + .reduce(new OnceFailingReducer(NUM_STRINGS)) + .addSink(new SinkFunction() { - if (!LeftIdentityCoRichFlatMapFunction.restoreCalledAtLeastOnce) { - Assert.fail("Restore was never called on counting CoMap function."); - } + @Override + public void invoke(PrefixCount value) throws Exception { + // Do nothing here + } + }); + } + + @Override + public void postSubmit() { + long filterSum = 0; + for (long l : StringRichFilterFunction.counts) { + filterSum += l; + } - // verify that we counted exactly right + long coMapSum = 0; + for (long l : LeftIdentityCoRichFlatMapFunction.counts) { + coMapSum += l; + } + + long mapSum = 0; + for (long l : StringPrefixCountRichMapFunction.counts) { + mapSum += l; + } - assertEquals(NUM_STRINGS, filterSum); - assertEquals(NUM_STRINGS, coMapSum); - assertEquals(NUM_STRINGS, mapSum); - assertEquals(NUM_STRINGS, countSum); + long countSum = 0; + for (long l : StatefulCounterFunction.counts) { + countSum += l; } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + + if (!StringPrefixCountRichMapFunction.restoreCalledAtLeastOnce) { + Assert.fail("Restore was never called on counting Map function."); + } + + if (!LeftIdentityCoRichFlatMapFunction.restoreCalledAtLeastOnce) { + Assert.fail("Restore was never called on counting CoMap function."); } + + // verify that we counted exactly right + + assertEquals(NUM_STRINGS, filterSum); + assertEquals(NUM_STRINGS, coMapSum); + assertEquals(NUM_STRINGS, mapSum); + assertEquals(NUM_STRINGS, countSum); } + // -------------------------------------------------------------------------------------------- // Custom Functions // -------------------------------------------------------------------------------------------- @@ -260,7 +193,6 @@ public void run(SourceContext ctx) throws Exception { synchronized (lockingObject) { index.update(index.value() + step); -// System.out.println("SOURCE EMIT: " + result); ctx.collect(result); } } @@ -341,30 +273,6 @@ public PrefixCount reduce(PrefixCount value1, PrefixCount value2) throws Excepti } } - // -------------------------------------------------------------------------------------------- - // Custom Type Classes - // -------------------------------------------------------------------------------------------- - - public static class PrefixCount { - - public String prefix; - public String value; - public long count; - - public PrefixCount() {} - - public PrefixCount(String prefix, String value, long count) { - this.prefix = prefix; - this.value = value; - this.count = count; - } - - @Override - public String toString() { - return prefix + " / " + value; - } - } - private static class StringRichFilterFunction extends RichFilterFunction implements Checkpointed { Long count = 0L; @@ -434,7 +342,6 @@ private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunc @Override public void flatMap1(String value, Collector out) throws IOException { count += 1; -// System.out.println("Co-Map COUNT: " + count); out.collect(value); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java index 88361e28cceea..d942a9e75df74 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java @@ -52,82 +52,36 @@ * It is designed to check partitioned states. */ @SuppressWarnings("serial") -public class PartitionedStateCheckpointingITCase { - - private static final int NUM_TASK_MANAGERS = 2; - private static final int NUM_TASK_SLOTS = 3; - private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; - - private static ForkableFlinkMiniCluster cluster; - - @BeforeClass - public static void startCluster() { - try { - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); - config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - - cluster = new ForkableFlinkMiniCluster(config, false); - } catch (Exception e) { - e.printStackTrace(); - fail("Failed to start test cluster: " + e.getMessage()); - } - } - - @AfterClass - public static void shutdownCluster() { - try { - cluster.shutdown(); - cluster = null; - } catch (Exception e) { - e.printStackTrace(); - fail("Failed to stop test cluster: " + e.getMessage()); - } - } +public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTestBase { - @SuppressWarnings("unchecked") - @Test - public void runCheckpointedProgram() { + final long NUM_STRINGS = 10_000_000L; - final long NUM_STRINGS = 10000000L; + @Override + public void testProgram(StreamExecutionEnvironment env) { assertTrue("Broken test setup", (NUM_STRINGS/2) % 40 == 0); - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", - cluster.getJobManagerRPCPort()); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(500); - env.getConfig().disableSysoutLogging(); - - DataStream stream1 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2)); - DataStream stream2 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2)); + DataStream stream1 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2)); + DataStream stream2 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2)); - stream1.union(stream2) - .groupBy(new IdentityKeySelector()) - .map(new OnceFailingPartitionedSum(NUM_STRINGS)) - .keyBy(0) - .addSink(new CounterSink()); - - env.execute(); + stream1.union(stream2) + .groupBy(new IdentityKeySelector()) + .map(new OnceFailingPartitionedSum(NUM_STRINGS)) + .keyBy(0) + .addSink(new CounterSink()); + } - // verify that we counted exactly right - for (Entry sum : OnceFailingPartitionedSum.allSums.entrySet()) { - assertEquals(new Long(sum.getKey() * NUM_STRINGS / 40), sum.getValue()); - } - System.out.println("new"); - for (Long count : CounterSink.allCounts.values()) { - assertEquals(new Long(NUM_STRINGS / 40), count); - } - - assertEquals(40, CounterSink.allCounts.size()); - assertEquals(40, OnceFailingPartitionedSum.allSums.size()); - - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + @Override + public void postSubmit() { + // verify that we counted exactly right + for (Entry sum : OnceFailingPartitionedSum.allSums.entrySet()) { + assertEquals(new Long(sum.getKey() * NUM_STRINGS / 40), sum.getValue()); + } + for (Long count : CounterSink.allCounts.values()) { + assertEquals(new Long(NUM_STRINGS / 40), count); } + + assertEquals(40, CounterSink.allCounts.size()); + assertEquals(40, OnceFailingPartitionedSum.allSums.size()); } // -------------------------------------------------------------------------------------------- diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java index 39ff2e512d034..072086b09224b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java @@ -49,48 +49,15 @@ /** * A simple test that runs a streaming topology with checkpointing enabled. - * + * * The test triggers a failure after a while and verifies that, after completion, the - * state reflects the "exactly once" semantics. + * state defined with either the {@link OperatorState} or the {@link Checkpointed} + * interface reflects the "exactly once" semantics. */ @SuppressWarnings("serial") -public class StateCheckpoinedITCase { - - private static final int NUM_TASK_MANAGERS = 2; - private static final int NUM_TASK_SLOTS = 3; - private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; - - private static ForkableFlinkMiniCluster cluster; - - @BeforeClass - public static void startCluster() { - try { - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); - config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - - cluster = new ForkableFlinkMiniCluster(config, false); - } - catch (Exception e) { - e.printStackTrace(); - fail("Failed to start test cluster: " + e.getMessage()); - } - } - - @AfterClass - public static void shutdownCluster() { - try { - cluster.shutdown(); - cluster = null; - } - catch (Exception e) { - e.printStackTrace(); - fail("Failed to stop test cluster: " + e.getMessage()); - } - } +public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase { + final long NUM_STRINGS = 10_000_000L; /** * Runs the following program: @@ -99,69 +66,56 @@ public static void shutdownCluster() { * [ (source)->(filter)->(map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ] * */ - @Test - public void runCheckpointedProgram() { - - final long NUM_STRINGS = 10000000L; + @Override + public void testProgram(StreamExecutionEnvironment env) { assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); - - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( - "localhost", cluster.getJobManagerRPCPort()); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(500); - env.getConfig().disableSysoutLogging(); - - DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); - - stream - // -------------- first vertex, chained to the source ---------------- - .filter(new StringRichFilterFunction()) - // -------------- seconds vertex - one-to-one connected ---------------- - .map(new StringPrefixCountRichMapFunction()) - .startNewChain() - .map(new StatefulCounterFunction()) + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); - // -------------- third vertex - reducer and the sink ---------------- - .partitionByHash("prefix") - .flatMap(new OnceFailingAggregator(NUM_STRINGS)) - .addSink(new ValidatingSink()); + stream + // -------------- first vertex, chained to the source ---------------- + .filter(new StringRichFilterFunction()) - env.execute(); - - long filterSum = 0; - for (long l : StringRichFilterFunction.counts) { - filterSum += l; - } + // -------------- seconds vertex - one-to-one connected ---------------- + .map(new StringPrefixCountRichMapFunction()) + .startNewChain() + .map(new StatefulCounterFunction()) - long mapSum = 0; - for (long l : StringPrefixCountRichMapFunction.counts) { - mapSum += l; - } + // -------------- third vertex - reducer and the sink ---------------- + .partitionByHash("prefix") + .flatMap(new OnceFailingAggregator(NUM_STRINGS)) + .addSink(new ValidatingSink()); + } - long countSum = 0; - for (long l : StatefulCounterFunction.counts) { - countSum += l; - } + @Override + public void postSubmit() { + long filterSum = 0; + for (long l : StringRichFilterFunction.counts) { + filterSum += l; + } + + long mapSum = 0; + for (long l : StringPrefixCountRichMapFunction.counts) { + mapSum += l; + } - // verify that we counted exactly right - assertEquals(NUM_STRINGS, filterSum); - assertEquals(NUM_STRINGS, mapSum); - assertEquals(NUM_STRINGS, countSum); + long countSum = 0; + for (long l : StatefulCounterFunction.counts) { + countSum += l; + } - for (Map map : ValidatingSink.maps) { - for (Long count : map.values()) { - assertEquals(NUM_STRINGS / 40, count.longValue()); - } + // verify that we counted exactly right + assertEquals(NUM_STRINGS, filterSum); + assertEquals(NUM_STRINGS, mapSum); + assertEquals(NUM_STRINGS, countSum); + + for (Map map : ValidatingSink.maps) { + for (Long count : map.values()) { + assertEquals(NUM_STRINGS / 40, count.longValue()); } } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } } - + // -------------------------------------------------------------------------------------------- // Custom Functions // -------------------------------------------------------------------------------------------- @@ -406,28 +360,4 @@ public void restoreState(HashMap state) { counts.putAll(state); } } - - // -------------------------------------------------------------------------------------------- - // Custom Type Classes - // -------------------------------------------------------------------------------------------- - - public static class PrefixCount implements Serializable { - - public String prefix; - public String value; - public long count; - - public PrefixCount() {} - - public PrefixCount(String prefix, String value, long count) { - this.prefix = prefix; - this.value = value; - this.count = count; - } - - @Override - public String toString() { - return prefix + " / " + value; - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java new file mode 100644 index 0000000000000..4c1ac4ea0e0f3 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java @@ -0,0 +1,341 @@ +/* + * 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.test.checkpointing; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.functions.RichReduceFunction; +import org.apache.flink.api.common.state.OperatorState; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Random; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Integration test for the {@link CheckpointNotifier} interface. The test ensures that + * {@link CheckpointNotifier#notifyCheckpointComplete(long)} is called for some completed + * checkpoints, that it is called at most once for any checkpoint id and that it is not + * called for a deliberately failed checkpoint. + * + *

    + * Note that as a result of doing the checks on the task level there is no way to verify + * that the {@link CheckpointNotifier#notifyCheckpointComplete(long)} is called for every + * successfully completed checkpoint. + */ +@SuppressWarnings("serial") +public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase { + + final long NUM_STRINGS = 10_000_000L; + + /** + * Runs the following program: + * + *

    +	 *     [ (source)->(filter)->(map) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
    +	 * 
    + */ + @Override + public void testProgram(StreamExecutionEnvironment env) { + + assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); + + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); + + stream + // -------------- first vertex, chained to the src ---------------- + .filter(new StringRichFilterFunction()) + + // -------------- second vertex, applying the co-map ---------------- + .connect(stream).flatMap(new LeftIdentityCoRichFlatMapFunction()) + + // -------------- third vertex - the stateful one that also fails ---------------- + .map(new StringPrefixCountRichMapFunction()) + .startNewChain() + .map(new IdentityMapFunction()) + + // -------------- fourth vertex - reducer and the sink ---------------- + .groupBy("prefix") + .reduce(new OnceFailingReducer(NUM_STRINGS)) + .addSink(new SinkFunction() { + @Override + public void invoke(PrefixCount value) { + // do nothing + } + }); + } + + @Override + public void postSubmit() { + List[][] checkList = new List[][]{ StringGeneratingSourceFunction.completedCheckpoints, + IdentityMapFunction.completedCheckpoints, + StringPrefixCountRichMapFunction.completedCheckpoints, + LeftIdentityCoRichFlatMapFunction.completedCheckpoints}; + + for(List[] parallelNotifications : checkList) { + for (int i = 0; i < PARALLELISM; i++){ + List notifications = parallelNotifications[i]; + assertTrue("No checkpoint notification was received.", + notifications.size() > 0); + assertFalse("Failure checkpoint was marked as completed.", + notifications.contains(OnceFailingReducer.failureCheckpointID)); + assertTrue("Checkpoint notification was received multiple times", + notifications.size() == new HashSet(notifications).size()); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Custom Functions + // -------------------------------------------------------------------------------------------- + + private static class StringGeneratingSourceFunction extends RichSourceFunction + implements ParallelSourceFunction, CheckpointNotifier { + + // operator life cycle + private volatile boolean isRunning; + + // operator behaviour + private final long numElements; + private Random rnd; + + private StringBuilder stringBuilder; + private OperatorState index; + private int step; + + // test behaviour + private int subtaskId; + public static List[] completedCheckpoints = new List[PARALLELISM]; + + StringGeneratingSourceFunction(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) throws IOException { + rnd = new Random(); + stringBuilder = new StringBuilder(); + step = getRuntimeContext().getNumberOfParallelSubtasks(); + subtaskId = getRuntimeContext().getIndexOfThisSubtask(); + index = getRuntimeContext().getOperatorState("index", subtaskId, false); + + // Create a collection on the first open + if (completedCheckpoints[subtaskId] == null) { + completedCheckpoints[subtaskId] = new ArrayList(); + } + + isRunning = true; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final Object lockingObject = ctx.getCheckpointLock(); + + while (isRunning && index.value() < numElements) { + char first = (char) ((index.value() % 40) + 40); + + stringBuilder.setLength(0); + stringBuilder.append(first); + + String result = randomString(stringBuilder, rnd); + + synchronized (lockingObject) { + index.update(index.value() + step); + ctx.collect(result); + } + } + } + + @Override + public void cancel() { + isRunning = false; + } + + private static String randomString(StringBuilder bld, Random rnd) { + final int len = rnd.nextInt(10) + 5; + + for (int i = 0; i < len; i++) { + char next = (char) (rnd.nextInt(20000) + 33); + bld.append(next); + } + + return bld.toString(); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + completedCheckpoints[subtaskId].add(checkpointId); + } + } + + private static class IdentityMapFunction extends RichMapFunction + implements CheckpointNotifier { + + public static List[] completedCheckpoints = new List[PARALLELISM]; + private int subtaskId; + + @Override + public PrefixCount map(PrefixCount value) throws Exception { + return value; + } + + @Override + public void open(Configuration conf) throws IOException { + subtaskId = getRuntimeContext().getIndexOfThisSubtask(); + + // Create a collection on the first open + if (completedCheckpoints[subtaskId] == null) { + completedCheckpoints[subtaskId] = new ArrayList(); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + completedCheckpoints[subtaskId].add(checkpointId); + } + } + + private static class OnceFailingReducer extends RichReduceFunction implements Checkpointed{ + + private static volatile boolean hasFailed = false; + public static volatile long failureCheckpointID; + + private final long numElements; + + private long failurePos; + private long count; + + + OnceFailingReducer(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) { + long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); + long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); + + failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; + count = 0; + } + + @Override + public PrefixCount reduce(PrefixCount value1, PrefixCount value2) throws Exception { + count++; + value1.count += value2.count; + return value1; + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + if (!hasFailed && count >= failurePos) { + hasFailed = true; + failureCheckpointID = checkpointId; + throw new Exception("Test Failure"); + } + return count; + } + + @Override + public void restoreState(Long state) { + count = state; + } + } + + private static class StringRichFilterFunction implements FilterFunction { + @Override + public boolean filter(String value) { + return value.length() < 100; + } + } + + private static class StringPrefixCountRichMapFunction extends RichMapFunction + implements CheckpointNotifier { + + public static List[] completedCheckpoints = new List[PARALLELISM]; + private int subtaskId; + + @Override + public void open(Configuration conf) throws IOException { + subtaskId = getRuntimeContext().getIndexOfThisSubtask(); + + // Create a collection on the first open + if (completedCheckpoints[subtaskId] == null) { + completedCheckpoints[subtaskId] = new ArrayList(); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + completedCheckpoints[subtaskId].add(checkpointId); + } + + @Override + public PrefixCount map(String value) throws IOException { + return new PrefixCount(value.substring(0, 1), value, 1L); + } + } + + private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction + implements CheckpointNotifier { + + public static List[] completedCheckpoints = new List[PARALLELISM]; + private int subtaskId; + + @Override + public void open(Configuration conf) throws IOException { + subtaskId = getRuntimeContext().getIndexOfThisSubtask(); + + // Create a collection on the first open + if (completedCheckpoints[subtaskId] == null) { + completedCheckpoints[subtaskId] = new ArrayList(); + } + } + + @Override + public void flatMap1(String value, Collector out) throws IOException { + out.collect(value); + } + + @Override + public void flatMap2(String value, Collector out) throws IOException { + // we ignore the values from the second input + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + completedCheckpoints[subtaskId].add(checkpointId); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java index 93dda5f684d42..d54d4254839dd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java @@ -18,21 +18,9 @@ package org.apache.flink.test.checkpointing; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; - import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.OperatorState; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -40,56 +28,25 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import java.io.IOException; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** * A simple test that runs a streaming topology with checkpointing enabled. * * The test triggers a failure after a while and verifies that, after completion, the - * state reflects the "exactly once" semantics. + * state defined with the {@link Checkpointed} interface reflects the "exactly once" semantics. */ @SuppressWarnings("serial") -public class StreamCheckpointingITCase { - - private static final int NUM_TASK_MANAGERS = 2; - private static final int NUM_TASK_SLOTS = 3; - private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; - - private static ForkableFlinkMiniCluster cluster; - - @BeforeClass - public static void startCluster() { - try { - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); - config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - - cluster = new ForkableFlinkMiniCluster(config, false); - } - catch (Exception e) { - e.printStackTrace(); - fail("Failed to start test cluster: " + e.getMessage()); - } - } - - @AfterClass - public static void shutdownCluster() { - try { - cluster.shutdown(); - cluster = null; - } - catch (Exception e) { - e.printStackTrace(); - fail("Failed to stop test cluster: " + e.getMessage()); - } - } +public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase { + final long NUM_STRINGS = 10_000_000L; /** * Runs the following program: @@ -98,78 +55,63 @@ public static void shutdownCluster() { * [ (source)->(filter) ]-s->[ (map) ] -> [ (map) ] -> [ (groupBy/count)->(sink) ] * */ - @Test - public void runCheckpointedProgram() { + @Override + public void testProgram(StreamExecutionEnvironment env) { + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); + + stream + // -------------- first vertex, chained to the source ---------------- + .filter(new StringRichFilterFunction()).shuffle() + + // -------------- seconds vertex - the stateful one that also fails ---------------- + .map(new StringPrefixCountRichMapFunction()) + .startNewChain() + .map(new StatefulCounterFunction()) + + // -------------- third vertex - counter and the sink ---------------- + .groupBy("prefix") + .map(new OnceFailingPrefixCounter(NUM_STRINGS)) + .addSink(new SinkFunction() { + + @Override + public void invoke(PrefixCount value) throws Exception { + // Do nothing here + } + }); + } - final long NUM_STRINGS = 10000000L; - assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); - - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( - "localhost", cluster.getJobManagerRPCPort()); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(500); - env.getConfig().disableSysoutLogging(); - - DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); - - stream - // -------------- first vertex, chained to the source ---------------- - .filter(new StringRichFilterFunction()).shuffle() - - // -------------- seconds vertex - the stateful one that also fails ---------------- - .map(new StringPrefixCountRichMapFunction()) - .startNewChain() - .map(new StatefulCounterFunction()) - - // -------------- third vertex - counter and the sink ---------------- - .groupBy("prefix") - .map(new OnceFailingPrefixCounter(NUM_STRINGS)) - .addSink(new SinkFunction() { - - @Override - public void invoke(PrefixCount value) throws Exception { - // Do nothing here - } - }); - - env.execute(); - - long filterSum = 0; - for (long l : StringRichFilterFunction.counts) { - filterSum += l; - } + @Override + public void postSubmit() { + long filterSum = 0; + for (long l : StringRichFilterFunction.counts) { + filterSum += l; + } - long mapSum = 0; - for (long l : StringPrefixCountRichMapFunction.counts) { - mapSum += l; - } + long mapSum = 0; + for (long l : StringPrefixCountRichMapFunction.counts) { + mapSum += l; + } - long countSum = 0; - for (long l : StatefulCounterFunction.counts) { - countSum += l; - } - - long reduceInputCount = 0; - for(long l: OnceFailingPrefixCounter.counts){ - reduceInputCount += l; - } - - assertEquals(NUM_STRINGS, filterSum); - assertEquals(NUM_STRINGS, mapSum); - assertEquals(NUM_STRINGS, countSum); - assertEquals(NUM_STRINGS, reduceInputCount); - // verify that we counted exactly right - for (Long count : OnceFailingPrefixCounter.prefixCounts.values()) { - assertEquals(new Long(NUM_STRINGS / 40), count); - } + long countSum = 0; + for (long l : StatefulCounterFunction.counts) { + countSum += l; } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + + long reduceInputCount = 0; + for(long l: OnceFailingPrefixCounter.counts){ + reduceInputCount += l; + } + + assertEquals(NUM_STRINGS, filterSum); + assertEquals(NUM_STRINGS, mapSum); + assertEquals(NUM_STRINGS, countSum); + assertEquals(NUM_STRINGS, reduceInputCount); + // verify that we counted exactly right + for (Long count : OnceFailingPrefixCounter.prefixCounts.values()) { + assertEquals(new Long(NUM_STRINGS / 40), count); } } - + // -------------------------------------------------------------------------------------------- // Custom Functions // -------------------------------------------------------------------------------------------- @@ -246,27 +188,31 @@ private static String randomString(StringBuilder bld, Random rnd) { } } - private static class StatefulCounterFunction extends RichMapFunction { + private static class StatefulCounterFunction extends RichMapFunction implements Checkpointed { - private OperatorState count; + private long count; static final long[] counts = new long[PARALLELISM]; @Override public PrefixCount map(PrefixCount value) throws Exception { - count.update(count.value() + 1); + count++; return value; } @Override - public void open(Configuration conf) throws IOException { - count = getRuntimeContext().getOperatorState("count", 0L, false); + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count; } @Override - public void close() throws IOException { - counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value(); + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return count; + } + + @Override + public void restoreState(Long state) { + count = state; } - } private static class OnceFailingPrefixCounter extends RichMapFunction { @@ -320,30 +266,6 @@ public PrefixCount map(PrefixCount value) throws Exception { return value; } } - - // -------------------------------------------------------------------------------------------- - // Custom Type Classes - // -------------------------------------------------------------------------------------------- - - public static class PrefixCount implements Serializable { - - public String prefix; - public String value; - public long count; - - public PrefixCount() {} - - public PrefixCount(String prefix, String value, long count) { - this.prefix = prefix; - this.value = value; - this.count = count; - } - - @Override - public String toString() { - return prefix + " / " + value; - } - } private static class StringRichFilterFunction extends RichFilterFunction implements Checkpointed { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java new file mode 100644 index 0000000000000..29933156d7d03 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -0,0 +1,150 @@ +/* + * 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.test.checkpointing; + + +import org.apache.flink.api.common.functions.RichFilterFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.state.OperatorState; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + + +/** + * Test base for fault tolerant streaming programs + */ +@SuppressWarnings("serial") +public abstract class StreamFaultToleranceTestBase { + + protected static final int NUM_TASK_MANAGERS = 2; + protected static final int NUM_TASK_SLOTS = 3; + protected static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + + private static ForkableFlinkMiniCluster cluster; + + @BeforeClass + public static void startCluster() { + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); + config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + + cluster = new ForkableFlinkMiniCluster(config, false); + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to start test cluster: " + e.getMessage()); + } + } + + @AfterClass + public static void shutdownCluster() { + try { + cluster.shutdown(); + cluster = null; + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to stop test cluster: " + e.getMessage()); + } + } + + /** + * Implementations are expected to assemble the test topology in this function + * using the provided {@link StreamExecutionEnvironment}. + */ + abstract public void testProgram(StreamExecutionEnvironment env); + + /** + * Implementations are expected to provide test here to verify the correct behavior. + */ + abstract public void postSubmit(); + + /** + * Runs the following program the test program defined in {@link #testProgram(StreamExecutionEnvironment)} + * followed by the checks in {@link #postSubmit}. + */ + @Test + public void runCheckpointedProgram() { + try { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(500); + env.getConfig().disableSysoutLogging(); + + testProgram(env); + + env.execute(); + + postSubmit(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + // Frequently used utilities + // -------------------------------------------------------------------------------------------- + + public static class PrefixCount implements Serializable { + + public String prefix; + public String value; + public long count; + + public PrefixCount() {} + + public PrefixCount(String prefix, String value, long count) { + this.prefix = prefix; + this.value = value; + this.count = count; + } + + @Override + public String toString() { + return prefix + " / " + value; + } + } + +} From 10ce2e2d033ddafb326c3b9b81e19e669c19df96 Mon Sep 17 00:00:00 2001 From: mbalassi Date: Sat, 8 Aug 2015 22:02:58 +0200 Subject: [PATCH 111/175] [FLINK-2423] [streaming] ITCase for checkpoint notifications Closes #980 --- .../StreamCheckpointNotifierITCase.java | 135 +++++++++--------- 1 file changed, 70 insertions(+), 65 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java index 4c1ac4ea0e0f3..de8ee9dd5d7de 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java @@ -18,10 +18,11 @@ package org.apache.flink.test.checkpointing; -import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.state.OperatorState; +import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -31,6 +32,8 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.util.Collector; import java.io.IOException; @@ -41,7 +44,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * Integration test for the {@link CheckpointNotifier} interface. The test ensures that @@ -50,6 +52,10 @@ * called for a deliberately failed checkpoint. * *

    + * The topology tested here includes a number of {@link OneInputStreamOperator}s and a + * {@link TwoInputStreamOperator}. + * + *

    * Note that as a result of doing the checks on the task level there is no way to verify * that the {@link CheckpointNotifier#notifyCheckpointComplete(long)} is called for every * successfully completed checkpoint. @@ -57,40 +63,37 @@ @SuppressWarnings("serial") public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase { - final long NUM_STRINGS = 10_000_000L; + final long NUM_LONGS = 10_000_000L; /** * Runs the following program: * *

    -	 *     [ (source)->(filter)->(map) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
    +	 *     [ (source)->(filter) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
     	 * 
    */ @Override public void testProgram(StreamExecutionEnvironment env) { - assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); - - DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); + DataStream stream = env.addSource(new GeneratingSourceFunction(NUM_LONGS)); stream // -------------- first vertex, chained to the src ---------------- - .filter(new StringRichFilterFunction()) + .filter(new LongRichFilterFunction()) - // -------------- second vertex, applying the co-map ---------------- + // -------------- second vertex, applying the co-map ---------------- .connect(stream).flatMap(new LeftIdentityCoRichFlatMapFunction()) // -------------- third vertex - the stateful one that also fails ---------------- - .map(new StringPrefixCountRichMapFunction()) - .startNewChain() .map(new IdentityMapFunction()) + .startNewChain() - // -------------- fourth vertex - reducer and the sink ---------------- - .groupBy("prefix") - .reduce(new OnceFailingReducer(NUM_STRINGS)) - .addSink(new SinkFunction() { + // -------------- fourth vertex - reducer and the sink ---------------- + .groupBy(0) + .reduce(new OnceFailingReducer(NUM_LONGS)) + .addSink(new SinkFunction>() { @Override - public void invoke(PrefixCount value) { + public void invoke(Tuple1 value) { // do nothing } }); @@ -98,18 +101,24 @@ public void invoke(PrefixCount value) { @Override public void postSubmit() { - List[][] checkList = new List[][]{ StringGeneratingSourceFunction.completedCheckpoints, + List[][] checkList = new List[][]{ GeneratingSourceFunction.completedCheckpoints, IdentityMapFunction.completedCheckpoints, - StringPrefixCountRichMapFunction.completedCheckpoints, + LongRichFilterFunction.completedCheckpoints, LeftIdentityCoRichFlatMapFunction.completedCheckpoints}; + long failureCheckpointID = OnceFailingReducer.failureCheckpointID; + for(List[] parallelNotifications : checkList) { for (int i = 0; i < PARALLELISM; i++){ List notifications = parallelNotifications[i]; assertTrue("No checkpoint notification was received.", notifications.size() > 0); assertFalse("Failure checkpoint was marked as completed.", - notifications.contains(OnceFailingReducer.failureCheckpointID)); + notifications.contains(failureCheckpointID)); + assertFalse("No checkpoint received before failure.", + notifications.get(0) == failureCheckpointID); + assertFalse("No checkpoint received after failure.", + notifications.get(notifications.size() - 1) == failureCheckpointID); assertTrue("Checkpoint notification was received multiple times", notifications.size() == new HashSet(notifications).size()); } @@ -120,17 +129,20 @@ public void postSubmit() { // Custom Functions // -------------------------------------------------------------------------------------------- - private static class StringGeneratingSourceFunction extends RichSourceFunction - implements ParallelSourceFunction, CheckpointNotifier { + /** + * Generates some Long values and as an implementation for the {@link CheckpointNotifier} + * interface it stores all the checkpoint ids it has seen in a static list. + */ + private static class GeneratingSourceFunction extends RichSourceFunction + implements ParallelSourceFunction, CheckpointNotifier { // operator life cycle private volatile boolean isRunning; // operator behaviour private final long numElements; - private Random rnd; + private long result; - private StringBuilder stringBuilder; private OperatorState index; private int step; @@ -138,14 +150,12 @@ private static class StringGeneratingSourceFunction extends RichSourceFunction ctx) throws Exception { + public void run(SourceContext ctx) throws Exception { final Object lockingObject = ctx.getCheckpointLock(); while (isRunning && index.value() < numElements) { - char first = (char) ((index.value() % 40) + 40); - stringBuilder.setLength(0); - stringBuilder.append(first); - - String result = randomString(stringBuilder, rnd); + result = index.value() % 10; synchronized (lockingObject) { index.update(index.value() + step); @@ -182,32 +188,25 @@ public void cancel() { isRunning = false; } - private static String randomString(StringBuilder bld, Random rnd) { - final int len = rnd.nextInt(10) + 5; - - for (int i = 0; i < len; i++) { - char next = (char) (rnd.nextInt(20000) + 33); - bld.append(next); - } - - return bld.toString(); - } - @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { completedCheckpoints[subtaskId].add(checkpointId); } } - private static class IdentityMapFunction extends RichMapFunction + /** + * Identity transform on Long values wrapping the output in a tuple. As an implementation + * for the {@link CheckpointNotifier} interface it stores all the checkpoint ids it has seen in a static list. + */ + private static class IdentityMapFunction extends RichMapFunction> implements CheckpointNotifier { public static List[] completedCheckpoints = new List[PARALLELISM]; private int subtaskId; @Override - public PrefixCount map(PrefixCount value) throws Exception { - return value; + public Tuple1 map(Long value) throws Exception { + return Tuple1.of(value); } @Override @@ -226,7 +225,10 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { } } - private static class OnceFailingReducer extends RichReduceFunction implements Checkpointed{ + /** + * Reducer that causes one failure between seeing 40% to 70% of the records. + */ + private static class OnceFailingReducer extends RichReduceFunction> implements Checkpointed { private static volatile boolean hasFailed = false; public static volatile long failureCheckpointID; @@ -251,9 +253,9 @@ public void open(Configuration parameters) { } @Override - public PrefixCount reduce(PrefixCount value1, PrefixCount value2) throws Exception { + public Tuple1 reduce(Tuple1 value1, Tuple1 value2) throws Exception { count++; - value1.count += value2.count; + value1.f0 += value2.f0; return value1; } @@ -273,19 +275,22 @@ public void restoreState(Long state) { } } - private static class StringRichFilterFunction implements FilterFunction { - @Override - public boolean filter(String value) { - return value.length() < 100; - } - } - - private static class StringPrefixCountRichMapFunction extends RichMapFunction + /** + * Filter on Long values supposedly letting all values through. As an implementation + * for the {@link CheckpointNotifier} interface it stores all the checkpoint ids + * it has seen in a static list. + */ + private static class LongRichFilterFunction extends RichFilterFunction implements CheckpointNotifier { public static List[] completedCheckpoints = new List[PARALLELISM]; private int subtaskId; + @Override + public boolean filter(Long value) { + return value < 100; + } + @Override public void open(Configuration conf) throws IOException { subtaskId = getRuntimeContext().getIndexOfThisSubtask(); @@ -300,14 +305,14 @@ public void open(Configuration conf) throws IOException { public void notifyCheckpointComplete(long checkpointId) throws Exception { completedCheckpoints[subtaskId].add(checkpointId); } - - @Override - public PrefixCount map(String value) throws IOException { - return new PrefixCount(value.substring(0, 1), value, 1L); - } } - private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction + /** + * CoFlatMap on Long values as identity transform on the left input, while ignoring the right. + * As an implementation for the {@link CheckpointNotifier} interface it stores all the checkpoint + * ids it has seen in a static list. + */ + private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction implements CheckpointNotifier { public static List[] completedCheckpoints = new List[PARALLELISM]; @@ -324,12 +329,12 @@ public void open(Configuration conf) throws IOException { } @Override - public void flatMap1(String value, Collector out) throws IOException { + public void flatMap1(Long value, Collector out) throws IOException { out.collect(value); } @Override - public void flatMap2(String value, Collector out) throws IOException { + public void flatMap2(Long value, Collector out) throws IOException { // we ignore the values from the second input } From d8d074809e72ece8593ae4d3d17b9ec4febb1fb2 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Tue, 11 Aug 2015 22:40:19 +0200 Subject: [PATCH 112/175] [FLINK-2511] [runtime] Make sure to close FileInputStream in FlinkZooKeeperQuorumPeer --- .../flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java index c9d3ec4be1949..a0ce88cb7ae38 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java @@ -79,8 +79,9 @@ public static void runFlinkZkQuorumPeer(String zkConfigFile, int peerId) throws Properties zkProps = new Properties(); - InputStream inStream = new FileInputStream(new File(zkConfigFile)); - zkProps.load(inStream); + try (InputStream inStream = new FileInputStream(new File(zkConfigFile))) { + zkProps.load(inStream); + } LOG.info("Configuration: " + zkProps); From eeec1912b478ed43a045449d82e0a2fd3700d720 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 11 Aug 2015 16:07:22 +0200 Subject: [PATCH 113/175] [FLINK-2509] [runtime] Add class loader info into the exception message when user code classes are not found. This closes #1008 --- .../flink/runtime/util/ClassLoaderUtil.java | 126 +++++++++++++++++ .../runtime/util/ClassLoaderUtilsTest.java | 131 ++++++++++++++++++ .../streaming/api/graph/StreamConfig.java | 23 ++- 3 files changed, 276 insertions(+), 4 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java new file mode 100644 index 0000000000000..fbb707ee9b99e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java @@ -0,0 +1,126 @@ +/* + * 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.runtime.util; + +import org.apache.flink.util.ExceptionUtils; + +import java.io.File; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.jar.JarFile; + +/** + * Utilities for information with respect to class loaders, specifically class loaders for + * the dynamic loading of user defined classes. + */ +public class ClassLoaderUtil { + + /** + * Gets information about URL class loaders. The returned info string contains all URLs of the + * class loader. For file URLs, it contains in addition whether the referenced file exists, + * is a valid JAR file, or is a directory. + * + *

    NOTE: This method makes a best effort to provide information about the classloader, and + * never throws an exception.

    + * + * @param loader The classloader to get the info string for. + * @return The classloader information string. + */ + public static String getUserCodeClassLoaderInfo(ClassLoader loader) { + if (loader instanceof URLClassLoader) { + URLClassLoader cl = (URLClassLoader) loader; + + try { + StringBuilder bld = new StringBuilder(); + + if (cl == ClassLoader.getSystemClassLoader()) { + bld.append("System ClassLoader: "); + } + else { + bld.append("URL ClassLoader:"); + } + + for (URL url : cl.getURLs()) { + bld.append("\n "); + if (url == null) { + bld.append("(null)"); + } + else if ("file".equals(url.getProtocol())) { + String filePath = url.getPath(); + File fileFile = new File(filePath); + + bld.append("file: '").append(filePath).append('\''); + + if (fileFile.exists()) { + if (fileFile.isDirectory()) { + bld.append(" (directory)"); + } + else { + try { + new JarFile(filePath); + bld.append(" (valid JAR)"); + } + catch (Exception e) { + bld.append(" (invalid JAR: ").append(e.getMessage()).append(')'); + } + } + } + else { + bld.append(" (missing)"); + } + } + else { + bld.append("url: ").append(url); + } + } + + return bld.toString(); + } + catch (Throwable t) { + return "Cannot access classloader info due to an exception.\n" + + ExceptionUtils.stringifyException(t); + } + } + else { + return "No user code ClassLoader"; + } + } + + /** + * Checks, whether the class that was not found in the given exception, can be resolved through + * the given class loader. + * + * @param cnfe The ClassNotFoundException that defines the name of the class. + * @param cl The class loader to use for the class resolution. + * @return True, if the class can be resolved with the given class loader, false if not. + */ + public static boolean validateClassLoadable(ClassNotFoundException cnfe, ClassLoader cl) { + try { + String className = cnfe.getMessage(); + Class.forName(className, false, cl); + return true; + } + catch (ClassNotFoundException e) { + return false; + } + catch (Exception e) { + return false; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java new file mode 100644 index 0000000000000..d5f3c9ebded76 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java @@ -0,0 +1,131 @@ +/* + * 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.runtime.util; + +import static org.junit.Assert.*; + +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.jar.JarFile; + +/** + * Tests that validate the {@link ClassLoaderUtil}. + */ +public class ClassLoaderUtilsTest { + + @Test + public void testWithURLClassLoader() { + File validJar = null; + File invalidJar = null; + + try { + // file with jar contents + validJar = File.createTempFile("flink-url-test", ".tmp"); + JarFileCreator jarFileCreator = new JarFileCreator(validJar); + jarFileCreator.addClass(ClassLoaderUtilsTest.class); + jarFileCreator.createJarFile(); + + // validate that the JAR is correct and the test setup is not broken + try { + new JarFile(validJar.getAbsolutePath()); + } + catch (Exception e) { + e.printStackTrace(); + fail("test setup broken: cannot create a valid jar file"); + } + + // file with some random contents + invalidJar = File.createTempFile("flink-url-test", ".tmp"); + try (FileOutputStream invalidout = new FileOutputStream(invalidJar)) { + invalidout.write(new byte[] { -1, 1, -2, 3, -3, 4, }); + } + + // non existing file + File nonExisting = File.createTempFile("flink-url-test", ".tmp"); + assertTrue("Cannot create and delete temp file", nonExisting.delete()); + + + // create a URL classloader with + // - a HTTP URL + // - a file URL for an existing jar file + // - a file URL for an existing file that is not a jar file + // - a file URL for a non-existing file + + URL[] urls = { + new URL("http", "localhost", 26712, "/some/file/path"), + new URL("file", null, validJar.getAbsolutePath()), + new URL("file", null, invalidJar.getAbsolutePath()), + new URL("file", null, nonExisting.getAbsolutePath()), + }; + + URLClassLoader loader = new URLClassLoader(urls, getClass().getClassLoader()); + String info = ClassLoaderUtil.getUserCodeClassLoaderInfo(loader); + + assertTrue(info.indexOf("/some/file/path") > 0); + assertTrue(info.indexOf(validJar.getAbsolutePath() + "' (valid") > 0); + assertTrue(info.indexOf(invalidJar.getAbsolutePath() + "' (invalid JAR") > 0); + assertTrue(info.indexOf(nonExisting.getAbsolutePath() + "' (missing") > 0); + + System.out.println(info); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + finally { + if (validJar != null) { + //noinspection ResultOfMethodCallIgnored + validJar.delete(); + } + if (invalidJar != null) { + //noinspection ResultOfMethodCallIgnored + invalidJar.delete(); + } + } + } + + @Test + public void testWithAppClassLoader() { + try { + // must return something when invoked with 'null' + String result = ClassLoaderUtil.getUserCodeClassLoaderInfo(ClassLoader.getSystemClassLoader()); + assertTrue(result.toLowerCase().contains("system classloader")); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testInvalidClassLoaders() { + try { + // must return something when invoked with 'null' + assertNotNull(ClassLoaderUtil.getUserCodeClassLoaderInfo(null)); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 62735afcc7256..a8486d3cf1a58 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -28,6 +28,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.runtime.util.ClassLoaderUtil; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.operators.StreamOperator; @@ -198,12 +199,26 @@ public void setStreamOperator(StreamOperator operator) { } } } - - @SuppressWarnings({ "unchecked" }) + public T getStreamOperator(ClassLoader cl) { try { - return (T) InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl); - } catch (Exception e) { + @SuppressWarnings("unchecked") + T result = (T) InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl); + return result; + } + catch (ClassNotFoundException e) { + String classLoaderInfo = ClassLoaderUtil.getUserCodeClassLoaderInfo(cl); + boolean loadableDoubleCheck = ClassLoaderUtil.validateClassLoadable(e, cl); + + String exceptionMessage = "Cannot load user class: " + e.getMessage() + + "\nClassLoader info: " + classLoaderInfo + + (loadableDoubleCheck ? + "Class was actually found in classloader - deserialization issue." : + "Class not resolveable through given classloader."); + + throw new StreamTaskException(exceptionMessage); + } + catch (Exception e) { throw new StreamTaskException("Cannot instantiate user function.", e); } } From 54311aae8ac20fa4ed1f6217b12bffb02fabf43a Mon Sep 17 00:00:00 2001 From: ffbin <869218239@qq.com> Date: Wed, 12 Aug 2015 08:36:11 +0800 Subject: [PATCH 114/175] [FLINK-2507] [storm-compat] Rename the function tansformAndEmit Closes #1007 --- .../stormcompatibility/wrappers/AbstractStormCollector.java | 2 +- .../flink/stormcompatibility/wrappers/StormBoltCollector.java | 2 +- .../flink/stormcompatibility/wrappers/StormSpoutCollector.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java index dc77ca13507bd..4a8fb7de929fe 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java @@ -82,7 +82,7 @@ public AbstractStormCollector(final int numberOfAttributes) throws UnsupportedOp * @return the return value of {@link #doEmit(Object)} */ @SuppressWarnings("unchecked") - protected final List tansformAndEmit(final List tuple) { + protected final List transformAndEmit(final List tuple) { List taskIds; if (this.numberOfAttributes > 0) { assert (tuple.size() == this.numberOfAttributes); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java index 8b088c3d88cef..81ad9a6921a12 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java @@ -72,7 +72,7 @@ public void reportError(final Throwable error) { @Override public List emit(final String streamId, final Collection anchors, final List tuple) { - return this.tansformAndEmit(tuple); + return this.transformAndEmit(tuple); } @Override diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java index 01c980abf7fa6..09a7ac7a18788 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java @@ -68,7 +68,7 @@ public void reportError(final Throwable error) { @Override public List emit(final String streamId, final List tuple, final Object messageId) { - return this.tansformAndEmit(tuple); + return this.transformAndEmit(tuple); } From a41bc8cc137096916e79f145387804fe94864307 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Thu, 30 Jul 2015 17:38:43 +0200 Subject: [PATCH 115/175] [FLINK-2437] handle the case of a non-public default ctor in TypeExtractor.analyzePojo Also changed some prints which printed the word "class" twice, because class.toString also prepends it to the class name. This closes #960. --- docs/apis/programming_guide.md | 2 +- .../api/java/typeutils/TypeExtractor.java | 24 ++++++++++++------- .../type/extractor/TypeExtractorTest.java | 7 ++++++ 3 files changed, 23 insertions(+), 10 deletions(-) diff --git a/docs/apis/programming_guide.md b/docs/apis/programming_guide.md index 598e204d95288..85c639ec88940 100644 --- a/docs/apis/programming_guide.md +++ b/docs/apis/programming_guide.md @@ -1520,7 +1520,7 @@ Java and Scala classes are treated by Flink as a special POJO data type if they - The type of a field must be supported by Flink. At the moment, Flink uses [Avro](http://avro.apache.org) to serialize arbitrary objects (such as `Date`). -Flink analyzes the structure of POJO types, i.e., it learns about the fields of a POJO. As a result POJO types are easier to use than general types. Moreover, they Flink can process POJOs more efficiently than general types. +Flink analyzes the structure of POJO types, i.e., it learns about the fields of a POJO. As a result POJO types are easier to use than general types. Moreover, Flink can process POJOs more efficiently than general types. The following example shows a simple POJO with two public fields. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index 1ae8d3d62fa1f..2e4510708d158 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.typeutils; +import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.GenericArrayType; import java.lang.reflect.Method; @@ -1298,10 +1299,10 @@ private boolean isValidPojoField(Field f, Class clazz, ArrayList typeHi return true; } else { if(!hasGetter) { - LOG.debug("Class "+clazz+" does not contain a getter for field "+f.getName() ); + LOG.debug(clazz+" does not contain a getter for field "+f.getName() ); } if(!hasSetter) { - LOG.debug("Class "+clazz+" does not contain a setter for field "+f.getName() ); + LOG.debug(clazz+" does not contain a setter for field "+f.getName() ); } return false; } @@ -1323,7 +1324,7 @@ else if(typeHierarchy.size() <= 1) { List fields = getAllDeclaredFields(clazz); if(fields.size() == 0) { - LOG.info("No fields detected for class " + clazz + ". Cannot be used as a PojoType. Will be handled as GenericType"); + LOG.info("No fields detected for " + clazz + ". Cannot be used as a PojoType. Will be handled as GenericType"); return new GenericTypeInfo(clazz); } @@ -1331,7 +1332,7 @@ else if(typeHierarchy.size() <= 1) { for (Field field : fields) { Type fieldType = field.getGenericType(); if(!isValidPojoField(field, clazz, typeHierarchy)) { - LOG.info("Class " + clazz + " is not a valid POJO type"); + LOG.info(clazz + " is not a valid POJO type"); return null; } try { @@ -1357,24 +1358,29 @@ else if(typeHierarchy.size() <= 1) { List methods = getAllDeclaredMethods(clazz); for (Method method : methods) { if (method.getName().equals("readObject") || method.getName().equals("writeObject")) { - LOG.info("Class "+clazz+" contains custom serialization methods we do not call."); + LOG.info(clazz+" contains custom serialization methods we do not call."); return null; } } // Try retrieving the default constructor, if it does not have one // we cannot use this because the serializer uses it. + Constructor defaultConstructor = null; try { - clazz.getDeclaredConstructor(); + defaultConstructor = clazz.getDeclaredConstructor(); } catch (NoSuchMethodException e) { if (clazz.isInterface() || Modifier.isAbstract(clazz.getModifiers())) { - LOG.info("Class " + clazz + " is abstract or an interface, having a concrete " + + LOG.info(clazz + " is abstract or an interface, having a concrete " + "type can increase performance."); } else { - LOG.info("Class " + clazz + " must have a default constructor to be used as a POJO."); + LOG.info(clazz + " must have a default constructor to be used as a POJO."); return null; } } + if(defaultConstructor != null && !Modifier.isPublic(defaultConstructor.getModifiers())) { + LOG.info("The default constructor of " + clazz + " should be Public to be used as a POJO."); + return null; + } // everything is checked, we return the pojo return pojoType; @@ -1394,7 +1400,7 @@ public static List getAllDeclaredFields(Class clazz) { continue; // we have no use for transient or static fields } if(hasFieldWithSameName(field.getName(), result)) { - throw new RuntimeException("The field "+field+" is already contained in the hierarchy of the class "+clazz+"." + throw new RuntimeException("The field "+field+" is already contained in the hierarchy of the "+clazz+"." + "Please use unique field names through your classes hierarchy"); } result.add(field); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java index dade55c009c63..925e8e5e28b37 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java @@ -313,6 +313,11 @@ public int getMyField2() { } } + public static class PojoWithNonPublicDefaultCtor { + public int foo, bar; + PojoWithNonPublicDefaultCtor() {} + } + @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testPojo() { @@ -345,6 +350,8 @@ public CustomType cross(CustomType first, Integer second) throws Exception { Assert.assertFalse(ti2.isTupleType()); Assert.assertTrue(ti2 instanceof PojoTypeInfo); Assert.assertEquals(ti2.getTypeClass(), CustomType.class); + + Assert.assertFalse(TypeExtractor.getForClass(PojoWithNonPublicDefaultCtor.class) instanceof PojoTypeInfo); } From 7349ef815e8625efaa6d2722fbe583d4c6067f1a Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 12 Aug 2015 18:10:41 +0200 Subject: [PATCH 116/175] [hotfix] read from old and new parallelism config key regression of c6358024454cd8225cf27a91db7f64ffa13189ee --- .../streaming/api/environment/StreamContextEnvironment.java | 2 +- .../flink/streaming/api/environment/StreamPlanEnvironment.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java index 5240f65b5b878..9bfeb2f3dae16 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java @@ -47,7 +47,7 @@ protected StreamContextEnvironment(Client client, List jars, int paralleli } else { // first check for old parallelism config key setParallelism(GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD, ConfigConstants.DEFAULT_PARALLELISM)); // then for new setParallelism(GlobalConfiguration.getInteger( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java index 9b6b933e4d2f2..02fccd00b644f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java @@ -39,7 +39,7 @@ protected StreamPlanEnvironment(ExecutionEnvironment env) { } else { // first check for old parallelism config key setParallelism(GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD, ConfigConstants.DEFAULT_PARALLELISM)); // then for new setParallelism(GlobalConfiguration.getInteger( From 7cc85c7d02698cf4b3e324e496fcee7aa43a4b2a Mon Sep 17 00:00:00 2001 From: hsaputra Date: Wed, 12 Aug 2015 13:46:44 -0700 Subject: [PATCH 117/175] [CLEANUP] Add space between quotes and plus sign in String concat for readability in flink-yarn module While working on Flink on YARN, do some simple nit cleanups to add space between quote and plus sign for readability in yarn module for readability. Use StringBuilder instad of StringBuffer in FlinkYarnClient since the String materializes immediately, so no need synchronize protection. Author: hsaputra Closes #1010 from hsaputra/cleanup_yarn_branch and squashes the following commits: a6fc9e1 [hsaputra] Add space between quote and plus sign for readability in flink-yarn module. --- .../apache/flink/yarn/FlinkYarnClient.java | 54 ++++++++++--------- .../apache/flink/yarn/FlinkYarnCluster.java | 10 ++-- .../java/org/apache/flink/yarn/Utils.java | 10 ++-- .../yarn/appMaster/YarnTaskManagerRunner.java | 4 +- .../flink/yarn/FlinkYarnSessionCliTest.java | 2 +- 5 files changed, 41 insertions(+), 39 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java index deb48099a80d9..c7ec2540b73ce 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java @@ -167,8 +167,8 @@ public FlinkYarnClient() { @Override public void setJobManagerMemory(int memoryMb) { if(memoryMb < MIN_JM_MEMORY) { - throw new IllegalArgumentException("The JobManager memory ("+memoryMb+") is below the minimum required memory amount " - + "of "+MIN_JM_MEMORY+" MB"); + throw new IllegalArgumentException("The JobManager memory (" + memoryMb + ") is below the minimum required memory amount " + + "of " + MIN_JM_MEMORY+ " MB"); } this.jobManagerMemoryMb = memoryMb; } @@ -176,8 +176,8 @@ public void setJobManagerMemory(int memoryMb) { @Override public void setTaskManagerMemory(int memoryMb) { if(memoryMb < MIN_TM_MEMORY) { - throw new IllegalArgumentException("The TaskManager memory ("+memoryMb+") is below the minimum required memory amount " - + "of "+MIN_TM_MEMORY+" MB"); + throw new IllegalArgumentException("The TaskManager memory (" + memoryMb + ") is below the minimum required memory amount " + + "of " + MIN_TM_MEMORY+ " MB"); } this.taskManagerMemoryMb = memoryMb; } @@ -208,7 +208,7 @@ public void setQueue(String queue) { @Override public void setLocalJarPath(Path localJarPath) { if(!localJarPath.toString().endsWith("jar")) { - throw new IllegalArgumentException("The passed jar path ('"+localJarPath+"') does not end with the 'jar' extension"); + throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); } this.flinkJarPath = localJarPath; } @@ -385,7 +385,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { LOG.debug("The YARN cluster does not have any queues configured"); } } catch(Throwable e) { - LOG.warn("Error while getting queue information from YARN: "+e.getMessage()); + LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); if(LOG.isDebugEnabled()) { LOG.debug("Error details", e); } @@ -398,7 +398,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { final int yarnMinAllocationMB = conf.getInt("yarn.scheduler.minimum-allocation-mb", 0); if(jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " - + "The value of 'yarn.scheduler.minimum-allocation-mb' is '"+yarnMinAllocationMB+"'. Please increase the memory size." + + + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + "you requested will start."); } @@ -416,13 +416,13 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { if(jobManagerMemoryMb > maxRes.getMemory() ) { failSessionDuringDeployment(); throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" - + "Maximum Memory: "+maxRes.getMemory() + "MB Requested: "+jobManagerMemoryMb+"MB. " + NOTE); + + "Maximum Memory: " + maxRes.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + NOTE); } if(taskManagerMemoryMb > maxRes.getMemory() ) { failSessionDuringDeployment(); throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" - + "Maximum Memory: " + maxRes.getMemory() + " Requested: "+taskManagerMemoryMb + "MB. " + NOTE); + + "Maximum Memory: " + maxRes.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + NOTE); } final String NOTE_RSC = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + @@ -437,8 +437,8 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { } if(taskManagerMemoryMb > freeClusterMem.containerLimit) { - LOG.warn("The requested amount of memory for the TaskManagers ("+taskManagerMemoryMb+"MB) is more than " - + "the largest possible YARN container: "+freeClusterMem.containerLimit + NOTE_RSC); + LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterMem.containerLimit + NOTE_RSC); } if(jobManagerMemoryMb > freeClusterMem.containerLimit) { LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than " @@ -451,14 +451,15 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { // first, allocate the jobManager somewhere. if(!allocateResource(nmFree, jobManagerMemoryMb)) { LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " + - "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: "+Arrays.toString(freeClusterMem.nodeManagersFree) + NOTE_RSC); + "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " + + Arrays.toString(freeClusterMem.nodeManagersFree) + NOTE_RSC); } // allocate TaskManagers for(int i = 0; i < taskManagerCount; i++) { if(!allocateResource(nmFree, taskManagerMemoryMb)) { LOG.warn("There is not enough memory available in the YARN cluster. " + "The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " + - "NodeManagers available: "+Arrays.toString(freeClusterMem.nodeManagersFree) + "\n" + + "NodeManagers available: " + Arrays.toString(freeClusterMem.nodeManagersFree) + "\n" + "After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " + "the following NodeManagers are available: " + Arrays.toString(nmFree) + NOTE_RSC ); } @@ -486,10 +487,10 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { .newRecord(ContainerLaunchContext.class); String amCommand = "$JAVA_HOME/bin/java" - + " -Xmx"+Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration)+"M " +javaOpts; + + " -Xmx" + Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration) + "M " +javaOpts; if(hasLogback || hasLog4j) { - amCommand += " -Dlog.file=\""+ApplicationConstants.LOG_DIR_EXPANSION_VAR +"/jobmanager-main.log\""; + amCommand += " -Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-main.log\""; } if(hasLogback) { @@ -499,13 +500,13 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { amCommand += " -Dlog4j.configuration=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; } - amCommand += " "+ApplicationMaster.class.getName()+" " + amCommand += " " + ApplicationMaster.class.getName() + " " + " 1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-stdout.log" + " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-stderr.log"; amContainer.setCommands(Collections.singletonList(amCommand)); - LOG.debug("Application Master start command: "+amCommand); + LOG.debug("Application Master start command: " + amCommand); // intialize HDFS // Copy the application master jar to the filesystem @@ -538,7 +539,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { // setup security tokens (code from apache storm) final Path[] paths = new Path[2 + shipFiles.size()]; - StringBuffer envShipFileList = new StringBuffer(); + StringBuilder envShipFileList = new StringBuilder(); // upload ship files for (int i = 0; i < shipFiles.size(); i++) { File shipFile = shipFiles.get(i); @@ -594,7 +595,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { String name; if(customName == null) { - name = "Flink session with "+taskManagerCount+" TaskManagers"; + name = "Flink session with " + taskManagerCount + " TaskManagers"; if(detached) { name += " (detached)"; } @@ -623,16 +624,16 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { case FINISHED: case KILLED: throw new YarnDeploymentException("The YARN application unexpectedly switched to state " - + appState +" during deployment. \n" + - "Diagnostics from YARN: "+report.getDiagnostics() + "\n" + + + appState + " during deployment. \n" + + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + "If log aggregation is enabled on your cluster, use this command to further invesitage the issue:\n" + - "yarn logs -applicationId "+appId); + "yarn logs -applicationId " + appId); //break .. case RUNNING: LOG.info("YARN application has been deployed successfully."); break loop; default: - LOG.info("Deploying cluster, current state "+appState); + LOG.info("Deploying cluster, current state " + appState); if(waittime > 60000) { LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster"); } @@ -714,16 +715,17 @@ public String getClusterDescription() throws Exception { totalMemory += res.getMemory(); totalCores += res.getVirtualCores(); ps.format(format, "NodeID", rep.getNodeId()); - ps.format(format, "Memory", res.getMemory()+" MB"); + ps.format(format, "Memory", res.getMemory() + " MB"); ps.format(format, "vCores", res.getVirtualCores()); ps.format(format, "HealthReport", rep.getHealthReport()); ps.format(format, "Containers", rep.getNumContainers()); ps.println("+---------------------------------------+"); } - ps.println("Summary: totalMemory "+totalMemory+" totalCores "+totalCores); + ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores); List qInfo = yarnClient.getAllQueues(); for(QueueInfo q : qInfo) { - ps.println("Queue: "+q.getQueueName()+", Current Capacity: "+q.getCurrentCapacity()+" Max Capacity: "+q.getMaximumCapacity()+" Applications: "+q.getApplications().size()); + ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + + q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); } yarnClient.stop(); return baos.toString(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java index e408edb897e75..5fa3ac7704ba9 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java @@ -282,7 +282,7 @@ public FlinkYarnClusterStatus getClusterStatus() { } else if(clusterStatus instanceof Some) { return (FlinkYarnClusterStatus) (((Some) clusterStatus).get()); } else { - throw new RuntimeException("Unexpected type: "+clusterStatus.getClass().getCanonicalName()); + throw new RuntimeException("Unexpected type: " + clusterStatus.getClass().getCanonicalName()); } } @@ -368,9 +368,9 @@ public List getNewMessages() { if(obj instanceof Messages.YarnMessage) { Messages.YarnMessage msg = (Messages.YarnMessage) obj; - ret.add("["+msg.date()+"] "+msg.message()); + ret.add("[" + msg.date() + "] " + msg.message()); } else { - LOG.warn("LocalGetYarnMessage returned unexpected type: "+messageOption); + LOG.warn("LocalGetYarnMessage returned unexpected type: " + messageOption); } } } @@ -429,7 +429,7 @@ public void shutdown(boolean failApplication) { actorSystem = null; } - LOG.info("Deleting files in "+sessionFilesDir ); + LOG.info("Deleting files in " + sessionFilesDir ); try { FileSystem shutFS = FileSystem.get(hadoopConfig); shutFS.delete(sessionFilesDir, true); // delete conf and jar file. @@ -522,7 +522,7 @@ public void run() { } if(running.get() && !yarnClient.isInState(Service.STATE.STARTED)) { // == if the polling thread is still running but the yarn client is stopped. - LOG.warn("YARN client is unexpected in state "+yarnClient.getServiceState()); + LOG.warn("YARN client is unexpected in state " + yarnClient.getServiceState()); } } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index a85a47fbe0975..f246ebfe2e60d 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -60,10 +60,10 @@ public static int calculateHeapSize(int memory, org.apache.flink.configuration.C int minCutoff = conf.getInteger(ConfigConstants.YARN_HEAP_CUTOFF_MIN, ConfigConstants.DEFAULT_YARN_MIN_HEAP_CUTOFF); if (memoryCutoffRatio > 1 || memoryCutoffRatio < 0) { - throw new IllegalArgumentException("The configuration value '"+ConfigConstants.YARN_HEAP_CUTOFF_RATIO+"' must be between 0 and 1. Value given="+memoryCutoffRatio); + throw new IllegalArgumentException("The configuration value '" + ConfigConstants.YARN_HEAP_CUTOFF_RATIO + "' must be between 0 and 1. Value given=" + memoryCutoffRatio); } if (minCutoff > memory) { - throw new IllegalArgumentException("The configuration value '"+ConfigConstants.YARN_HEAP_CUTOFF_MIN +"' is higher ("+minCutoff+") than the requested amount of memory "+memory); + throw new IllegalArgumentException("The configuration value '" + ConfigConstants.YARN_HEAP_CUTOFF_MIN + "' is higher (" + minCutoff + ") than the requested amount of memory " + memory); } int heapLimit = (int)((float)memory * memoryCutoffRatio); @@ -94,7 +94,7 @@ public static Path setupLocalResource(Configuration conf, FileSystem fs, String Path dst = new Path(homedir, suffix); - LOG.info("Copying from "+localRsrcPath+" to "+dst ); + LOG.info("Copying from " + localRsrcPath + " to " + dst ); fs.copyFromLocalFile(localRsrcPath, dst); registerLocalResource(fs, dst, appMasterJar); return dst; @@ -119,7 +119,7 @@ public static void setTokensFor(ContainerLaunchContext amContainer, Path[] paths Collection> usrTok = currUsr.getTokens(); for(Token token : usrTok) { final Text id = new Text(token.getIdentifier()); - LOG.info("Adding user token "+id+" with "+token); + LOG.info("Adding user token " + id + " with " + token); credentials.addToken(id, token); } DataOutputBuffer dob = new DataOutputBuffer(); @@ -138,7 +138,7 @@ public static void logFilesInCurrentDirectory(final Logger logger) { @Override public boolean accept(File dir, String name) { - logger.info(dir.getAbsolutePath()+"/"+name); + logger.info(dir.getAbsolutePath() + "/" + name); return true; } }); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java index 3f13990d0b507..1a13f939a2267 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java @@ -82,8 +82,8 @@ public static void main(final String[] args) throws IOException { "specified in the Flink config: " + flinkTempDirs); } - LOG.info("YARN daemon runs as '" + UserGroupInformation.getCurrentUser().getShortUserName() - +"' setting user to execute Flink TaskManager to '"+yarnClientUsername+"'"); + LOG.info("YARN daemon runs as '" + UserGroupInformation.getCurrentUser().getShortUserName() + + "' setting user to execute Flink TaskManager to '" + yarnClientUsername + "'"); // tell akka to die in case of an error configuration.setBoolean(ConfigConstants.AKKA_JVM_EXIT_ON_FATAL_ERROR, true); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 7b22e16dc4874..fb644c30e737d 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -62,7 +62,7 @@ public void testDynamicProperties() throws IOException { cmd = parser.parse(options, new String[]{"run", "-j", "fake.jar", "-n", "15", "-D", "akka.ask.timeout=5 min"}); } catch(Exception e) { e.printStackTrace(); - Assert.fail("Parsing failed with "+e.getMessage()); + Assert.fail("Parsing failed with " + e.getMessage()); } AbstractFlinkYarnClient flinkYarnClient = cli.createFlinkYarnClient(cmd); From 26c6447e92e0bb07de80ab0de0bfcc08e7d78405 Mon Sep 17 00:00:00 2001 From: Sachin Goel Date: Thu, 6 Aug 2015 09:22:35 +0530 Subject: [PATCH 118/175] [FLINK-1819][core] allow access to RuntimeContext from Input and OutputFormats 1. Allow access to Runtime Context from I/O formats. 2. Make all existing I/O formats context aware. This closes #966. --- .../flink/api/common/io/FileInputFormat.java | 4 +- .../flink/api/common/io/FileOutputFormat.java | 5 +- .../api/common/io/GenericInputFormat.java | 4 +- .../api/common/io/ReplicatingInputFormat.java | 20 ++- .../flink/api/common/io/RichInputFormat.java | 50 +++++++ .../flink/api/common/io/RichOutputFormat.java | 49 +++++++ .../api/common/io/SerializedOutputFormat.java | 3 +- .../common/operators/CollectionExecutor.java | 32 ++++- .../common/operators/GenericDataSinkBase.java | 18 ++- .../operators/GenericDataSourceBase.java | 12 +- .../api/common/io/RichInputFormatTest.java | 45 +++++++ .../api/common/io/RichOutputFormatTest.java | 45 +++++++ .../operators/GenericDataSinkBaseTest.java | 105 +++++++++++++++ .../operators/GenericDataSourceBaseTest.java | 91 +++++++++++++ .../util/NonRichGenericInputFormat.java | 86 ++++++++++++ .../api/common/operators/util/TestIOData.java | 25 ++++ .../util/TestNonRichInputFormat.java | 50 +++++++ .../util/TestNonRichOutputFormat.java | 51 ++++++++ .../operators/util/TestRichInputFormat.java | 52 ++++++++ .../operators/util/TestRichOutputFormat.java | 52 ++++++++ .../hadoop/mapred/HadoopInputFormatBase.java | 4 +- .../hadoop/mapred/HadoopOutputFormatBase.java | 4 +- .../mapreduce/HadoopInputFormatBase.java | 4 +- .../mapreduce/HadoopOutputFormatBase.java | 4 +- .../java/io/LocalCollectionOutputFormat.java | 4 +- .../api/java/io/PrintingOutputFormat.java | 4 +- .../flink/runtime/operators/DataSinkTask.java | 18 +++ .../runtime/operators/DataSourceTask.java | 24 +++- .../flink/addons/hbase/TableInputFormat.java | 3 +- .../flink/hcatalog/HCatInputFormatBase.java | 4 +- .../api/java/io/jdbc/JDBCInputFormat.java | 4 +- .../api/java/io/jdbc/JDBCOutputFormat.java | 4 +- .../flink/test/io/InputOutputITCase.java | 47 +++++++ .../flink/test/io/RichInputOutputITCase.java | 123 ++++++++++++++++++ 34 files changed, 1001 insertions(+), 49 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/io/RichInputFormat.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/io/RichOutputFormat.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/util/NonRichGenericInputFormat.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestIOData.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichInputFormat.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichOutputFormat.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichInputFormat.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichOutputFormat.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java index c9c9ec184a9dc..4dee9c77ed66f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java @@ -46,7 +46,7 @@ import org.apache.flink.core.fs.Path; /** - * The base class for {@link InputFormat}s that read from files. For specific input types the + * The base class for {@link RichInputFormat}s that read from files. For specific input types the * {@link #nextRecord(Object)} and {@link #reachedEnd()} methods need to be implemented. * Additionally, one may override {@link #open(FileInputSplit)} and {@link #close()} to * change the life cycle behavior. @@ -54,7 +54,7 @@ *

    After the {@link #open(FileInputSplit)} method completed, the file input data is available * from the {@link #stream} field.

    */ -public abstract class FileInputFormat implements InputFormat { +public abstract class FileInputFormat extends RichInputFormat { // -------------------------------------- Constants ------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java index 08d0be23aadc0..6854268b249d3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java @@ -33,10 +33,11 @@ import org.apache.flink.core.fs.FileSystem.WriteMode; /** - * The abstract base class for all output formats that are file based. Contains the logic to open/close the target + * The abstract base class for all Rich output formats that are file based. Contains the logic to + * open/close the target * file streams. */ -public abstract class FileOutputFormat implements OutputFormat, InitializeOnMaster, CleanupWhenUnsuccessful { +public abstract class FileOutputFormat extends RichOutputFormat implements InitializeOnMaster, CleanupWhenUnsuccessful { private static final long serialVersionUID = 1L; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java index b7d4c8ea9516b..0e3b1c21e75b1 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java @@ -26,9 +26,9 @@ import org.apache.flink.core.io.GenericInputSplit; /** - * Generic base class for all inputs that are not based on files. + * Generic base class for all Rich inputs that are not based on files. */ -public abstract class GenericInputFormat implements InputFormat { +public abstract class GenericInputFormat extends RichInputFormat { private static final long serialVersionUID = 1L; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java index cc25539347fb9..3d0ea99004266 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common.io; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplit; @@ -52,6 +53,7 @@ * @param The InputSplit type of the wrapped InputFormat. * * @see org.apache.flink.api.common.io.InputFormat + * @see org.apache.flink.api.common.io.RichInputFormat * @see org.apache.flink.api.common.operators.base.JoinOperatorBase * @see org.apache.flink.api.common.operators.base.CrossOperatorBase * @see org.apache.flink.api.common.operators.base.MapOperatorBase @@ -59,7 +61,7 @@ * @see org.apache.flink.api.common.operators.base.FilterOperatorBase * @see org.apache.flink.api.common.operators.base.MapPartitionOperatorBase */ -public final class ReplicatingInputFormat implements InputFormat { +public final class ReplicatingInputFormat extends RichInputFormat { private static final long serialVersionUID = 1L; @@ -112,4 +114,20 @@ public OT nextRecord(OT reuse) throws IOException { public void close() throws IOException { this.replicatedIF.close(); } + + @Override + public void setRuntimeContext(RuntimeContext context){ + if(this.replicatedIF instanceof RichInputFormat){ + ((RichInputFormat)this.replicatedIF).setRuntimeContext(context); + } + } + + @Override + public RuntimeContext getRuntimeContext(){ + if(this.replicatedIF instanceof RichInputFormat){ + return ((RichInputFormat)this.replicatedIF).getRuntimeContext(); + } else{ + throw new RuntimeException("The underlying input format to this ReplicatingInputFormat isn't context aware"); + } + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/RichInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/RichInputFormat.java new file mode 100644 index 0000000000000..acc81bcb99d9a --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/RichInputFormat.java @@ -0,0 +1,50 @@ +/* +c * 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.api.common.io; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.core.io.InputSplit; + +/** + * An abstract stub implementation for Rich input formats. + * Rich formats have access to their runtime execution context via {@link #getRuntimeContext()}. + */ +public abstract class RichInputFormat implements InputFormat { + + private static final long serialVersionUID = 1L; + + // -------------------------------------------------------------------------------------------- + // Runtime context access + // -------------------------------------------------------------------------------------------- + + private transient RuntimeContext runtimeContext; + + public void setRuntimeContext(RuntimeContext t) { + this.runtimeContext = t; + } + + public RuntimeContext getRuntimeContext() { + if (this.runtimeContext != null) { + return this.runtimeContext; + } else { + throw new IllegalStateException("The runtime context has not been initialized yet. Try accessing " + + "it in one of the other life cycle methods."); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/RichOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/RichOutputFormat.java new file mode 100644 index 0000000000000..927a974902848 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/RichOutputFormat.java @@ -0,0 +1,49 @@ +/* +c * 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.api.common.io; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** + * An abstract stub implementation for Rich output formats. + * Rich formats have access to their runtime execution context via {@link #getRuntimeContext()}. + */ +public abstract class RichOutputFormat implements OutputFormat { + + private static final long serialVersionUID = 1L; + + // -------------------------------------------------------------------------------------------- + // Runtime context access + // -------------------------------------------------------------------------------------------- + + private transient RuntimeContext runtimeContext; + + public void setRuntimeContext(RuntimeContext t) { + this.runtimeContext = t; + } + + public RuntimeContext getRuntimeContext() { + if (this.runtimeContext != null) { + return this.runtimeContext; + } else { + throw new IllegalStateException("The runtime context has not been initialized yet. Try accessing " + + "it in one of the other life cycle methods."); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/SerializedOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/SerializedOutputFormat.java index 71e08f2d947d3..24fc77d82a235 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/SerializedOutputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/SerializedOutputFormat.java @@ -28,8 +28,7 @@ * * @see SerializedInputFormat */ -public class SerializedOutputFormat extends - BinaryOutputFormat { +public class SerializedOutputFormat extends BinaryOutputFormat { private static final long serialVersionUID = 1L; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java index 0a9146c08bbf1..e8e001297ddff 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java @@ -40,6 +40,8 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.operators.base.BulkIterationBase; import org.apache.flink.api.common.operators.base.BulkIterationBase.PartialSolutionPlaceHolder; import org.apache.flink.api.common.operators.base.DeltaIterationBase; @@ -128,10 +130,10 @@ else if (operator instanceof DualInputOperator) { result = executeBinaryOperator((DualInputOperator) operator, superStep); } else if (operator instanceof GenericDataSourceBase) { - result = executeDataSource((GenericDataSourceBase) operator); + result = executeDataSource((GenericDataSourceBase) operator, superStep); } else if (operator instanceof GenericDataSinkBase) { - executeDataSink((GenericDataSinkBase) operator); + executeDataSink((GenericDataSinkBase) operator, superStep); result = Collections.emptyList(); } else { @@ -148,7 +150,7 @@ else if (operator instanceof GenericDataSinkBase) { // Operator class specific execution methods // -------------------------------------------------------------------------------------------- - private void executeDataSink(GenericDataSinkBase sink) throws Exception { + private void executeDataSink(GenericDataSinkBase sink, int superStep) throws Exception { Operator inputOp = sink.getInput(); if (inputOp == null) { throw new InvalidProgramException("The data sink " + sink.getName() + " has no input."); @@ -160,13 +162,31 @@ private void executeDataSink(GenericDataSinkBase sink) throws Exception @SuppressWarnings("unchecked") GenericDataSinkBase typedSink = (GenericDataSinkBase) sink; - typedSink.executeOnCollections(input, executionConfig); + // build the runtime context and compute broadcast variables, if necessary + RuntimeUDFContext ctx; + if (RichOutputFormat.class.isAssignableFrom(typedSink.getUserCodeWrapper().getUserCodeClass())) { + ctx = superStep == 0 ? new RuntimeUDFContext(typedSink.getName(), 1, 0, getClass().getClassLoader(), executionConfig, accumulators) : + new IterationRuntimeUDFContext(typedSink.getName(), 1, 0, classLoader, executionConfig, accumulators); + } else { + ctx = null; + } + + typedSink.executeOnCollections(input, ctx, executionConfig); } - private List executeDataSource(GenericDataSourceBase source) throws Exception { + private List executeDataSource(GenericDataSourceBase source, int superStep) + throws Exception { @SuppressWarnings("unchecked") GenericDataSourceBase typedSource = (GenericDataSourceBase) source; - return typedSource.executeOnCollections(executionConfig); + // build the runtime context and compute broadcast variables, if necessary + RuntimeUDFContext ctx; + if (RichInputFormat.class.isAssignableFrom(typedSource.getUserCodeWrapper().getUserCodeClass())) { + ctx = superStep == 0 ? new RuntimeUDFContext(source.getName(), 1, 0, getClass().getClassLoader(), executionConfig, accumulators) : + new IterationRuntimeUDFContext(source.getName(), 1, 0, classLoader, executionConfig, accumulators); + } else { + ctx = null; + } + return typedSource.executeOnCollections(ctx, executionConfig); } private List executeUnaryOperator(SingleInputOperator operator, int superStep) throws Exception { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java index 57f427afe09dc..00cef92d3ccc3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java @@ -19,15 +19,17 @@ package org.apache.flink.api.common.operators; +import java.util.List; import java.util.Collections; import java.util.Comparator; -import java.util.List; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.io.FinalizeOnMaster; -import org.apache.flink.api.common.io.InitializeOnMaster; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.common.io.InitializeOnMaster; +import org.apache.flink.api.common.io.FinalizeOnMaster; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.api.common.typeinfo.AtomicType; @@ -296,11 +298,11 @@ public void accept(Visitor> visitor) { visitor.postVisit(this); } } - + // -------------------------------------------------------------------------------------------- @SuppressWarnings("unchecked") - protected void executeOnCollections(List inputData, ExecutionConfig executionConfig) throws Exception { + protected void executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { OutputFormat format = this.formatWrapper.getUserCodeObject(); TypeInformation inputType = getInput().getOperatorInfo().getOutputType(); @@ -328,9 +330,11 @@ public int compare(IN o1, IN o2) { if(format instanceof InitializeOnMaster) { ((InitializeOnMaster)format).initializeGlobal(1); } - format.configure(this.parameters); - + + if(format instanceof RichOutputFormat){ + ((RichOutputFormat) format).setRuntimeContext(ctx); + } format.open(0, 1); for (IN element : inputData) { format.writeRecord(element); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java index 05babf465a4cc..d9a3a082192ff 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java @@ -24,7 +24,9 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.operators.util.UserCodeWrapper; @@ -200,14 +202,18 @@ public void accept(Visitor> visitor) { visitor.postVisit(this); } } - + // -------------------------------------------------------------------------------------------- - protected List executeOnCollections(ExecutionConfig executionConfig) throws Exception { + protected List executeOnCollections(RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { @SuppressWarnings("unchecked") InputFormat inputFormat = (InputFormat) this.formatWrapper.getUserCodeObject(); inputFormat.configure(this.parameters); - + + if(inputFormat instanceof RichInputFormat){ + ((RichInputFormat) inputFormat).setRuntimeContext(ctx); + } + List result = new ArrayList(); // splits diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java new file mode 100644 index 0000000000000..126a51117f6cc --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java @@ -0,0 +1,45 @@ +/* + * 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.api.common.io; + +import java.util.HashMap; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.functions.util.RuntimeUDFContext; +import org.apache.flink.types.Value; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests runtime context access from inside an RichInputFormat class + */ +public class RichInputFormatTest { + + @Test + public void testCheckRuntimeContextAccess() { + final SerializedInputFormat inputFormat = new SerializedInputFormat(); + inputFormat.setRuntimeContext(new RuntimeUDFContext("test name", 3, 1, + getClass().getClassLoader(), new ExecutionConfig(), new HashMap>())); + + Assert.assertEquals(inputFormat.getRuntimeContext().getIndexOfThisSubtask(), 1); + Assert.assertEquals(inputFormat.getRuntimeContext().getNumberOfParallelSubtasks(),3); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java new file mode 100644 index 0000000000000..8d410391fdda7 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java @@ -0,0 +1,45 @@ +/* + * 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.api.common.io; + +import java.util.HashMap; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.functions.util.RuntimeUDFContext; +import org.apache.flink.types.Value; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests runtime context access from inside an RichOutputFormat class + */ +public class RichOutputFormatTest { + + @Test + public void testCheckRuntimeContextAccess() { + final SerializedOutputFormat inputFormat = new SerializedOutputFormat(); + inputFormat.setRuntimeContext(new RuntimeUDFContext("test name", 3, 1, + getClass().getClassLoader(), new ExecutionConfig(), new HashMap>())); + + Assert.assertEquals(inputFormat.getRuntimeContext().getIndexOfThisSubtask(), 1); + Assert.assertEquals(inputFormat.getRuntimeContext().getNumberOfParallelSubtasks(),3); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java new file mode 100644 index 0000000000000..e30e5361fabf5 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java @@ -0,0 +1,105 @@ +/* + * 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.api.common.operators; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.functions.util.RuntimeUDFContext; +import org.apache.flink.api.common.operators.util.TestIOData; +import org.apache.flink.api.common.operators.util.TestNonRichOutputFormat; +import org.apache.flink.api.common.operators.util.TestNonRichInputFormat; +import org.apache.flink.api.common.operators.util.TestRichOutputFormat; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.types.Nothing; +import org.junit.Test; + +import java.util.HashMap; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** + * Checks the GenericDataSinkBase operator for both Rich and non-Rich output formats. + */ +@SuppressWarnings("serial") +public class GenericDataSinkBaseTest implements java.io.Serializable { + + private static TestNonRichInputFormat in = new TestNonRichInputFormat(); + GenericDataSourceBase source = + new GenericDataSourceBase( + in, new OperatorInformation(BasicTypeInfo.STRING_TYPE_INFO), "testSource"); + + + @Test + public void testDataSourcePlain() { + try { + TestNonRichOutputFormat out = new TestNonRichOutputFormat(); + GenericDataSinkBase sink = new GenericDataSinkBase( + out, + new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.getInfoFor(Nothing.class)), + "test_sink"); + sink.setInput(source); + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + in.reset(); + sink.executeOnCollections(asList(TestIOData.NAMES), null, executionConfig); + assertEquals(out.output, asList(TestIOData.NAMES)); + + executionConfig.enableObjectReuse(); + out.clear(); + in.reset(); + sink.executeOnCollections(asList(TestIOData.NAMES), null, executionConfig); + assertEquals(out.output, asList(TestIOData.NAMES)); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testDataSourceWithRuntimeContext() { + try { + TestRichOutputFormat out = new TestRichOutputFormat(); + GenericDataSinkBase sink = new GenericDataSinkBase( + out, + new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.getInfoFor(Nothing.class)), + "test_sink"); + sink.setInput(source); + + ExecutionConfig executionConfig = new ExecutionConfig(); + final HashMap> accumulatorMap = new HashMap>(); + executionConfig.disableObjectReuse(); + in.reset(); + sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext("test_sink", 1, 0, null, executionConfig, accumulatorMap), executionConfig); + assertEquals(out.output, asList(TestIOData.RICH_NAMES)); + + executionConfig.enableObjectReuse(); + out.clear(); + in.reset(); + sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext("test_sink", 1, 0, null, executionConfig, accumulatorMap), executionConfig); + assertEquals(out.output, asList(TestIOData.RICH_NAMES)); + } catch(Exception e){ + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java new file mode 100644 index 0000000000000..64d33583ee7a1 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java @@ -0,0 +1,91 @@ +/* + * 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.api.common.operators; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.functions.util.RuntimeUDFContext; +import org.apache.flink.api.common.operators.util.TestIOData; +import org.apache.flink.api.common.operators.util.TestNonRichInputFormat; +import org.apache.flink.api.common.operators.util.TestRichInputFormat; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.junit.Test; + +import java.util.HashMap; +import java.util.List; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** + * Checks the GenericDataSourceBase operator for both Rich and non-Rich input formats. + */ +@SuppressWarnings("serial") +public class GenericDataSourceBaseTest implements java.io.Serializable { + + @Test + public void testDataSourcePlain() { + try { + TestNonRichInputFormat in = new TestNonRichInputFormat(); + GenericDataSourceBase source = + new GenericDataSourceBase( + in, new OperatorInformation(BasicTypeInfo.STRING_TYPE_INFO), "testSource"); + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultMutableSafe = source.executeOnCollections(null, executionConfig); + + in.reset(); + executionConfig.enableObjectReuse(); + List resultRegular = source.executeOnCollections(null, executionConfig); + assertEquals(asList(TestIOData.NAMES), resultMutableSafe); + assertEquals(asList(TestIOData.NAMES), resultRegular); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testDataSourceWithRuntimeContext() { + try { + TestRichInputFormat in = new TestRichInputFormat(); + GenericDataSourceBase source = + new GenericDataSourceBase( + in, new OperatorInformation(BasicTypeInfo.STRING_TYPE_INFO), "testSource"); + + final HashMap> accumulatorMap = new HashMap>(); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultMutableSafe = source.executeOnCollections(new RuntimeUDFContext("test_source", 1, 0, null, executionConfig, accumulatorMap), executionConfig); + + in.reset(); + executionConfig.enableObjectReuse(); + List resultRegular = source.executeOnCollections(new RuntimeUDFContext("test_source", 1, 0, null, executionConfig, accumulatorMap), executionConfig); + + assertEquals(asList(TestIOData.RICH_NAMES), resultMutableSafe); + assertEquals(asList(TestIOData.RICH_NAMES), resultRegular); + } catch(Exception e){ + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/NonRichGenericInputFormat.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/NonRichGenericInputFormat.java new file mode 100644 index 0000000000000..053f835cf7d77 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/NonRichGenericInputFormat.java @@ -0,0 +1,86 @@ +/* + * 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.api.common.operators.util; + +import java.io.IOException; + +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.NonParallelInput; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.GenericInputSplit; + +/** + * Generic base class for all inputs that are not based on files. + * This is copied from {@link org.apache.flink.api.common.io.GenericInputFormat} + * This class doesn't provide access to RuntimeContext. + */ +public abstract class NonRichGenericInputFormat implements InputFormat { + + private static final long serialVersionUID = 1L; + + /** + * The partition of this split. + */ + protected int partitionNumber; + + // -------------------------------------------------------------------------------------------- + + @Override + public void configure(Configuration parameters) { + // nothing by default + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { + // no statistics available, by default. + return cachedStatistics; + } + + @Override + public GenericInputSplit[] createInputSplits(int numSplits) throws IOException { + if (numSplits < 1) { + throw new IllegalArgumentException("Number of input splits has to be at least 1."); + } + + numSplits = (this instanceof NonParallelInput) ? 1 : numSplits; + GenericInputSplit[] splits = new GenericInputSplit[numSplits]; + for (int i = 0; i < splits.length; i++) { + splits[i] = new GenericInputSplit(i, numSplits); + } + return splits; + } + + @Override + public DefaultInputSplitAssigner getInputSplitAssigner(GenericInputSplit[] splits) { + return new DefaultInputSplitAssigner(splits); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public void open(GenericInputSplit split) throws IOException { + this.partitionNumber = split.getSplitNumber(); + } + + @Override + public void close() throws IOException {} +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestIOData.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestIOData.java new file mode 100644 index 0000000000000..d32261a480708 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestIOData.java @@ -0,0 +1,25 @@ +/* + * 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.api.common.operators.util; + +public class TestIOData { + public static final String[] NAMES = { "Peter", "Bob", "Liddy", "Alexander", "Stan" }; + public static final String[] RICH_NAMES = { "Peter01", "Bob01", "Liddy01", "Alexander01", "Stan01"}; +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichInputFormat.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichInputFormat.java new file mode 100644 index 0000000000000..5d71126599598 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichInputFormat.java @@ -0,0 +1,50 @@ +/* + * 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.api.common.operators.util; + +import org.apache.flink.api.common.io.NonParallelInput; + +import java.io.IOException; + +/** + * Test Non rich input format class which emits just five elements. + */ +public class TestNonRichInputFormat extends NonRichGenericInputFormat implements NonParallelInput{ + + private static final long serialVersionUID = 1L; + private static final int NUM = 5; + private static final String[] NAMES = TestIOData.NAMES; + private int count = 0; + + @Override + public boolean reachedEnd() throws IOException { + return count >= NUM; + } + + @Override + public String nextRecord(String reuse) throws IOException { + count++; + return NAMES[count - 1]; + } + + public void reset(){ + count = 0; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichOutputFormat.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichOutputFormat.java new file mode 100644 index 0000000000000..73650eff05ccb --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestNonRichOutputFormat.java @@ -0,0 +1,51 @@ +/* + * 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.api.common.operators.util; + +import java.util.LinkedList; +import java.util.List; + +import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.configuration.Configuration; + +/** + * Non rich test output format which stores everything in a list. + */ +public class TestNonRichOutputFormat implements OutputFormat { + public List output = new LinkedList(); + + @Override + public void configure(Configuration parameters){} + + @Override + public void open(int a, int b){} + + @Override + public void close(){} + + @Override + public void writeRecord(String record){ + output.add(record); + } + + public void clear(){ + output.clear(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichInputFormat.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichInputFormat.java new file mode 100644 index 0000000000000..09453914a3cf5 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichInputFormat.java @@ -0,0 +1,52 @@ +/* + * 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.api.common.operators.util; + +import java.io.IOException; + +import org.apache.flink.api.common.io.GenericInputFormat; +import org.apache.flink.api.common.io.NonParallelInput; + +/** + * Same as the non rich test input format, except it provide access to runtime context. + */ +public class TestRichInputFormat extends GenericInputFormat implements NonParallelInput{ + + private static final long serialVersionUID = 1L; + private static final int NUM = 5; + private static final String[] NAMES = TestIOData.NAMES; + private int count = 0; + + @Override + public boolean reachedEnd() throws IOException { + return count >= NUM; + } + + @Override + public String nextRecord(String reuse) throws IOException { + count++; + return NAMES[count - 1] + getRuntimeContext().getIndexOfThisSubtask() + "" + + getRuntimeContext().getNumberOfParallelSubtasks(); + } + + public void reset(){ + count = 0; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichOutputFormat.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichOutputFormat.java new file mode 100644 index 0000000000000..10cccdc480b6d --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/TestRichOutputFormat.java @@ -0,0 +1,52 @@ +/* + * 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.api.common.operators.util; + +import java.util.LinkedList; +import java.util.List; + +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.configuration.Configuration; + +/** + * Same as the non rich test output format, except it provide access to runtime context. + */ +public class TestRichOutputFormat extends RichOutputFormat { + public List output = new LinkedList(); + + @Override + public void configure(Configuration parameters){} + + @Override + public void open(int a, int b){} + + @Override + public void close(){} + + @Override + public void writeRecord(String record){ + output.add(record + getRuntimeContext().getIndexOfThisSubtask() + "" + + getRuntimeContext().getNumberOfParallelSubtasks()); + } + + public void clear(){ + output.clear(); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormatBase.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormatBase.java index d5dbf389082b4..b24e3517ddb40 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormatBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormatBase.java @@ -20,7 +20,7 @@ package org.apache.flink.api.java.hadoop.mapred; import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics; -import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.LocatableInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils; @@ -45,7 +45,7 @@ import java.io.ObjectOutputStream; import java.util.ArrayList; -public abstract class HadoopInputFormatBase implements InputFormat { +public abstract class HadoopInputFormatBase extends RichInputFormat { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormatBase.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormatBase.java index d6dfc2e79150a..a5baa7e2ad71c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormatBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormatBase.java @@ -20,7 +20,7 @@ package org.apache.flink.api.java.hadoop.mapred; import org.apache.flink.api.common.io.FinalizeOnMaster; -import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils; import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyProgressable; import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter; @@ -41,7 +41,7 @@ import java.io.ObjectOutputStream; -public abstract class HadoopOutputFormatBase implements OutputFormat, FinalizeOnMaster { +public abstract class HadoopOutputFormatBase extends RichOutputFormat implements FinalizeOnMaster { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java index 1236884d6e48a..3693176762944 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.hadoop.mapreduce; import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics; -import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.LocatableInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.hadoop.mapreduce.utils.HadoopUtils; @@ -48,7 +48,7 @@ import static com.google.common.base.Preconditions.checkNotNull; -public abstract class HadoopInputFormatBase implements InputFormat { +public abstract class HadoopInputFormatBase extends RichInputFormat { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopOutputFormatBase.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopOutputFormatBase.java index 0e592571e6318..14d5c81549ce4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopOutputFormatBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopOutputFormatBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.hadoop.mapreduce; import org.apache.flink.api.common.io.FinalizeOnMaster; -import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.hadoop.mapreduce.utils.HadoopUtils; import org.apache.flink.configuration.Configuration; import org.apache.hadoop.conf.Configurable; @@ -38,7 +38,7 @@ import java.io.ObjectOutputStream; -public abstract class HadoopOutputFormatBase implements OutputFormat, FinalizeOnMaster { +public abstract class HadoopOutputFormatBase extends RichOutputFormat implements FinalizeOnMaster { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java index 23c19f7a00243..f3ffd93645aa5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java @@ -26,7 +26,7 @@ import java.util.Map; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.InputTypeConfigurable; @@ -35,7 +35,7 @@ /** * An output format that writes record into collection */ -public class LocalCollectionOutputFormat implements OutputFormat, InputTypeConfigurable { +public class LocalCollectionOutputFormat extends RichOutputFormat implements InputTypeConfigurable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java index d24a3699b4f62..6b67c4673ffb0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java @@ -20,11 +20,11 @@ import java.io.PrintStream; -import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.configuration.Configuration; -public class PrintingOutputFormat implements OutputFormat { +public class PrintingOutputFormat extends RichOutputFormat { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index d291b5a3547d6..39a0a288d051d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -28,6 +29,7 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.execution.CancelTaskException; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.reader.MutableReader; import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader; import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; @@ -35,6 +37,7 @@ import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.util.CloseableInputProvider; +import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext; import org.apache.flink.runtime.operators.util.ReaderIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.plugable.DeserializationDelegate; @@ -109,6 +112,13 @@ public void invoke() throws Exception LOG.debug(getLogString("Starting data sink operator")); } + if(RichOutputFormat.class.isAssignableFrom(this.format.getClass())){ + ((RichOutputFormat) this.format).setRuntimeContext(createRuntimeContext()); + if (LOG.isDebugEnabled()) { + LOG.debug(getLogString("Rich Sink detected. Initializing runtime context.")); + } + } + ExecutionConfig executionConfig; try { ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( @@ -393,4 +403,12 @@ private void initInputReaders() throws Exception { private String getLogString(String message) { return RegularPactTask.constructLogString(message, this.getEnvironment().getTaskName(), this); } + + public DistributedRuntimeUDFContext createRuntimeContext() { + Environment env = getEnvironment(); + + return new DistributedRuntimeUDFContext(env.getTaskName(), env.getNumberOfSubtasks(), + env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(), + env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index df814089ea3cf..5806e778ab662 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -19,19 +19,21 @@ package org.apache.flink.runtime.operators; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplit; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.execution.CancelTaskException; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.operators.chaining.ChainedDriver; import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; +import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; @@ -42,7 +44,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.NoSuchElementException; /** @@ -103,6 +104,13 @@ public void invoke() throws Exception { LOG.debug(getLogString("Starting data source operator")); } + if(RichInputFormat.class.isAssignableFrom(this.format.getClass())){ + ((RichInputFormat) this.format).setRuntimeContext(createRuntimeContext()); + if (LOG.isDebugEnabled()) { + LOG.debug(getLogString("Rich Source detected. Initializing runtime context.")); + } + } + ExecutionConfig executionConfig; try { ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( @@ -295,10 +303,8 @@ private void initOutputs(ClassLoader cl) throws Exception { final AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry(); final AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter(); - Map> accumulatorMap = accumulatorRegistry.getUserMap(); - this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs, - getExecutionConfig(), reporter, accumulatorMap); + getExecutionConfig(), reporter, getEnvironment().getAccumulatorRegistry().getUserMap()); } // ------------------------------------------------------------------------ @@ -377,4 +383,12 @@ public void remove() { } }; } + + public DistributedRuntimeUDFContext createRuntimeContext() { + Environment env = getEnvironment(); + + return new DistributedRuntimeUDFContext(env.getTaskName(), env.getNumberOfSubtasks(), + env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(), + env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap()); + } } diff --git a/flink-staging/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java b/flink-staging/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java index 9c861ed07a15c..ca1eb72d73bb9 100644 --- a/flink-staging/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java +++ b/flink-staging/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.LocatableInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.tuple.Tuple; @@ -43,7 +44,7 @@ * {@link InputFormat} subclass that wraps the access for HTables. * */ -public abstract class TableInputFormat implements InputFormat{ +public abstract class TableInputFormat extends RichInputFormat{ private static final long serialVersionUID = 1L; diff --git a/flink-staging/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java b/flink-staging/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java index f23ac962d59da..859b706b3598d 100644 --- a/flink-staging/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java +++ b/flink-staging/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java @@ -18,7 +18,7 @@ package org.apache.flink.hcatalog; -import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.LocatableInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; @@ -63,7 +63,7 @@ * * @param */ -public abstract class HCatInputFormatBase implements InputFormat, ResultTypeQueryable { +public abstract class HCatInputFormatBase extends RichInputFormat implements ResultTypeQueryable { private static final long serialVersionUID = 1L; diff --git a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index 3cfaeb9390c2b..eb3ac31671ced 100644 --- a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; -import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.configuration.Configuration; @@ -48,7 +48,7 @@ * @see Tuple * @see DriverManager */ -public class JDBCInputFormat implements InputFormat, NonParallelInput { +public class JDBCInputFormat extends RichInputFormat implements NonParallelInput { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class); diff --git a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 67717722e2766..614c5b71cfb6f 100644 --- a/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-staging/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -26,7 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.configuration.Configuration; @@ -38,7 +38,7 @@ * @see Tuple * @see DriverManager */ -public class JDBCOutputFormat implements OutputFormat { +public class JDBCOutputFormat extends RichOutputFormat { private static final long serialVersionUID = 1L; @SuppressWarnings("unused") diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java new file mode 100644 index 0000000000000..cfdc31a0e65ad --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java @@ -0,0 +1,47 @@ +/* + * 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.test.io; + +import org.apache.flink.api.common.operators.util.TestNonRichInputFormat; +import org.apache.flink.api.common.operators.util.TestNonRichOutputFormat; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.test.util.JavaProgramTestBase; + +import static org.junit.Assert.fail; + +/** + * Tests for non rich DataSource and DataSink input output formats being correctly used at runtime. + */ +public class InputOutputITCase extends JavaProgramTestBase { + + @Override + protected void testProgram() throws Exception { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + TestNonRichOutputFormat output = new TestNonRichOutputFormat(); + env.createInput(new TestNonRichInputFormat()).output(output); + try { + env.execute(); + } catch(Exception e){ + // we didn't break anything by making everything rich. + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java new file mode 100644 index 0000000000000..98f19ff43bbc5 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java @@ -0,0 +1,123 @@ +/* + * 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.test.io; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.TextInputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.test.util.JavaProgramTestBase; + +import java.io.IOException; +import java.util.concurrent.ConcurrentLinkedQueue; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for rich DataSource and DataSink input output formats accessing RuntimeContext by + * checking accumulator values. + */ +public class RichInputOutputITCase extends JavaProgramTestBase { + + private String inputPath; + private static ConcurrentLinkedQueue readCalls; + private static ConcurrentLinkedQueue writeCalls; + + @Override + protected void preSubmit() throws Exception { + inputPath = createTempFile("input", "ab\n" + + "cd\n" + + "ef\n"); + } + + @Override + protected void testProgram() throws Exception { + // test verifying the number of records read and written vs the accumulator counts + + readCalls = new ConcurrentLinkedQueue(); + writeCalls = new ConcurrentLinkedQueue(); + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.createInput(new TestInputFormat(new Path(inputPath))).output(new TestOutputFormat()); + + JobExecutionResult result = env.execute(); + Object a = result.getAllAccumulatorResults().get("DATA_SOURCE_ACCUMULATOR"); + Object b = result.getAllAccumulatorResults().get("DATA_SINK_ACCUMULATOR"); + long recordsRead = (Long) a; + long recordsWritten = (Long) b; + assertEquals(recordsRead, readCalls.size()); + assertEquals(recordsWritten, writeCalls.size()); + } + + private static final class TestInputFormat extends TextInputFormat { + private static final long serialVersionUID = 1L; + + private LongCounter counter = new LongCounter(); + + public TestInputFormat(Path filePath) { + super(filePath); + } + + @Override + public void open(FileInputSplit split) throws IOException{ + try{ + getRuntimeContext().addAccumulator("DATA_SOURCE_ACCUMULATOR", counter); + } catch(UnsupportedOperationException e){ + // the accumulator is already added + } + super.open(split); + } + + @Override + public String nextRecord(String reuse) throws IOException{ + readCalls.add(1); + counter.add(1); + return super.nextRecord(reuse); + } + } + + private static final class TestOutputFormat extends RichOutputFormat { + + private LongCounter counter = new LongCounter(); + + @Override + public void configure(Configuration parameters){} + + @Override + public void open(int a, int b){ + try{ + getRuntimeContext().addAccumulator("DATA_SINK_ACCUMULATOR", counter); + } catch(UnsupportedOperationException e){ + // the accumulator is already added + } + } + + @Override + public void close() throws IOException{} + + @Override + public void writeRecord(String record){ + writeCalls.add(1); + counter.add(1); + } + } +} From 4cc8d66fbf7d288581d1912527f54c3d0f1943d7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 13 Aug 2015 17:00:18 +0200 Subject: [PATCH 119/175] [FLINK-2509] [hotfix] Address pull request comments for ClassLoaderUtils Correcting commit, because the wrong commit was pushed earlier. --- .../org/apache/flink/runtime/util/ClassLoaderUtilsTest.java | 3 --- .../org/apache/flink/streaming/api/graph/StreamConfig.java | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java index d5f3c9ebded76..654001df254c4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java @@ -85,8 +85,6 @@ public void testWithURLClassLoader() { assertTrue(info.indexOf(validJar.getAbsolutePath() + "' (valid") > 0); assertTrue(info.indexOf(invalidJar.getAbsolutePath() + "' (invalid JAR") > 0); assertTrue(info.indexOf(nonExisting.getAbsolutePath() + "' (missing") > 0); - - System.out.println(info); } catch (Exception e) { e.printStackTrace(); @@ -107,7 +105,6 @@ public void testWithURLClassLoader() { @Test public void testWithAppClassLoader() { try { - // must return something when invoked with 'null' String result = ClassLoaderUtil.getUserCodeClassLoaderInfo(ClassLoader.getSystemClassLoader()); assertTrue(result.toLowerCase().contains("system classloader")); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index a8486d3cf1a58..4f19db6febcf7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -213,8 +213,8 @@ public T getStreamOperator(ClassLoader cl) { String exceptionMessage = "Cannot load user class: " + e.getMessage() + "\nClassLoader info: " + classLoaderInfo + (loadableDoubleCheck ? - "Class was actually found in classloader - deserialization issue." : - "Class not resolveable through given classloader."); + "\nClass was actually found in classloader - deserialization issue." : + "\nClass not resolvable through given classloader."); throw new StreamTaskException(exceptionMessage); } From 06e2da352fb63f7922f634e6aaf5381d89de57a5 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 13 Aug 2015 16:43:53 +0200 Subject: [PATCH 120/175] [FLINK-2515] [job manager] Checkpoint coordinator triggers checkpoints only when tasks are running. --- .../checkpoint/CheckpointCoordinator.java | 3 +- .../checkpoint/CheckpointCoordinatorTest.java | 54 ++++++++++++++++++- .../CheckpointStateRestoreTest.java | 14 +++-- 3 files changed, 63 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 96941321b96b7..de83ad9f15bb1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -23,6 +23,7 @@ import akka.actor.Props; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; @@ -224,7 +225,7 @@ public boolean triggerCheckpoint(final long timestamp) { ExecutionAttemptID[] triggerIDs = new ExecutionAttemptID[tasksToTrigger.length]; for (int i = 0; i < tasksToTrigger.length; i++) { Execution ee = tasksToTrigger[i].getCurrentExecutionAttempt(); - if (ee != null) { + if (ee != null && ee.getState() == ExecutionState.RUNNING) { triggerIDs[i] = ee.getAttemptId(); } else { LOG.info("Checkpoint triggering task {} is not being executed at the moment. Aborting checkpoint.", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index c02d301b7d0bc..cfe77d36743f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -22,6 +22,7 @@ import static org.mockito.Mockito.*; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -38,7 +39,7 @@ */ public class CheckpointCoordinatorTest { - ClassLoader cl = Thread.currentThread().getContextClassLoader(); + private static final ClassLoader cl = Thread.currentThread().getContextClassLoader(); @Test public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { @@ -82,6 +83,50 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { } } + @Test + public void testCheckpointAbortsIfTriggerTasksAreFinished() { + try { + final JobID jid = new JobID(); + final long timestamp = System.currentTimeMillis(); + + // create some mock Execution vertices that receive the checkpoint trigger messages + final ExecutionAttemptID triggerAttemptID1 = new ExecutionAttemptID(); + final ExecutionAttemptID triggerAttemptID2 = new ExecutionAttemptID(); + ExecutionVertex triggerVertex1 = mockExecutionVertex(triggerAttemptID1); + ExecutionVertex triggerVertex2 = mockExecutionVertex(triggerAttemptID2, ExecutionState.FINISHED); + + // create some mock Execution vertices that need to ack the checkpoint + final ExecutionAttemptID ackAttemptID1 = new ExecutionAttemptID(); + final ExecutionAttemptID ackAttemptID2 = new ExecutionAttemptID(); + ExecutionVertex ackVertex1 = mockExecutionVertex(ackAttemptID1); + ExecutionVertex ackVertex2 = mockExecutionVertex(ackAttemptID2); + + // set up the coordinator and validate the initial state + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, 1, 600000, + new ExecutionVertex[] { triggerVertex1, triggerVertex2 }, + new ExecutionVertex[] { ackVertex1, ackVertex2 }, + new ExecutionVertex[] {}, cl ); + + // nothing should be happening + assertEquals(0, coord.getNumberOfPendingCheckpoints()); + assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); + + // trigger the first checkpoint. this should not succeed + assertFalse(coord.triggerCheckpoint(timestamp)); + + // still, nothing should be happening + assertEquals(0, coord.getNumberOfPendingCheckpoints()); + assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); + + coord.shutdown(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + @Test public void testCheckpointAbortsIfAckTasksAreNotExecuted() { try { @@ -609,10 +654,15 @@ public void handleMessagesForNonExistingCheckpoints() { fail(e.getMessage()); } } - + private static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) { + return mockExecutionVertex(attemptID, ExecutionState.RUNNING); + } + + private static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID, ExecutionState state) { final Execution exec = mock(Execution.class); when(exec.getAttemptId()).thenReturn(attemptID); + when(exec.getState()).thenReturn(state); ExecutionVertex vertex = mock(ExecutionVertex.class); when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index c23aacada9f98..902eb4be3020d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -45,13 +45,13 @@ */ public class CheckpointStateRestoreTest { - ClassLoader cl = Thread.currentThread().getContextClassLoader(); + private static final ClassLoader cl = Thread.currentThread().getContextClassLoader(); @Test public void testSetState() { try { final SerializedValue> serializedState = new SerializedValue>( - new LocalStateHandle(new SerializableObject())); + new LocalStateHandle(new SerializableObject())); final JobID jid = new JobID(); final JobVertexID statefulId = new JobVertexID(); @@ -120,7 +120,7 @@ public void testSetState() { public void testStateOnlyPartiallyAvailable() { try { final SerializedValue> serializedState = new SerializedValue>( - new LocalStateHandle(new SerializableObject())); + new LocalStateHandle(new SerializableObject())); final JobID jid = new JobID(); final JobVertexID statefulId = new JobVertexID(); @@ -208,11 +208,15 @@ public void testNoCheckpointAvailable() { } // ------------------------------------------------------------------------ - + private Execution mockExecution() { + return mockExecution(ExecutionState.RUNNING); + } + + private Execution mockExecution(ExecutionState state) { Execution mock = mock(Execution.class); when(mock.getAttemptId()).thenReturn(new ExecutionAttemptID()); - when(mock.getState()).thenReturn(ExecutionState.CREATED); + when(mock.getState()).thenReturn(state); return mock; } From e1d1bd0a224b32f6a488e400f5f07e4ab4b65869 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 14 Aug 2015 12:16:26 +0200 Subject: [PATCH 121/175] [FLINK-2519] [streaming] Make checkpoint alignment aware of finished partitions. --- .../network/partition/consumer/InputGate.java | 40 +++++ .../streaming/runtime/io/BarrierBuffer.java | 70 +++++--- .../runtime/io/BarrierBufferTest.java | 169 +++++++++++------- .../runtime/io/BarrierTrackerTest.java | 63 ++----- .../streaming/runtime/io/MockInputGate.java | 94 ++++++++++ 5 files changed, 294 insertions(+), 142 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java index 1f42cfa3a3885..f18c7e4d4a22b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java @@ -23,6 +23,46 @@ import java.io.IOException; +/** + * An input gate consumes one or more partitions of a single produced intermediate result. + * + *

    Each intermediate result is partitioned over its producing parallel subtasks; each of these + * partitions is furthermore partitioned into one or more subpartitions. + * + *

    As an example, consider a map-reduce program, where the map operator produces data and the + * reduce operator consumes the produced data. + * + *

    + * +-----+              +---------------------+              +--------+
    + * | Map | = produce => | Intermediate Result | <= consume = | Reduce |
    + * +-----+              +---------------------+              +--------+
    + * 
    + * + *

    When deploying such a program in parallel, the intermediate result will be partitioned over its + * producing parallel subtasks; each of these partitions is furthermore partitioned into one or more + * subpartitions. + * + *

    + *                            Intermediate result
    + *               +-----------------------------------------+
    + *               |                      +----------------+ |              +-----------------------+
    + * +-------+     | +-------------+  +=> | Subpartition 1 | | <=======+=== | Input Gate | Reduce 1 |
    + * | Map 1 | ==> | | Partition 1 | =|   +----------------+ |         |    +-----------------------+
    + * +-------+     | +-------------+  +=> | Subpartition 2 | | <==+    |
    + *               |                      +----------------+ |    |    | Subpartition request
    + *               |                                         |    |    |
    + *               |                      +----------------+ |    |    |
    + * +-------+     | +-------------+  +=> | Subpartition 1 | | <==+====+
    + * | Map 2 | ==> | | Partition 2 | =|   +----------------+ |    |         +-----------------------+
    + * +-------+     | +-------------+  +=> | Subpartition 2 | | <==+======== | Input Gate | Reduce 2 |
    + *               |                      +----------------+ |              +-----------------------+
    + *               +-----------------------------------------+
    + * 
    + * + *

    In the above example, two map subtasks produce the intermediate result in parallel, resulting + * in two partitions (Partition 1 and 2). Each of these partitions is further partitioned into two + * subpartitions -- one for each parallel reduce subtask. + */ public interface InputGate { int getNumberOfInputChannels(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index fd896c9d83697..0bcdb7435ec46 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -21,6 +21,7 @@ import java.util.ArrayDeque; import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; @@ -68,7 +69,10 @@ public class BarrierBuffer implements CheckpointBarrierHandler { private long currentCheckpointId = -1L; /** The number of received barriers (= number of blocked/buffered channels) */ - private long numReceivedBarriers; + private int numBarriersReceived; + + /** The number of already closed channels */ + private int numClosedChannels; /** Flag to indicate whether we have drawn all available input */ private boolean endOfStream; @@ -99,32 +103,38 @@ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedExceptio while (true) { // process buffered BufferOrEvents before grabbing new ones BufferOrEvent next; - if (currentBuffered != null) { + if (currentBuffered == null) { + next = inputGate.getNextBufferOrEvent(); + } + else { next = currentBuffered.getNext(); if (next == null) { - currentBuffered.cleanup(); - currentBuffered = queuedBuffered.pollFirst(); - if (currentBuffered != null) { - currentBuffered.open(); - } + completeBufferedSequence(); return getNextNonBlocked(); } } - else { - next = inputGate.getNextBufferOrEvent(); - } if (next != null) { if (isBlocked(next.getChannelIndex())) { // if the channel is blocked we, we just store the BufferOrEvent bufferSpiller.add(next); } - else if (next.isBuffer() || next.getEvent().getClass() != CheckpointBarrier.class) { + else if (next.isBuffer()) { return next; } - else if (!endOfStream) { - // process barriers only if there is a chance of the checkpoint completing - processBarrier((CheckpointBarrier) next.getEvent(), next.getChannelIndex()); + else if (next.getEvent().getClass() == CheckpointBarrier.class) { + if (!endOfStream) { + // process barriers only if there is a chance of the checkpoint completing + processBarrier((CheckpointBarrier) next.getEvent(), next.getChannelIndex()); + } + } + else { + if (next.getEvent().getClass() == EndOfPartitionEvent.class) { + numClosedChannels++; + // no chance to complete this checkpoint + releaseBlocks(); + } + return next; } } else if (!endOfStream) { @@ -139,10 +149,18 @@ else if (!endOfStream) { } } + private void completeBufferedSequence() throws IOException { + currentBuffered.cleanup(); + currentBuffered = queuedBuffered.pollFirst(); + if (currentBuffered != null) { + currentBuffered.open(); + } + } + private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws IOException { final long barrierId = receivedBarrier.getId(); - if (numReceivedBarriers > 0) { + if (numBarriersReceived > 0) { // subsequent barrier of a checkpoint. if (barrierId == currentCheckpointId) { // regular case @@ -174,7 +192,7 @@ else if (barrierId > currentCheckpointId) { } // check if we have all barriers - if (numReceivedBarriers == totalNumberOfInputChannels) { + if (numBarriersReceived + numClosedChannels == totalNumberOfInputChannels) { if (LOG.isDebugEnabled()) { LOG.debug("Received all barrier, triggering checkpoint {} at {}", receivedBarrier.getId(), receivedBarrier.getTimestamp()); @@ -232,7 +250,7 @@ private boolean isBlocked(int channelIndex) { private void onBarrier(int channelIndex) throws IOException { if (!blockedChannels[channelIndex]) { blockedChannels[channelIndex] = true; - numReceivedBarriers++; + numBarriersReceived++; if (LOG.isDebugEnabled()) { LOG.debug("Received barrier from channel " + channelIndex); @@ -255,7 +273,7 @@ private void releaseBlocks() throws IOException { for (int i = 0; i < blockedChannels.length; i++) { blockedChannels[i] = false; } - numReceivedBarriers = 0; + numBarriersReceived = 0; if (currentBuffered == null) { // common case: no more buffered data @@ -266,13 +284,14 @@ private void releaseBlocks() throws IOException { } else { // uncommon case: buffered data pending - // push back the pending data - queuedBuffered.addFirst(currentBuffered); + // push back the pending data, if we have any - // since we did not fully drain the previous sequence, we need to allocate a new buffer for this one - currentBuffered = bufferSpiller.rollOverWithNewBuffer(); - if (currentBuffered != null) { - currentBuffered.open(); + // since we did not fully drain the previous sequence, we need to allocate a new buffer for this one + BufferSpiller.SpilledBufferOrEventSequence bufferedNow = bufferSpiller.rollOverWithNewBuffer(); + if (bufferedNow != null) { + bufferedNow.open(); + queuedBuffered.addFirst(currentBuffered); + currentBuffered = bufferedNow; } } } @@ -296,6 +315,7 @@ public long getCurrentCheckpointId() { @Override public String toString() { - return String.format("last checkpoint: %d, current barriers: %d", currentCheckpointId, numReceivedBarriers); + return String.format("last checkpoint: %d, current barriers: %d, closed channels: %d", + currentCheckpointId, numBarriersReceived, numClosedChannels); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index dd4d395990226..a95839a1901c3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -19,14 +19,12 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; @@ -35,10 +33,7 @@ import org.junit.Test; import java.io.File; -import java.util.ArrayDeque; import java.util.Arrays; -import java.util.List; -import java.util.Queue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -84,7 +79,7 @@ public void testSingleChannelNoBarriers() { createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); for (BufferOrEvent boe : sequence) { @@ -117,7 +112,7 @@ public void testMultiChannelNoBarriers() { createBuffer(1), createEndOfPartition(1), createBuffer(2), createEndOfPartition(2) }; - MockInputGate gate = new MockInputGate(4, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 4, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); for (BufferOrEvent boe : sequence) { @@ -154,7 +149,7 @@ public void testSingleChannelWithBarriers() { createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -220,7 +215,7 @@ public void testMultiChannelWithBarriers() { createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -313,7 +308,7 @@ public void testMultiChannelTrailingBlockedData() { createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -402,7 +397,7 @@ public void testMultiChannelWithQueuedFutureBarriers() { createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -455,11 +450,15 @@ public void testMultiChannelWithQueuedFutureBarriers() { check(sequence[42], buffer.getNextNonBlocked()); check(sequence[45], buffer.getNextNonBlocked()); check(sequence[46], buffer.getNextNonBlocked()); + + // abort checkpoint 5 (end of partition) + check(sequence[37], buffer.getNextNonBlocked()); + + // start checkpoint 6 alignment check(sequence[47], buffer.getNextNonBlocked()); check(sequence[48], buffer.getNextNonBlocked()); // end of input, emit remainder - check(sequence[37], buffer.getNextNonBlocked()); check(sequence[43], buffer.getNextNonBlocked()); check(sequence[44], buffer.getNextNonBlocked()); @@ -504,7 +503,7 @@ public void testMultiChannelSkippingCheckpoints() { createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -534,13 +533,13 @@ public void testMultiChannelSkippingCheckpoints() { check(sequence[16], buffer.getNextNonBlocked()); check(sequence[19], buffer.getNextNonBlocked()); check(sequence[20], buffer.getNextNonBlocked()); - check(sequence[23], buffer.getNextNonBlocked()); - check(sequence[24], buffer.getNextNonBlocked()); - - // end of input, emit remainder + + // checkpoint 3 aborted (end of partition) check(sequence[18], buffer.getNextNonBlocked()); check(sequence[21], buffer.getNextNonBlocked()); check(sequence[22], buffer.getNextNonBlocked()); + check(sequence[23], buffer.getNextNonBlocked()); + check(sequence[24], buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); @@ -587,7 +586,7 @@ public void testMultiChannelJumpingOverCheckpoint() { createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -620,16 +619,16 @@ public void testMultiChannelJumpingOverCheckpoint() { check(sequence[18], buffer.getNextNonBlocked()); check(sequence[22], buffer.getNextNonBlocked()); - // align remainder + // align checkpoint 4 remainder check(sequence[25], buffer.getNextNonBlocked()); check(sequence[26], buffer.getNextNonBlocked()); - check(sequence[29], buffer.getNextNonBlocked()); - check(sequence[30], buffer.getNextNonBlocked()); - // end of input, emit remainder + // checkpoint 4 aborted (due to end of partition) check(sequence[24], buffer.getNextNonBlocked()); check(sequence[27], buffer.getNextNonBlocked()); check(sequence[28], buffer.getNextNonBlocked()); + check(sequence[29], buffer.getNextNonBlocked()); + check(sequence[30], buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); assertNull(buffer.getNextNonBlocked()); @@ -688,7 +687,7 @@ public void testMultiChannelSkippingCheckpointsViaBlockedInputs() { createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); // checkpoint 1 @@ -759,7 +758,7 @@ public void testEarlyCleanup() { createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(); @@ -795,6 +794,88 @@ public void testEarlyCleanup() { fail(e.getMessage()); } } + + @Test + public void testStartAlignmentWithClosedChannels() { + try { + BufferOrEvent[] sequence = { + // close some channels immediately + createEndOfPartition(2), createEndOfPartition(1), + + // checkpoint without blocked data + createBuffer(0), createBuffer(0), createBuffer(3), + createBarrier(2, 3), createBarrier(2, 0), + + // checkpoint with blocked data + createBuffer(3), createBuffer(0), + createBarrier(3, 3), + createBuffer(3), createBuffer(0), + createBarrier(3, 0), + + // empty checkpoint + createBarrier(4, 0), createBarrier(4, 3), + + // some data, one channel closes + createBuffer(0), createBuffer(0), createBuffer(3), + createEndOfPartition(0), + + // checkpoint on last remaining channel + createBuffer(3), + createBarrier(5, 3), + createBuffer(3), + createEndOfPartition(3) + }; + + MockInputGate gate = new MockInputGate(PAGE_SIZE, 4, Arrays.asList(sequence)); + + BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER); + + // pre checkpoint 2 + check(sequence[0], buffer.getNextNonBlocked()); + check(sequence[1], buffer.getNextNonBlocked()); + check(sequence[2], buffer.getNextNonBlocked()); + check(sequence[3], buffer.getNextNonBlocked()); + check(sequence[4], buffer.getNextNonBlocked()); + + // checkpoint 3 alignment + check(sequence[7], buffer.getNextNonBlocked()); + assertEquals(2L, buffer.getCurrentCheckpointId()); + check(sequence[8], buffer.getNextNonBlocked()); + check(sequence[11], buffer.getNextNonBlocked()); + + // checkpoint 3 buffered + check(sequence[10], buffer.getNextNonBlocked()); + assertEquals(3L, buffer.getCurrentCheckpointId()); + + // after checkpoint 4 + check(sequence[15], buffer.getNextNonBlocked()); + assertEquals(4L, buffer.getCurrentCheckpointId()); + check(sequence[16], buffer.getNextNonBlocked()); + check(sequence[17], buffer.getNextNonBlocked()); + check(sequence[18], buffer.getNextNonBlocked()); + + check(sequence[19], buffer.getNextNonBlocked()); + check(sequence[21], buffer.getNextNonBlocked()); + assertEquals(5L, buffer.getCurrentCheckpointId()); + check(sequence[22], buffer.getNextNonBlocked()); + + assertNull(buffer.getNextNonBlocked()); + assertNull(buffer.getNextNonBlocked()); + + buffer.cleanup(); + + checkNoTempFilesRemain(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testEndOfStreamWhileCheckpoint() { + + } // ------------------------------------------------------------------------ // Utils @@ -846,46 +927,6 @@ private static void checkNoTempFilesRemain() { // Testing Mocks // ------------------------------------------------------------------------ - private static class MockInputGate implements InputGate { - - private final int numChannels; - private final Queue boes; - - public MockInputGate(int numChannels, List boes) { - this.numChannels = numChannels; - this.boes = new ArrayDeque(boes); - } - - @Override - public int getNumberOfInputChannels() { - return numChannels; - } - - @Override - public boolean isFinished() { - return boes.isEmpty(); - } - - @Override - public void requestPartitions() {} - - @Override - public BufferOrEvent getNextBufferOrEvent() { - return boes.poll(); - } - - @Override - public void sendTaskEvent(TaskEvent event) {} - - @Override - public void registerListener(EventListener listener) {} - - @Override - public int getPageSize() { - return PAGE_SIZE; - } - } - private static class ValidatingCheckpointHandler implements EventListener { private long nextExpectedCheckpointId = -1L; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java index fb61633b38081..c6010d6863a08 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java @@ -19,20 +19,15 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.junit.Test; -import java.util.ArrayDeque; import java.util.Arrays; -import java.util.List; -import java.util.Queue; import static org.junit.Assert.*; @@ -41,12 +36,14 @@ */ public class BarrierTrackerTest { + private static final int PAGE_SIZE = 512; + @Test public void testSingleChannelNoBarriers() { try { BufferOrEvent[] sequence = { createBuffer(0), createBuffer(0), createBuffer(0) }; - MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); for (BufferOrEvent boe : sequence) { @@ -70,7 +67,7 @@ public void testMultiChannelNoBarriers() { createBuffer(1), createBuffer(1), createBuffer(2) }; - MockInputGate gate = new MockInputGate(4, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 4, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); for (BufferOrEvent boe : sequence) { @@ -99,7 +96,7 @@ public void testSingleChannelWithBarriers() { createBuffer(0) }; - MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); CheckpointSequenceValidator validator = @@ -134,7 +131,7 @@ public void testSingleChannelWithSkippedBarriers() { createBuffer(0) }; - MockInputGate gate = new MockInputGate(1, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); CheckpointSequenceValidator validator = @@ -178,7 +175,7 @@ public void testMultiChannelWithBarriers() { createBuffer(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); CheckpointSequenceValidator validator = @@ -226,7 +223,7 @@ public void testMultiChannelSkippingCheckpoints() { createBuffer(0) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); CheckpointSequenceValidator validator = @@ -310,7 +307,7 @@ public void testCompleteCheckpointsOnLateBarriers() { createBuffer(1), createBuffer(0), createBuffer(2) }; - MockInputGate gate = new MockInputGate(3, Arrays.asList(sequence)); + MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); CheckpointSequenceValidator validator = @@ -348,47 +345,7 @@ private static BufferOrEvent createBuffer(int channel) { // ------------------------------------------------------------------------ // Testing Mocks // ------------------------------------------------------------------------ - - private static class MockInputGate implements InputGate { - - private final int numChannels; - private final Queue boes; - - public MockInputGate(int numChannels, List boes) { - this.numChannels = numChannels; - this.boes = new ArrayDeque(boes); - } - - @Override - public int getNumberOfInputChannels() { - return numChannels; - } - - @Override - public boolean isFinished() { - return boes.isEmpty(); - } - - @Override - public void requestPartitions() {} - - @Override - public BufferOrEvent getNextBufferOrEvent() { - return boes.poll(); - } - - @Override - public void sendTaskEvent(TaskEvent event) {} - - @Override - public void registerListener(EventListener listener) {} - - @Override - public int getPageSize() { - return 2; - } - } - + private static class CheckpointSequenceValidator implements EventListener { private final long[] checkpointIDs; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java new file mode 100644 index 0000000000000..cb8a058e48cb6 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java @@ -0,0 +1,94 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.runtime.event.TaskEvent; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.util.event.EventListener; + +import java.util.ArrayDeque; +import java.util.List; +import java.util.Queue; + +public class MockInputGate implements InputGate { + + private final int pageSize; + + private final int numChannels; + + private final Queue boes; + + private final boolean[] closed; + + private int closedChannels; + + + public MockInputGate(int pageSize, int numChannels, List boes) { + this.pageSize = pageSize; + this.numChannels = numChannels; + this.boes = new ArrayDeque(boes); + this.closed = new boolean[numChannels]; + } + + @Override + public int getPageSize() { + return pageSize; + } + + @Override + public int getNumberOfInputChannels() { + return numChannels; + } + + @Override + public boolean isFinished() { + return boes.isEmpty(); + } + + @Override + public BufferOrEvent getNextBufferOrEvent() { + BufferOrEvent next = boes.poll(); + if (next == null) { + return null; + } + + int channelIdx = next.getChannelIndex(); + if (closed[channelIdx]) { + throw new RuntimeException("Inconsistent: Channel " + channelIdx + + " has data even though it is already closed."); + } + if (next.isEvent() && next.getEvent() instanceof EndOfPartitionEvent) { + closed[channelIdx] = true; + closedChannels++; + } + return next; + } + + @Override + public void requestPartitions() {} + + @Override + public void sendTaskEvent(TaskEvent event) {} + + @Override + public void registerListener(EventListener listener) {} + +} \ No newline at end of file From 51872d73b83a86603fd1f2e8e481d3cceb755e38 Mon Sep 17 00:00:00 2001 From: Nezih Yigitbasi Date: Fri, 14 Aug 2015 01:34:11 -0700 Subject: [PATCH 122/175] [FLINK-2517] [docs] Minor fix to streaming guide This closes #1013 --- docs/apis/streaming_guide.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index b20482e5b84ea..e375dabb51905 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -1597,7 +1597,7 @@ A class providing an interface for sending data to Kafka. The followings have to be provided for the `KafkaSink(…)` constructor in order: -1. Zookeeper hostname +1. Broker address (in hostname:port format, can be a comma separated list) 2. The topic name 3. Serialization schema @@ -1606,12 +1606,12 @@ Example:

    {% highlight java %} -stream.addSink(new KafkaSink("localhost:2181", "test", new SimpleStringSchema())); +stream.addSink(new KafkaSink("localhost:9092", "test", new SimpleStringSchema())); {% endhighlight %}
    {% highlight scala %} -stream.addSink(new KafkaSink[String]("localhost:2181", "test", new SimpleStringSchema)) +stream.addSink(new KafkaSink[String]("localhost:9092", "test", new SimpleStringSchema)) {% endhighlight %}
    @@ -1633,7 +1633,7 @@ public KafkaSink(String zookeeperAddress, String topicId, Properties producerCon -If this constructor is used, the user needs to make sure to set the broker with the "metadata.broker.list" property. Also the serializer configuration should be left default, the serialization should be set via SerializationSchema. +If this constructor is used, the user needs to make sure to set the broker(s) with the "metadata.broker.list" property. Also the serializer configuration should be left default, the serialization should be set via SerializationSchema. More about Kafka can be found [here](https://kafka.apache.org/documentation.html). From 6b92fc11cb5aad0b5f3409aaf1865b546bcbd0a7 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 13 Aug 2015 15:15:56 +0100 Subject: [PATCH 123/175] Fixing pull-request comments. --- .../accumulators/UserAccumulators.java | 9 +- .../executiongraph/ExecutionGraph.java | 444 ++++++++---------- .../apache/flink/runtime/akka/AkkaUtils.scala | 6 +- .../flink/runtime/jobmanager/JobManager.scala | 25 +- 4 files changed, 208 insertions(+), 276 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java index 243e5163eb99f..4c6dcdd2e45cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java @@ -33,9 +33,10 @@ public class UserAccumulators implements java.io.Serializable { private final SerializedValue>> smallUserAccumulators; /** - * In case some accumulators do not fit in an Akka message payload, we store them in the blobCache and put - * in the snapshot only the mapping between the name of the accumulator, and its blobKey - * in the cache. This list holds exactly this mapping. + * In case some accumulators do not fit in an Akka message payload, we store them + * in the blobCache. This list holds the mapping between the name of the accumulator, + * and its blobKey in the cache is put in the snapshot. This list holds exactly this + * mapping. * */ private final Map> largeUserAccumulatorBlobs; @@ -66,7 +67,7 @@ public UserAccumulators(SerializedValue>> userAccu * These accumulators had to be stored in the BlobCache and their blobKeys are returned here. * @return the maping between accumulator and its blobKeys. */ - public Map> getLargeAccumulatorBlobKeys() { + public Map> getLargeUserAccumulatorBlobKeys() { if(smallUserAccumulators != null) { return Collections.emptyMap(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 9f9c7e9b65c2c..c3cb2b54a9af9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -24,8 +24,8 @@ import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.accumulators.UserAccumulators; import org.apache.flink.runtime.blob.BlobKey; @@ -78,24 +78,24 @@ * The execution graph is the central data structure that coordinates the distributed * execution of a data flow. It keeps representations of each parallel task, each * intermediate result, and the communication between them. - *

    + * * The execution graph consists of the following constructs: *

      - *
    • The {@link ExecutionJobVertex} represents one vertex from the JobGraph (usually one operation like - * "map" or "join") during execution. It holds the aggregated state of all parallel subtasks. - * The ExecutionJobVertex is identified inside the graph by the {@link JobVertexID}, which it takes - * from the JobGraph's corresponding JobVertex.
    • - *
    • The {@link ExecutionVertex} represents one parallel subtask. For each ExecutionJobVertex, there are - * as many ExecutionVertices as the parallelism. The ExecutionVertex is identified by - * the ExecutionJobVertex and the number of the parallel subtask
    • - *
    • The {@link Execution} is one attempt to execute a ExecutionVertex. There may be multiple Executions - * for the ExecutionVertex, in case of a failure, or in the case where some data needs to be recomputed - * because it is no longer available when requested by later operations. An Execution is always - * identified by an {@link ExecutionAttemptID}. All messages between the JobManager and the TaskManager - * about deployment of tasks and updates in the task status always use the ExecutionAttemptID to - * address the message receiver.
    • + *
    • The {@link ExecutionJobVertex} represents one vertex from the JobGraph (usually one operation like + * "map" or "join") during execution. It holds the aggregated state of all parallel subtasks. + * The ExecutionJobVertex is identified inside the graph by the {@link JobVertexID}, which it takes + * from the JobGraph's corresponding JobVertex.
    • + *
    • The {@link ExecutionVertex} represents one parallel subtask. For each ExecutionJobVertex, there are + * as many ExecutionVertices as the parallelism. The ExecutionVertex is identified by + * the ExecutionJobVertex and the number of the parallel subtask
    • + *
    • The {@link Execution} is one attempt to execute a ExecutionVertex. There may be multiple Executions + * for the ExecutionVertex, in case of a failure, or in the case where some data needs to be recomputed + * because it is no longer available when requested by later operations. An Execution is always + * identified by an {@link ExecutionAttemptID}. All messages between the JobManager and the TaskManager + * about deployment of tasks and updates in the task status always use the ExecutionAttemptID to + * address the message receiver.
    • *
    - *

    + * *

    The ExecutionGraph implements {@link java.io.Serializable}, because it can be archived by * sending it to an archive actor via an actor message. The execution graph does contain some * non-serializable fields. These fields are not required in the archived form and are cleared @@ -108,189 +108,108 @@ public class ExecutionGraph implements Serializable { private static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ExecutionGraph.class, JobStatus.class, "state"); - /** - * The log object used for debugging. - */ + /** The log object used for debugging. */ static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class); private static final int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1; // -------------------------------------------------------------------------------------------- - /** - * The lock used to secure all access to mutable fields, especially the tracking of progress - * within the job. - */ + /** The lock used to secure all access to mutable fields, especially the tracking of progress + * within the job. */ private final SerializableObject progressLock = new SerializableObject(); - /** - * The ID of the job this graph has been built for. - */ + /** The ID of the job this graph has been built for. */ private final JobID jobID; - /** - * The name of the original job graph. - */ + /** The name of the original job graph. */ private final String jobName; - /** - * The job configuration that was originally attached to the JobGraph. - */ + /** The job configuration that was originally attached to the JobGraph. */ private final Configuration jobConfiguration; - /** - * All job vertices that are part of this graph - */ + /** All job vertices that are part of this graph */ private final ConcurrentHashMap tasks; - /** - * All vertices, in the order in which they were created - **/ + /** All vertices, in the order in which they were created **/ private final List verticesInCreationOrder; - /** - * All intermediate results that are part of this graph - */ + /** All intermediate results that are part of this graph */ private final ConcurrentHashMap intermediateResults; - /** - * The currently executed tasks, for callbacks - */ + /** The currently executed tasks, for callbacks */ private final ConcurrentHashMap currentExecutions; - /** - * Updates the accumulators during the runtime of a job. Final accumulator results are transferred - * through the UpdateTaskExecutionState message. - * - * @param snapshot The serialized flink and user-defined accumulators - */ - public void updateAccumulators(AccumulatorSnapshot snapshot) { - Map> flinkAccumulators; - UserAccumulators userAccumulators = snapshot.getUserAccumulators(); - try { - flinkAccumulators = snapshot.deserializeFlinkAccumulators(); - - ExecutionAttemptID execID = snapshot.getExecutionAttemptID(); - Execution execution = currentExecutions.get(execID); - if (execution != null) { - Map> largeAccumulators = - userAccumulators.getLargeAccumulatorBlobKeys(); - Map> smallAccumulators = - userAccumulators.deserializeSmallUserAccumulators(userClassLoader); - - execution.setAccumulators(flinkAccumulators, smallAccumulators, largeAccumulators); - } else { - LOG.warn("Received accumulator result for unknown execution {}.", execID); - } - } catch (Exception e) { - LOG.error("Cannot update accumulators for job " + jobID, e); - } - } - - /** - * A list of all libraries required during the job execution. Libraries have to be stored - * inside the BlobService and are referenced via the BLOB keys. - */ + /** A list of all libraries required during the job execution. Libraries have to be stored + * inside the BlobService and are referenced via the BLOB keys. */ private final List requiredJarFiles; - /** - * Listeners that receive messages when the entire job switches it status (such as from - * RUNNING to FINISHED) - */ + /** Listeners that receive messages when the entire job switches it status (such as from + * RUNNING to FINISHED) */ private final List jobStatusListenerActors; - /** - * Listeners that receive messages whenever a single task execution changes its status - */ + /** Listeners that receive messages whenever a single task execution changes its status */ private final List executionListenerActors; - /** - * Timestamps (in milliseconds as returned by {@code System.currentTimeMillis()} when + /** Timestamps (in milliseconds as returned by {@code System.currentTimeMillis()} when * the execution graph transitioned into a certain state. The index into this array is the * ordinal of the enum value, i.e. the timestamp when the graph went into state "RUNNING" is - * at {@code stateTimestamps[RUNNING.ordinal()]}. - */ + * at {@code stateTimestamps[RUNNING.ordinal()]}. */ private final long[] stateTimestamps; - /** - * The timeout for all messages that require a response/acknowledgement - */ + /** The timeout for all messages that require a response/acknowledgement */ private final FiniteDuration timeout; // ------ Configuration of the Execution ------- - /** - * The number of times failed executions should be retried. - */ + /** The number of times failed executions should be retried. */ private int numberOfRetriesLeft; - /** - * The delay that the system should wait before restarting failed executions. - */ + /** The delay that the system should wait before restarting failed executions. */ private long delayBeforeRetrying; - /** - * Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able - * to deploy them immediately. - */ + /** Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able + * to deploy them immediately. */ private boolean allowQueuedScheduling = false; - /** - * The mode of scheduling. Decides how to select the initial set of tasks to be deployed. + /** The mode of scheduling. Decides how to select the initial set of tasks to be deployed. * May indicate to deploy all sources, or to deploy everything, or to deploy via backtracking - * from results than need to be materialized. - */ + * from results than need to be materialized. */ private ScheduleMode scheduleMode = ScheduleMode.FROM_SOURCES; - /** - * Flag that indicate whether the executed dataflow should be periodically snapshotted - */ + /** Flag that indicate whether the executed dataflow should be periodically snapshotted */ private boolean snapshotCheckpointsEnabled; // ------ Execution status and progress. These values are volatile, and accessed under the lock ------- - /** - * Current status of the job execution - */ + /** Current status of the job execution */ private volatile JobStatus state = JobStatus.CREATED; - /** - * The exception that caused the job to fail. This is set to the first root exception - * that was not recoverable and triggered job failure - */ + /** The exception that caused the job to fail. This is set to the first root exception + * that was not recoverable and triggered job failure */ private volatile Throwable failureCause; - /** - * The number of job vertices that have reached a terminal state - */ + /** The number of job vertices that have reached a terminal state */ private volatile int numFinishedJobVertices; // ------ Fields that are relevant to the execution and need to be cleared before archiving ------- - /** - * The scheduler to use for scheduling new tasks as they are needed - */ + /** The scheduler to use for scheduling new tasks as they are needed */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private Scheduler scheduler; - /** - * The classloader for the user code. Needed for calls into user code classes - */ + /** The classloader for the user code. Needed for calls into user code classes */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private ClassLoader userClassLoader; - /** - * The coordinator for checkpoints, if snapshot checkpoints are enabled - */ + /** The coordinator for checkpoints, if snapshot checkpoints are enabled */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private CheckpointCoordinator checkpointCoordinator; - /** - * The execution context which is used to execute futures. - */ + /** The execution context which is used to execute futures. */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private ExecutionContext executionContext; @@ -313,13 +232,13 @@ public void updateAccumulators(AccumulatorSnapshot snapshot) { Configuration jobConfig, FiniteDuration timeout) { this( - executionContext, - jobId, - jobName, - jobConfig, - timeout, - new ArrayList(), - ExecutionGraph.class.getClassLoader() + executionContext, + jobId, + jobName, + jobConfig, + timeout, + new ArrayList(), + ExecutionGraph.class.getClassLoader() ); } @@ -348,7 +267,7 @@ public ExecutionGraph( this.verticesInCreationOrder = new ArrayList(); this.currentExecutions = new ConcurrentHashMap(); - this.jobStatusListenerActors = new CopyOnWriteArrayList(); + this.jobStatusListenerActors = new CopyOnWriteArrayList(); this.executionListenerActors = new CopyOnWriteArrayList(); this.stateTimestamps = new long[JobStatus.values().length]; @@ -471,7 +390,8 @@ private ExecutionVertex[] collectExecutionVertices(List jobV throw new IllegalArgumentException("Can only use ExecutionJobVertices of this ExecutionGraph"); } return jv.getTaskVertices(); - } else { + } + else { ArrayList all = new ArrayList(); for (ExecutionJobVertex jv : jobVertices) { if (jv.getGraph() != this) { @@ -489,7 +409,6 @@ private ExecutionVertex[] collectExecutionVertices(List jobV /** * Returns a list of BLOB keys referring to the JAR files required to run this job - * * @return list of BLOB keys referring to the JAR files required to run this job */ public List getRequiredJarFiles() { @@ -605,10 +524,9 @@ public ExecutionContext getExecutionContext() { /** * Gets the internal flink accumulator map of maps which contains some metrics. - * * @return A map of accumulators for every executed task. */ - public Map>> getFlinkAccumulators() { + public Map>> getFlinkAccumulators() { Map>> flinkAccumulators = new HashMap>>(); @@ -621,35 +539,17 @@ public ExecutionContext getExecutionContext() { } /** - * This works as cache for already merged accumulators, as, in some cases, - * we do not want to remerge accumulators as this may lead to duplicate entries. + * This works as cache for already merged accumulators. If accumulators are already + * merged, we do not want to re-merge them, as this may lead to duplicate entries. */ private Map> mergedSmallUserAccumulators; /** * Merges all accumulator results from the tasks previously executed in the Executions. - * * @return The accumulator map */ public Map> aggregateSmallUserAccumulators() { - return aggregateSmallUserAccumulators(true); - } - - /** - * Merges all accumulator results from the tasks previously executed in the Executions. - * If reaggregate is set to false, then no aggregation is performed, and - * the cache merge result is returned. Otherwise accumulators are merged. - * - * @param reaggregate true if we want to aggregate accumulators, - * false otherwise. - * @return The accumulator map - */ - public Map> aggregateSmallUserAccumulators(boolean reaggregate) { - if (!reaggregate) { - return mergedSmallUserAccumulators; - } this.mergedSmallUserAccumulators = new HashMap>(); - for (ExecutionVertex vertex : getAllExecutionVertices()) { Map> next = vertex.getCurrentExecutionAttempt().getSmallUserAccumulators(); if (next != null) { @@ -660,43 +560,51 @@ public ExecutionContext getExecutionContext() { } /** - * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the - * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, - * thus had to be sent through the BlobCache. + * Merges the partial accumulators referring to the same global accumulator received + * from the tasks, and serializes the objects of the accumulators (not only the content as the + * {@link #getSmallAccumulatorsContentSerialized()}. This means that the actual + * objects are serialized, thus merging can still be applied after deserialization. + * Each of the partial accumulators contains the partial result produced by each task, + * for that specific accumulator. * - * @return The accumulator map + * @return The accumulator map with serialized accumulator objects. + * @throws IOException */ - public Map> aggregateLargeUserAccumulatorBlobKeys() { - Map> largeUserAccumulatorRefs = new HashMap>(); + public Map> getSmallAccumulatorsSerialized() throws IOException { + Map> accumulatorMap = (mergedSmallUserAccumulators != null) ? + mergedSmallUserAccumulators : aggregateSmallUserAccumulators(); - for (ExecutionVertex vertex : getAllExecutionVertices()) { - Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); - mergeLargeUserAccumulatorBlobKeys(largeUserAccumulatorRefs, next); + Map> result = new HashMap>(); + for (Map.Entry> entry : accumulatorMap.entrySet()) { + result.put(entry.getKey(), new SerializedValue(entry.getValue())); } - return largeUserAccumulatorRefs; + return result; } /** - * Adds new blobKeys referring to blobs of large accumulators to the already existing ones. - * These refer to blobs in the blobCache holding accumulators (results of tasks) that did not - * fit in an akka frame, thus had to be sent through the BlobCache. + * Merges the partial accumulators referring to the same global accumulator received + * from the tasks, and serializes the content of the final accumulator, i.e. the result. + * This is unlike the {@link #getSmallAccumulatorsSerialized()} which serialized the whole + * accumulator object. Each of the partial accumulators contains the partial result + * produced by each task, for that specific accumulator. * - * @param target the initial blobKey map - * @param toMerge the new keys to add to the initial map - * @return The resulting accumulator map + * @return The accumulator map with serialized accumulator values. + * @throws IOException */ - public Map> addLargeUserAccumulatorBlobKeys( - Map> target, Map> toMerge) { - if (target == null) { - target = new HashMap>(); + public Map> getSmallAccumulatorsContentSerialized() throws IOException { + Map> accumulatorMap = (mergedSmallUserAccumulators != null) ? + mergedSmallUserAccumulators : aggregateSmallUserAccumulators(); + + Map> result = new HashMap>(); + for (Map.Entry> entry : accumulatorMap.entrySet()) { + result.put(entry.getKey(), new SerializedValue(entry.getValue().getLocalValue())); } - mergeLargeUserAccumulatorBlobKeys(target, toMerge); - return target; + return result; } - private void mergeLargeUserAccumulatorBlobKeys( + private void aggregateLargeUserAccumulatorBlobKeys( Map> target, Map> toMerge) { - if (toMerge == null || toMerge.isEmpty()) { + if (target == null || toMerge == null || toMerge.isEmpty()) { return; } @@ -711,76 +619,48 @@ private void mergeLargeUserAccumulatorBlobKeys( } /** - * Merges the partial accumulators referring to the same global accumulator received from the tasks, - * and serializes the final result. Each of the partial accumulators contains the partial result - * produced by each task, for that specific accumulator. + * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the + * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, + * thus had to be sent through the BlobCache. * - * @return The accumulator map with serialized accumulator values. - * @throws IOException + * @return The accumulator map */ - public Map> getSmallAccumulatorsContentSerialized() throws IOException { - Map> accumulatorMap = aggregateSmallUserAccumulators(true); + public Map> aggregateLargeUserAccumulatorBlobKeys() { + Map> largeUserAccumulatorRefs = new HashMap>(); - Map> result = new HashMap>(); - for (Map.Entry> entry : accumulatorMap.entrySet()) { - result.put(entry.getKey(), new SerializedValue(entry.getValue().getLocalValue())); + for (ExecutionVertex vertex : getAllExecutionVertices()) { + Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); + aggregateLargeUserAccumulatorBlobKeys(largeUserAccumulatorRefs, next); } - return result; + return largeUserAccumulatorRefs; } /** - * Serializes the objects of the accumulators (not only the content as the - * {@link #getSmallAccumulatorsContentSerialized()}. This means that the actual - * objects are serialized, thus merging can still be applied after deserialization. - * This method assumes that partial accumulators received from the tasks that refer to - * the same global accumulator have already been merged. + * Adds new blobKeys pointing to blobs of large accumulators in the BlobCache, to the list + * of already received BlobKeys. These refer to blobs in the blobCache holding accumulators + * (results of tasks) that did not fit in an akka framesize, thus had to be sent through + * the BlobCache. * - * @return The accumulator map with serialized accumulator objects. - * @throws IOException + * @param target the initial blobKey map + * @param toMerge the new keys to add to the initial map + * @return The resulting accumulator map */ - public Map> getSmallAccumulatorsSerialized() throws IOException { - Map> accumulatorMap = aggregateSmallUserAccumulators(false); + public Map> addLargeUserAccumulatorBlobKeys( + Map> target, Map> toMerge) { - Map> result = new HashMap>(); - for (Map.Entry> entry : accumulatorMap.entrySet()) { - result.put(entry.getKey(), new SerializedValue(entry.getValue())); + if (target == null) { + target = new HashMap>(); } - return result; + aggregateLargeUserAccumulatorBlobKeys(target, toMerge); + return target; } -// /** -// * Merges and serializes all accumulator results from the tasks previously executed in -// * the Executions. If onlyContent is set to true, then the Accumulators are -// * merged and the content of the resulting Accumulator is serialized and returned. In other -// * case, the result is assumed to be merged, so no additional merging is performed (as this -// * could lead to duplicate entries), and the whole accumulator object is serialized and -// * returned. -// * @param onlyContent true if we want to aggregate accumulators and serialize just -// * the content of the result, false if (partial) accumulators -// * are already merged (so no additional merging is required), and we want the -// * whole object serialized. -// * @return The accumulator map -// */ -// private Map> serializeAccumulators(boolean onlyContent) throws IOException { -// -// Map> accumulatorMap = aggregateSmallUserAccumulators(onlyContent); -// -// Map> result = new HashMap>(); -// for (Map.Entry> entry : accumulatorMap.entrySet()) { -// Object toSerialize = onlyContent ? entry.getValue().getLocalValue() : entry.getValue(); -// result.put(entry.getKey(), new SerializedValue(toSerialize)); -// } -// -// return result; -// } - /** * Returns the a stringified version of the user-defined accumulators. - * * @return an Array containing the StringifiedAccumulatorResult objects */ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { - Map> smallAccumulatorMap = aggregateSmallUserAccumulators(); + Map> smallAccumulatorMap = mergedSmallUserAccumulators; Map> largeAccumulatorMap = aggregateLargeUserAccumulatorBlobKeys(); // get the total number of (unique) accumulators @@ -890,8 +770,11 @@ public void scheduleForExecution(Scheduler scheduler) throws JobException { case BACKTRACKING: // go back from vertices that need computation to the ones we need to run throw new JobException("BACKTRACKING is currently not supported as schedule mode."); + default: + throw new JobException("Schedule mode is invalid."); } - } else { + } + else { throw new IllegalStateException("Job may only be scheduled from state " + JobStatus.CREATED); } } @@ -907,7 +790,8 @@ public void cancel() { } return; } - } else { + } + else { // no need to treat other states return; } @@ -919,7 +803,8 @@ public void fail(Throwable t) { JobStatus current = state; if (current == JobStatus.FAILED || current == JobStatus.FAILING) { return; - } else if (transitionState(current, JobStatus.FAILING, t)) { + } + else if (transitionState(current, JobStatus.FAILING, t)) { this.failureCause = t; if (!verticesInCreationOrder.isEmpty()) { @@ -974,7 +859,8 @@ public void restart() { } scheduleForExecution(scheduler); - } catch (Throwable t) { + } + catch (Throwable t) { fail(t); } } @@ -988,7 +874,7 @@ public void prepareForArchiving() { } // "unpack" execution config before we throw away the usercode classloader. try { - executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(jobConfiguration, ExecutionConfig.CONFIG_KEY, userClassLoader); + executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(jobConfiguration, ExecutionConfig.CONFIG_KEY,userClassLoader); } catch (Exception e) { LOG.warn("Error deserializing the execution config while archiving the execution graph", e); } @@ -1015,7 +901,6 @@ public ExecutionConfig getExecutionConfig() { /** * For testing: This waits until the job execution has finished. - * * @throws InterruptedException */ public void waitUntilFinished() throws InterruptedException { @@ -1039,7 +924,8 @@ private boolean transitionState(JobStatus current, JobStatus newState, Throwable stateTimestamps[newState.ordinal()] = System.currentTimeMillis(); notifyJobStatusChange(newState, error); return true; - } else { + } + else { return false; } } @@ -1064,12 +950,14 @@ void jobVertexInFinalState(ExecutionJobVertex ev) { postRunCleanup(); break; } - } else if (current == JobStatus.CANCELLING) { + } + else if (current == JobStatus.CANCELLING) { if (transitionState(current, JobStatus.CANCELED)) { postRunCleanup(); break; } - } else if (current == JobStatus.FAILING) { + } + else if (current == JobStatus.FAILING) { if (numberOfRetriesLeft > 0 && transitionState(current, JobStatus.RESTARTING)) { numberOfRetriesLeft--; future(new Callable() { @@ -1078,7 +966,8 @@ public Object call() throws Exception { try { LOG.info("Delaying retry of job execution for {} ms ...", delayBeforeRetrying); Thread.sleep(delayBeforeRetrying); - } catch (InterruptedException e) { + } + catch(InterruptedException e){ // should only happen on shutdown } restart(); @@ -1086,11 +975,13 @@ public Object call() throws Exception { } }, executionContext); break; - } else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, failureCause)) { + } + else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, failureCause)) { postRunCleanup(); break; } - } else { + } + else { fail(new Exception("ExecutionGraph went into final state from state " + current)); } } @@ -1109,7 +1000,8 @@ private void postRunCleanup() { if (coord != null) { coord.shutdown(); } - } catch (Exception e) { + } + catch (Exception e) { LOG.error("Error while cleaning up after execution", e); } } @@ -1120,7 +1012,6 @@ private void postRunCleanup() { /** * Updates the state of the Task and sets the final accumulator results. - * * @param state * @return */ @@ -1141,7 +1032,7 @@ public boolean updateState(TaskExecutionState state) { try { flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - largeAccumulators = userAccumulators.getLargeAccumulatorBlobKeys(); + largeAccumulators = userAccumulators.getLargeUserAccumulatorBlobKeys(); smallAccumulators = userAccumulators.deserializeSmallUserAccumulators(userClassLoader); } catch (Exception e) { // Exceptions would be thrown in the future here @@ -1162,7 +1053,8 @@ public boolean updateState(TaskExecutionState state) { attempt.fail(new Exception("TaskManager sent illegal state update: " + state.getExecutionState())); return false; } - } else { + } + else { return false; } } @@ -1174,7 +1066,8 @@ public void scheduleOrUpdateConsumers(ResultPartitionID partitionId) { if (execution == null) { fail(new IllegalStateException("Cannot find execution for execution ID " + partitionId.getPartitionId())); - } else if (execution.getVertex() == null) { + } + else if (execution.getVertex() == null){ fail(new IllegalStateException("Execution with execution ID " + partitionId.getPartitionId() + " has no vertex assigned.")); } else { @@ -1201,6 +1094,34 @@ void deregisterExecution(Execution exec) { } } + /** + * Updates the accumulators during the runtime of a job. Final accumulator results are transferred + * through the UpdateTaskExecutionState message. + * @param accumulatorSnapshot The serialized flink and user-defined accumulators + */ + public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { + Map> flinkAccumulators; + UserAccumulators userAccumulators = accumulatorSnapshot.getUserAccumulators(); + try { + flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); + + ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); + Execution execution = currentExecutions.get(execID); + if (execution != null) { + Map> largeAccumulators = + userAccumulators.getLargeUserAccumulatorBlobKeys(); + Map> smallAccumulators = + userAccumulators.deserializeSmallUserAccumulators(userClassLoader); + + execution.setAccumulators(flinkAccumulators, smallAccumulators, largeAccumulators); + } else { + LOG.warn("Received accumulator result for unknown execution {}.", execID); + } + } catch (Exception e) { + LOG.error("Cannot update accumulators for job " + jobID, e); + } + } + // -------------------------------------------------------------------------------------------- // Listeners & Observers // -------------------------------------------------------------------------------------------- @@ -1223,23 +1144,24 @@ private void notifyJobStatusChange(JobStatus newState, Throwable error) { ExecutionGraphMessages.JobStatusChanged message = new ExecutionGraphMessages.JobStatusChanged(jobID, newState, System.currentTimeMillis(), error); - for (ActorGateway listener : jobStatusListenerActors) { + for (ActorGateway listener: jobStatusListenerActors) { listener.tell(message); } } } void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionAttemptID executionID, ExecutionState - newExecutionState, Throwable error) { + newExecutionState, Throwable error) + { ExecutionJobVertex vertex = getJobVertex(vertexId); if (executionListenerActors.size() > 0) { String message = error == null ? null : ExceptionUtils.stringifyException(error); ExecutionGraphMessages.ExecutionStateChanged actorMessage = - new ExecutionGraphMessages.ExecutionStateChanged(jobID, vertexId, vertex.getJobVertex().getName(), - vertex.getParallelism(), subtask, - executionID, newExecutionState, - System.currentTimeMillis(), message); + new ExecutionGraphMessages.ExecutionStateChanged(jobID, vertexId, vertex.getJobVertex().getName(), + vertex.getParallelism(), subtask, + executionID, newExecutionState, + System.currentTimeMillis(), message); for (ActorGateway listener : executionListenerActors) { listener.tell(actorMessage); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 405c4140ffb6e..2432a173a5764 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -427,10 +427,14 @@ object AkkaUtils { ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d]", "").toLong def getLargeAccumulatorThreshold(config: Configuration): Long = { - val threshold = config.getDouble( + var threshold = config.getDouble( ConfigConstants.AKKA_FRAMESIZE_OVERSIZED_THRESHOLD, ConfigConstants.DEFAULT_AKKA_FRAMESIZE_OVERSIZED_THRESHOLD) + if(threshold < 0.0 || threshold > 0.8) { + threshold = ConfigConstants.AKKA_FRAMESIZE_OVERSIZED_THRESHOLD.toDouble + } + (threshold * getFramesize(config)).toLong } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 040a7c948ae4a..a1a2f140dfed1 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -19,14 +19,16 @@ package org.apache.flink.runtime.jobmanager import java.io.{File, IOException} -import java.lang.reflect.{Constructor, InvocationTargetException} +import java.lang.reflect.{InvocationTargetException, Constructor} import java.net.InetSocketAddress -import java.util.{Collections, UUID} +import java.util.{UUID, Collections} -import _root_.akka.pattern.ask import akka.actor.Status.{Failure, Success} import akka.actor._ +import _root_.akka.pattern.ask + import grizzled.slf4j.Logger + import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner @@ -295,8 +297,8 @@ class JobManager( case ex: Exception => log.error(s"Could not serialize the next input split of " + s"class ${nextInputSplit.getClass}.", ex) - vertex.fail(new RuntimeException("Could not serialize the next input " + - "split of class " + nextInputSplit.getClass + ".", ex)) + vertex.fail(new RuntimeException("Could not serialize the next input split " + + "of class " + nextInputSplit.getClass + ".", ex)) null } @@ -323,8 +325,8 @@ class JobManager( currentJobs.get(jobID) match { case Some((executionGraph, jobInfo)) => executionGraph.getJobName - log.info(s"Status of job $jobID (${executionGraph.getJobName}) " + - "changed to $newJobStatus.", error) + log.info(s"Status of job $jobID (${executionGraph.getJobName}) changed to $newJobStatus.", + error) if (newJobStatus.isTerminalState) { jobInfo.end = timeStamp @@ -356,6 +358,7 @@ class JobManager( val totalSize: Long = smallAccumulatorResults.asScala.map(_._2.getSizeInBytes).sum if (totalSize > AkkaUtils.getLargeAccumulatorThreshold(jobConfig)) { + // given that the client is going to do the final merging, we serialize and // store the accumulator objects, not only the content val serializedSmallAccumulators = executionGraph.getSmallAccumulatorsSerialized @@ -364,6 +367,8 @@ class JobManager( val newBlobKeys = LargeAccumulatorHelper.storeSerializedAccumulatorsToBlobCache( getBlobCacheServerAddress, serializedSmallAccumulators) + // given that the result is going to be sent through the BlobCache, clear its + // in-memory version. smallAccumulatorResults.clear() // and update the blobKeys to send to the client. @@ -814,9 +819,9 @@ class JobManager( archive.forward(message) } } catch { - case e: Exception => - log.error("Cannot serialize accumulator result.", e) - sender() ! decorateMessage(AccumulatorResultsErroneous(jobID, e)) + case e: Exception => + log.error("Cannot serialize accumulator result.", e) + sender() ! decorateMessage(AccumulatorResultsErroneous(jobID, e)) } case RequestAccumulatorResultsStringified(jobId) => From 21a0c94baafd77297c8eb88367fc8caaac43d8ee Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 14 Aug 2015 14:13:03 +0200 Subject: [PATCH 124/175] [FLINK-2393] [docs] Updates docs to describe switch between exactly-once and at-least-once --- docs/apis/streaming_guide.md | 27 ++++++++++++++++++++++++-- docs/internals/stream_checkpointing.md | 19 +++++++++++++++++- 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index e375dabb51905..35f6147794627 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -1274,6 +1274,29 @@ Rich functions provide, in addition to the user-defined function (`map()`, `redu [Back to top](#top) + + +Fault Tolerance +---------------- + +Flink has a checkpointig mechanism that recovers streaming jobs after failues. The checkpointing mechanism requires a *persistent* or *durable* source that +can be asked for prior records again (Apache Kafka is a good example of a durable source). + +The checkpointing mechanism stores the progress in the source as well as the user-defined state (see [Stateful Computation](#Stateful_computation)) +consistently to provide *exactly once* processing guarantees. + +To enable checkpointing, call `enableCheckpointing(n)` on the `StreamExecutionEnvironment`, where *n* is the checkpoint interval, in milliseconds. + +Other parameters for checkpointing include: + + - *Number of retries*: The `setNumberOfExecutionRerties()` method defines how many times the job is restarted after a failure. When checkpointing is activated, but this value is not explicitly set, the job is restarted infinitely often. + - *exactly-once vs. at-least-once*: You can optionally pass a mode to the `enableCheckpointing(n)` method to choose between the two guarantee levels. Exactly-once is preferrable for most applications. At-least-once may be relevant for certain super-low-latency (consistently few milliseconds) applications. + +The [docs on streaming fault tolerance](../internals/stream_checkpointing.html) describe in detail the technique behind Flink's streaming fault tolerance mechanism. + +[Back to top](#top) + + Stateful computation ------------ @@ -1488,9 +1511,9 @@ env.genereateSequence(1,10).map(myMap).setBufferTimeout(timeoutMillis) To maximise the throughput the user can call `setBufferTimeout(-1)` which will remove the timeout and buffers will only be flushed when they are full. To minimise latency, set the timeout to a value close to 0 (for example 5 or 10 ms). Theoretically, a buffer timeout of 0 will cause all output to be flushed when produced, but this setting should be avoided, because it can cause severe performance degradation. - [Back to top](#top) - + + Stream connectors ---------------- diff --git a/docs/internals/stream_checkpointing.md b/docs/internals/stream_checkpointing.md index 8428911de3c8e..27eae6b0298fb 100644 --- a/docs/internals/stream_checkpointing.md +++ b/docs/internals/stream_checkpointing.md @@ -30,7 +30,8 @@ This document describes Flink' fault tolerance mechanism for streaming data flow Apache Flink offers a fault tolerance mechanism to consistently recover the state of data streaming applications. The mechanism ensures that even in the presence of failures, the program's state will eventually reflect every -record from the data stream **exactly once**. +record from the data stream **exactly once**. Note that there is a switch to *downgrade* the guarantees to *at least once* +(described below). The fault tolerance mechanism continuously draws snapshots of the distributed streaming data flow. For streaming applications with small state, these snapshots are very light-weight and can be drawn frequently without impacting the performance much. @@ -113,6 +114,22 @@ The resulting snapshot now contains: Illustration of the Checkpointing Mechanism + +### Exactly Once vs. At Least Once + +The alignment step may add latency to the streaming program. Usually, this extra latency is in the order of a few milliseconds, but we have seen cases where the latency +of some outliers increased noticeably. For applications that require consistenty super low latencies (few milliseconds) for all records, Flink has a switch to skip the +stream alignment during a checkpoint. Checkpoint snapshots are still drawn as soon as an operator has seen the checkpoint barrier from each input. + +When the alignment is skipped, an operator keeps processing all inputs, even after some checkpoint barriers for checkpoint *n* arrived. That way, the operator also processes +elements that belong to checkpoint *n+1* before the state snapshot for checkpoint *n* was taken. +On a restore, these records will occur as duplicates, because they are both included in the state snapshot of checkpoint *n*, and will be replayed as part +of the data after checkoint *n*. + +*NOTE*: Alignment happens only for operators wih multiple predecessors (joins) as well as operators with multiple senders (after a stream repartitionging/shuffle). +Because of that, dataflows with only embarassingly parallel streaming operations (`map()`, `flatMap()`, `filter()`, ...) actually give *exactly once* guarantees even +in *at least once* mode. + Create basic utils (config, etc) and load operators + * +----> operator specific init() + * + * -- restoreState() + * + * -- invoke() + * | + * +----> open operators() + * +----> run() + * +----> close operators() + * +----> common cleanup + * +----> operator specific cleanup() + * + * + * @param + * @param + */ public abstract class StreamTask> extends AbstractInvokable implements OperatorStateCarrier>, CheckpointedOperator, CheckpointNotificationOperator { private static final Logger LOG = LoggerFactory.getLogger(StreamTask.class); - protected final Object checkpointLock = new Object(); + + private final Object checkpointLock = new Object(); + + private final EventListener checkpointBarrierListener; + + protected final List contexts; + protected StreamingRuntimeContext headContext; + protected StreamConfig configuration; + protected ClassLoader userClassLoader; + protected OutputHandler outputHandler; protected O streamOperator; protected boolean hasChainedOperators; - // needs to be initialized to true, so that early cancel() before invoke() behaves correctly - protected volatile boolean isRunning = true; - - protected List contexts; - - protected StreamingRuntimeContext headContext; - - protected ClassLoader userClassLoader; + /** Flag to mark the task "in operation", in which case check + * needs to be initialized to true, so that early cancel() before invoke() behaves correctly */ + private volatile boolean isRunning; + + // ------------------------------------------------------------------------ - private EventListener checkpointBarrierListener; - public StreamTask() { - streamOperator = null; checkpointBarrierListener = new CheckpointBarrierListener(); contexts = new ArrayList(); } + // ------------------------------------------------------------------------ + // Life cycle methods for specific implementations + // ------------------------------------------------------------------------ + + protected abstract void init() throws Exception; + + protected abstract void run() throws Exception; + + protected abstract void cleanup() throws Exception; + + protected abstract void cancelTask() throws Exception; + + // ------------------------------------------------------------------------ + // Core work methods of the Stream Task + // ------------------------------------------------------------------------ + @Override - public void registerInputOutput() { - this.userClassLoader = getUserCodeClassLoader(); - this.configuration = new StreamConfig(getTaskConfiguration()); + public final void registerInputOutput() throws Exception { + LOG.debug("Begin initialization for {}", getName()); + + userClassLoader = getUserCodeClassLoader(); + configuration = new StreamConfig(getTaskConfiguration()); streamOperator = configuration.getStreamOperator(userClassLoader); // Create and register Accumulators - Environment env = getEnvironment(); - AccumulatorRegistry accumulatorRegistry = env.getAccumulatorRegistry(); + AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry(); Map> accumulatorMap = accumulatorRegistry.getUserMap(); AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter(); @@ -108,69 +151,76 @@ public void registerInputOutput() { } hasChainedOperators = outputHandler.getChainedOperators().size() != 1; + + // operator specific initialization + init(); + + LOG.debug("Finish initialization for {}", getName()); } + + @Override + public final void invoke() throws Exception { + LOG.debug("Invoking {}", getName()); + + boolean operatorOpen = false; + try { + openAllOperators(); + operatorOpen = true; + + // let the task do its work + isRunning = true; + run(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Finished task {}", getName()); + } - public String getName() { - return getEnvironment().getTaskName(); - } - - public StreamingRuntimeContext createRuntimeContext(StreamConfig conf, Map> accumulatorMap) { - Environment env = getEnvironment(); - String operatorName = conf.getStreamOperator(userClassLoader).getClass().getSimpleName(); - - KeySelector statePartitioner = conf.getStatePartitioner(userClassLoader); - - return new StreamingRuntimeContext(operatorName, env, getUserCodeClassLoader(), - getExecutionConfig(), statePartitioner, getStateHandleProvider(), accumulatorMap); - } - - private StateHandleProvider getStateHandleProvider() { - - StateHandleProvider provider = configuration - .getStateHandleProvider(userClassLoader); - - // If the user did not specify a provider in the program we try to get it from the config - if (provider == null) { - String backendName = GlobalConfiguration.getString(ConfigConstants.STATE_BACKEND, - ConfigConstants.DEFAULT_STATE_BACKEND).toUpperCase(); - - StateBackend backend; - + // this is part of the main logic, so if this fails, the task is considered failed + closeAllOperators(); + operatorOpen = false; + + // make sure all data if flushed + outputHandler.flushOutputs(); + } + finally { + this.isRunning = false; + try { - backend = StateBackend.valueOf(backendName); - } catch (Exception e) { - throw new RuntimeException(backendName + " is not a valid state backend.\nSupported backends: jobmanager, filesystem."); + if (operatorOpen) { + // we came here in a failure + closeAllOperators(); + } } - - switch (backend) { - case JOBMANAGER: - LOG.info("State backend for state checkpoints is set to jobmanager."); - return new LocalStateHandle.LocalStateHandleProvider(); - case FILESYSTEM: - String checkpointDir = GlobalConfiguration.getString(ConfigConstants.STATE_BACKEND_FS_DIR, null); - if (checkpointDir != null) { - LOG.info("State backend for state checkpoints is set to filesystem with directory: " - + checkpointDir); - return FileStateHandle.createProvider(checkpointDir); - } else { - throw new RuntimeException( - "For filesystem checkpointing, a checkpoint directory needs to be specified.\nFor example: \"state.backend.dir: hdfs://checkpoints\""); - } - default: - throw new RuntimeException("Backend " + backend + " is not supported yet."); + catch (Throwable t) { + LOG.error("Error closing stream operators after an exception.", t); + } + finally { + // we must! perform this cleanup + + // release the output resources + if (outputHandler != null) { + outputHandler.releaseOutputs(); + } - } else { - LOG.info("Using user defined state backend for streaming checkpoitns."); - return provider; + // release this operator's resources + try { + cleanup(); + } + catch (Throwable t) { + LOG.error("Error during cleanup of stream task."); + } + } } } - - private enum StateBackend { - JOBMANAGER, FILESYSTEM + + @Override + public final void cancel() throws Exception { + isRunning = false; + cancelTask(); } - - protected void openOperator() throws Exception { + + private void openAllOperators() throws Exception { for (StreamOperator operator : outputHandler.getChainedOperators()) { if (operator != null) { operator.open(getTaskConfiguration()); @@ -178,7 +228,7 @@ protected void openOperator() throws Exception { } } - protected void closeOperator() throws Exception { + private void closeAllOperators() throws Exception { // We need to close them first to last, since upstream operators in the chain might emit // elements in their close methods. for (int i = outputHandler.getChainedOperators().size()-1; i >= 0; i--) { @@ -189,19 +239,20 @@ protected void closeOperator() throws Exception { } } - protected void clearBuffers() throws IOException { - if (outputHandler != null) { - outputHandler.clearWriters(); - } - } + // ------------------------------------------------------------------------ + // Access to properties and utilities + // ------------------------------------------------------------------------ - @Override - public void cancel() { - this.isRunning = false; + /** + * Gets the name of the task, in the form "taskname (2/5)". + * @return The name of the task. + */ + public String getName() { + return getEnvironment().getTaskNameWithSubtasks(); } - public EventListener getCheckpointBarrierListener() { - return this.checkpointBarrierListener; + public Object getCheckpointLock() { + return checkpointLock; } // ------------------------------------------------------------------------ @@ -212,8 +263,9 @@ public EventListener getCheckpointBarrierListener() { @Override public void setInitialState(StateHandle stateHandle) throws Exception { - // We retrieve end restore the states for the chained oeprators. - List, Map>> chainedStates = (List, Map>>) stateHandle.getState(); + // We retrieve end restore the states for the chained operators. + List, Map>> chainedStates = + (List, Map>>) stateHandle.getState(); // We restore all stateful operators for (int i = 0; i < chainedStates.size(); i++) { @@ -224,7 +276,6 @@ public void setInitialState(StateHandle stateHandle) throws Except ((StatefulStreamOperator) chainedOperator).restoreInitialState(state); } } - } @Override @@ -235,10 +286,9 @@ public void triggerCheckpoint(long checkpointId, long timestamp) throws Exceptio synchronized (checkpointLock) { if (isRunning) { try { - - - // We wrap the states of the chained operators in a list, marking non-stateful oeprators with null - List, Map>> chainedStates = new ArrayList, Map>>(); + // We wrap the states of the chained operators in a list, marking non-stateful operators with null + List, Map>> chainedStates + = new ArrayList, Map>>(); // A wrapper handle is created for the List of statehandles WrapperStateHandle stateHandle; @@ -278,35 +328,89 @@ public void triggerCheckpoint(long checkpointId, long timestamp) throws Exceptio } } } - } - - @SuppressWarnings("rawtypes") + @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { synchronized (checkpointLock) { - for (StreamOperator chainedOperator : outputHandler.getChainedOperators()) { if (chainedOperator instanceof StatefulStreamOperator) { - ((StatefulStreamOperator) chainedOperator).notifyCheckpointComplete(checkpointId); + ((StatefulStreamOperator) chainedOperator).notifyCheckpointComplete(checkpointId); } } + } + } + + // ------------------------------------------------------------------------ + // State backend + // ------------------------------------------------------------------------ + + private StateHandleProvider getStateHandleProvider() { + StateHandleProvider provider = configuration.getStateHandleProvider(userClassLoader); + + // If the user did not specify a provider in the program we try to get it from the config + if (provider == null) { + String backendName = GlobalConfiguration.getString(ConfigConstants.STATE_BACKEND, + ConfigConstants.DEFAULT_STATE_BACKEND).toUpperCase(); + + StateBackend backend; + + try { + backend = StateBackend.valueOf(backendName); + } catch (Exception e) { + throw new RuntimeException(backendName + " is not a valid state backend.\nSupported backends: jobmanager, filesystem."); + } + + switch (backend) { + case JOBMANAGER: + LOG.info("State backend for state checkpoints is set to jobmanager."); + return new LocalStateHandle.LocalStateHandleProvider(); + case FILESYSTEM: + String checkpointDir = GlobalConfiguration.getString(ConfigConstants.STATE_BACKEND_FS_DIR, null); + if (checkpointDir != null) { + LOG.info("State backend for state checkpoints is set to filesystem with directory: " + + checkpointDir); + return FileStateHandle.createProvider(checkpointDir); + } else { + throw new RuntimeException( + "For filesystem checkpointing, a checkpoint directory needs to be specified.\nFor example: \"state.backend.dir: hdfs://checkpoints\""); + } + default: + throw new RuntimeException("Backend " + backend + " is not supported yet."); + } + } else { + LOG.info("Using user defined state backend for streaming checkpoitns."); + return provider; } } + private enum StateBackend { + JOBMANAGER, FILESYSTEM + } // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ + public StreamingRuntimeContext createRuntimeContext(StreamConfig conf, Map> accumulatorMap) { + KeySelector statePartitioner = conf.getStatePartitioner(userClassLoader); + + return new StreamingRuntimeContext(getEnvironment(), getExecutionConfig(), + statePartitioner, getStateHandleProvider(), accumulatorMap); + } + @Override public String toString() { - return getEnvironment().getTaskNameWithSubtasks(); + return getName(); } // ------------------------------------------------------------------------ + public EventListener getCheckpointBarrierListener() { + return this.checkpointBarrierListener; + } + private class CheckpointBarrierListener implements EventListener { @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java index 8c354bee4dcb4..2ca286260a935 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java @@ -45,28 +45,32 @@ * Implementation of the {@link RuntimeContext}, created by runtime stream UDF * operators. */ -@SuppressWarnings("rawtypes") public class StreamingRuntimeContext extends RuntimeUDFContext { private final Environment env; - private final Map states; - private final List partitionedStates; + private final Map> states; + private final List> partitionedStates; private final KeySelector statePartitioner; private final StateHandleProvider provider; - private final ClassLoader cl; - + + @SuppressWarnings("unchecked") - public StreamingRuntimeContext(String name, Environment env, ClassLoader userCodeClassLoader, - ExecutionConfig executionConfig, KeySelector statePartitioner, - StateHandleProvider provider, Map> accumulatorMap) { - super(name, env.getNumberOfSubtasks(), env.getIndexInSubtaskGroup(), userCodeClassLoader, - executionConfig, env.getDistributedCacheEntries(), accumulatorMap); + public StreamingRuntimeContext( + Environment env, + ExecutionConfig executionConfig, + KeySelector statePartitioner, + StateHandleProvider provider, + Map> accumulatorMap) { + + super(env.getTaskName(), env.getNumberOfSubtasks(), env.getIndexInSubtaskGroup(), + env.getUserClassLoader(), executionConfig, + env.getDistributedCacheEntries(), accumulatorMap); + this.env = env; this.statePartitioner = statePartitioner; - this.states = new HashMap(); - this.partitionedStates = new LinkedList(); + this.states = new HashMap<>(); + this.partitionedStates = new LinkedList<>(); this.provider = (StateHandleProvider) provider; - this.cl = userCodeClassLoader; } /** @@ -121,14 +125,14 @@ public OperatorState getOperatorState(String name, S public StreamOperatorState getState(String name, boolean partitioned) { // Try fetching state from the map - StreamOperatorState state = states.get(name); + StreamOperatorState state = states.get(name); if (state == null) { // If not found, create empty state and add to the map state = createRawState(partitioned); states.put(name, state); // We keep a reference to all partitioned states for registering input if (state instanceof PartitionedStreamOperatorState) { - partitionedStates.add((PartitionedStreamOperatorState) state); + partitionedStates.add((PartitionedStreamOperatorState) state); } } return state; @@ -139,11 +143,11 @@ public OperatorState getOperatorState(String name, S * * @return An empty operator state. */ - @SuppressWarnings("unchecked") - public StreamOperatorState createRawState(boolean partitioned) { + @SuppressWarnings({"rawtypes", "unchecked"}) + public StreamOperatorState createRawState(boolean partitioned) { if (partitioned) { if (statePartitioner != null) { - return new PartitionedStreamOperatorState(provider, statePartitioner, cl); + return new PartitionedStreamOperatorState(provider, statePartitioner, getUserCodeClassLoader()); } else { throw new RuntimeException( "Partitioned state can only be used with KeyedDataStreams."); @@ -158,7 +162,7 @@ public StreamOperatorState createRawState(boolean partitioned) { * * @return All the states for the underlying operator. */ - public Map getOperatorStates() { + public Map> getOperatorStates() { return states; } @@ -169,7 +173,7 @@ public Map getOperatorStates() { * @param nextRecord * Next input of the operator. */ - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "rawtypes"}) public void setNextInput(StreamRecord nextRecord) { if (statePartitioner != null) { for (PartitionedStreamOperatorState state : partitionedStates) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index 8cf5a405a885e..5d0497d91ca5a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -17,7 +17,6 @@ package org.apache.flink.streaming.runtime.tasks; -import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -27,114 +26,65 @@ import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class TwoInputStreamTask extends StreamTask> { - private static final Logger LOG = LoggerFactory.getLogger(TwoInputStreamTask.class); - private StreamTwoInputProcessor inputProcessor; + + private volatile boolean running = true; @Override - public void registerInputOutput() { - try { - super.registerInputOutput(); + public void init() throws Exception { + TypeSerializer inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); + TypeSerializer inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); - TypeSerializer inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); - TypeSerializer inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); + int numberOfInputs = configuration.getNumberOfInputs(); - int numberOfInputs = configuration.getNumberOfInputs(); + ArrayList inputList1 = new ArrayList(); + ArrayList inputList2 = new ArrayList(); - ArrayList inputList1 = new ArrayList(); - ArrayList inputList2 = new ArrayList(); + List inEdges = configuration.getInPhysicalEdges(userClassLoader); - List inEdges = configuration.getInPhysicalEdges(userClassLoader); - - for (int i = 0; i < numberOfInputs; i++) { - int inputType = inEdges.get(i).getTypeNumber(); - InputGate reader = getEnvironment().getInputGate(i); - switch (inputType) { - case 1: - inputList1.add(reader); - break; - case 2: - inputList2.add(reader); - break; - default: - throw new RuntimeException("Invalid input type number: " + inputType); - } + for (int i = 0; i < numberOfInputs; i++) { + int inputType = inEdges.get(i).getTypeNumber(); + InputGate reader = getEnvironment().getInputGate(i); + switch (inputType) { + case 1: + inputList1.add(reader); + break; + case 2: + inputList2.add(reader); + break; + default: + throw new RuntimeException("Invalid input type number: " + inputType); } - - this.inputProcessor = new StreamTwoInputProcessor(inputList1, inputList2, - inputDeserializer1, inputDeserializer2, - getCheckpointBarrierListener(), - configuration.getCheckpointMode(), - getEnvironment().getIOManager(), - getExecutionConfig().areTimestampsEnabled()); - - // make sure that stream tasks report their I/O statistics - AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); - AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); - this.inputProcessor.setReporter(reporter); - } - catch (Exception e) { - throw new RuntimeException("Failed to initialize stream operator: " + e.getMessage(), e); } + + this.inputProcessor = new StreamTwoInputProcessor(inputList1, inputList2, + inputDeserializer1, inputDeserializer2, + getCheckpointBarrierListener(), + configuration.getCheckpointMode(), + getEnvironment().getIOManager(), + getExecutionConfig().areTimestampsEnabled()); + + // make sure that stream tasks report their I/O statistics + AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); + AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); + this.inputProcessor.setReporter(reporter); } @Override - public void invoke() throws Exception { - boolean operatorOpen = false; - - if (LOG.isDebugEnabled()) { - LOG.debug("Task {} invoked", getName()); - } - - try { - - openOperator(); - operatorOpen = true; - - while (inputProcessor.processInput(streamOperator)) { - // do nothing, just keep processing - } - - closeOperator(); - operatorOpen = false; - - if (LOG.isDebugEnabled()) { - LOG.debug("Task {} invocation finished", getName()); - } - - } - catch (Exception e) { - LOG.error(getEnvironment().getTaskNameWithSubtasks() + " failed", e); - - if (operatorOpen) { - try { - closeOperator(); - } - catch (Throwable t) { - LOG.warn("Exception while closing operator.", t); - } - } - - throw e; - } - finally { - this.isRunning = false; - // Cleanup - outputHandler.flushOutputs(); - clearBuffers(); - } - + protected void run() throws Exception { + while (running && inputProcessor.processInput(streamOperator)); } @Override - public void clearBuffers() throws IOException { - super.clearBuffers(); - inputProcessor.clearBuffers(); + protected void cleanup() throws Exception { inputProcessor.cleanup(); } + + @Override + protected void cancelTask() { + running = false; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java index 6ca38b7ffe442..60db798ddcbf1 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java @@ -141,9 +141,12 @@ private StreamMap createOperatorWithContext(List output KeySelector partitioner, byte[] serializedState) throws Exception { final List outputList = output; - StreamingRuntimeContext context = new StreamingRuntimeContext("MockTask", new MockEnvironment(3 * 1024 * 1024, - new MockInputSplitProvider(), 1024), null, new ExecutionConfig(), partitioner, - new LocalStateHandleProvider(), new HashMap>()); + StreamingRuntimeContext context = new StreamingRuntimeContext( + new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024), + new ExecutionConfig(), + partitioner, + new LocalStateHandleProvider(), + new HashMap>()); StreamMap op = new StreamMap(new StatefulMapper()); @@ -217,14 +220,15 @@ public void open(Configuration conf) throws IOException { } } - @SuppressWarnings({ "rawtypes", "unchecked" }) + @SuppressWarnings("unchecked") @Override public void close() throws Exception { - Map states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates(); + Map> states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates(); PartitionedStreamOperatorState groupCounter = (PartitionedStreamOperatorState) states.get("groupCounter"); for (Entry count : groupCounter.getPartitionedState().entrySet()) { Integer key = (Integer) count.getKey(); Integer expected = key < 3 ? 2 : 1; + assertEquals(new MutableInt(expected), count.getValue()); } } @@ -257,11 +261,12 @@ public void open(Configuration conf) throws IOException { groupCounter = getRuntimeContext().getOperatorState("groupCounter", 0, true); } - @SuppressWarnings({ "rawtypes", "unchecked" }) + @SuppressWarnings("unchecked") @Override public void close() throws Exception { - Map states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates(); - PartitionedStreamOperatorState groupCounter = (PartitionedStreamOperatorState) states.get("groupCounter"); + Map> states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates(); + PartitionedStreamOperatorState groupCounter = + (PartitionedStreamOperatorState) states.get("groupCounter"); for (Entry count : groupCounter.getPartitionedState().entrySet()) { Integer key = (Integer) count.getKey(); Integer expected = key < 3 ? 2 : 1; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java index f07e3a5960108..7cc19588447bb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java @@ -85,12 +85,7 @@ public void testPropagateAsyncFlushError() { } finally { if (testWriter != null) { - try { - testWriter.close(); - } - catch (IOException e) { - // ignore in tests - } + testWriter.close(); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 296324a7d6866..4c6957b488288 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -168,7 +168,6 @@ public void testWatermarkForwarding() throws Exception { * This test verifies that checkpoint barriers are correctly forwarded. */ @Test - @SuppressWarnings("unchecked") public void testCheckpointBarriers() throws Exception { final OneInputStreamTask mapTask = new OneInputStreamTask(); final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); @@ -226,7 +225,6 @@ public void testCheckpointBarriers() throws Exception { * then all inputs receive barriers from a later checkpoint. */ @Test - @SuppressWarnings("unchecked") public void testOvertakingCheckpointBarriers() throws Exception { final OneInputStreamTask mapTask = new OneInputStreamTask(); final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); @@ -282,7 +280,6 @@ public void testOvertakingCheckpointBarriers() throws Exception { testHarness.waitForInputProcessing(); - testHarness.endInput(); testHarness.waitForTaskCompletion(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java index 4f07fdb234dd5..7fb8ba35ddd6e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java @@ -62,7 +62,6 @@ public class OneInputStreamTaskTestHarness extends StreamTaskTestHarnes */ public OneInputStreamTaskTestHarness(OneInputStreamTask task, int numInputGates, - int numInputChannelsPerGate, TypeInformation inputType, TypeInformation outputType) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index 0f372cbdd90a8..987d0bc791cc9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -83,7 +83,7 @@ public class StreamTaskTestHarness { private ConcurrentLinkedQueue outputList; - protected Thread taskThread; + protected TaskThread taskThread; // These don't get initialized, the one-input/two-input specific test harnesses // must initialize these if they want to simulate input. We have them here so that all the @@ -161,32 +161,19 @@ public void invoke() throws Exception { task.registerInputOutput(); - taskThread = new Thread(new Runnable() { - @Override - public void run() { - - - - try { - task.invoke(); - shutdownIOManager(); - shutdownMemoryManager(); - } catch (Exception e) { - throw new RuntimeException(e); - } - - } - }); - + taskThread = new TaskThread(task); taskThread.start(); } - public void waitForTaskCompletion() throws InterruptedException { + public void waitForTaskCompletion() throws Exception { if (taskThread == null) { throw new IllegalStateException("Task thread was not started."); } taskThread.join(); + if (taskThread.getError() != null) { + throw new Exception("error in task", taskThread.getError()); + } } /** @@ -300,5 +287,36 @@ public void endInput() { inputGates[i].endInput(); } } + + // ------------------------------------------------------------------------ + + private class TaskThread extends Thread { + + private final AbstractInvokable task; + + private volatile Throwable error; + + + TaskThread(AbstractInvokable task) { + super("Task Thread"); + this.task = task; + } + + @Override + public void run() { + try { + task.invoke(); + shutdownIOManager(); + shutdownMemoryManager(); + } + catch (Throwable t) { + this.error = t; + } + } + + public Throwable getError() { + return error; + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java index 45ae88f5a0a5b..f9b0b090ce93e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java @@ -57,9 +57,11 @@ public Output> getOutput() { public static List createAndExecute(OneInputStreamOperator operator, List inputs) { MockContext mockContext = new MockContext(inputs); - StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext("MockTask", - new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), null, - new ExecutionConfig(), null, null, new HashMap>()); + StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext( + new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024), + new ExecutionConfig(), + null, null, + new HashMap>()); operator.setup(mockContext.output, runtimeContext); try { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index 6652fde09cced..f404d01378fc5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -60,9 +60,7 @@ public OneInputStreamOperatorTestHarness(OneInputStreamOperator operato executionConfig = new ExecutionConfig(); StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext( - "MockTwoInputTask", - new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), - getClass().getClassLoader(), + new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024), executionConfig, null, new LocalStateHandle.LocalStateHandleProvider(), diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java index 2d7f6b5776b5e..711dd41c4f97a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java @@ -42,8 +42,13 @@ public class SourceFunctionUtil { public static List runSourceFunction(SourceFunction sourceFunction) throws Exception { List outputs = new ArrayList(); if (sourceFunction instanceof RichFunction) { - RuntimeContext runtimeContext = new StreamingRuntimeContext("MockTask", new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), null, - new ExecutionConfig(), null, new LocalStateHandle.LocalStateHandleProvider(), new HashMap>()); + RuntimeContext runtimeContext = new StreamingRuntimeContext( + new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024), + new ExecutionConfig(), + null, + new LocalStateHandle.LocalStateHandleProvider(), + new HashMap>()); + ((RichFunction) sourceFunction).setRuntimeContext(runtimeContext); ((RichFunction) sourceFunction).open(new Configuration()); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java index a9ebd0bcee0e0..09db1f4f305b2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java @@ -66,7 +66,7 @@ public class StreamingMultipleProgramsTestBase extends TestBaseUtils { // ------------------------------------------------------------------------ public StreamingMultipleProgramsTestBase() { - TestStreamEnvironment clusterEnv = new TestStreamEnvironment(cluster, 4); + TestStreamEnvironment clusterEnv = new TestStreamEnvironment(cluster, DEFAULT_PARALLELISM); clusterEnv.setAsContext(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java index 1e8b5c62dcc39..21e2e1e6ffd59 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util; import org.apache.flink.api.common.ExecutionConfig; @@ -33,7 +34,6 @@ import java.io.Serializable; import java.util.HashMap; -import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -60,9 +60,7 @@ public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator o executionConfig = new ExecutionConfig(); StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext( - "MockTwoInputTask", - new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), - getClass().getClassLoader(), + new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024), new ExecutionConfig(), null, new LocalStateHandle.LocalStateHandleProvider(), From 3fcc04ab3583e14d9d0acd1e29adee900738ffde Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 16 Aug 2015 16:52:16 +0200 Subject: [PATCH 140/175] [tests] Reinforce StateCheckpoinedITCase to make sure actual checkpointing has happened before a failure. --- .../checkpointing/StateCheckpoinedITCase.java | 62 ++++++++++++------- .../StreamFaultToleranceTestBase.java | 20 +----- 2 files changed, 40 insertions(+), 42 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java index 072086b09224b..2c2f2b48166f9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java @@ -22,30 +22,24 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.OperatorState; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.Collector; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; import java.io.IOException; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.Random; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * A simple test that runs a streaming topology with checkpointing enabled. @@ -53,6 +47,10 @@ * The test triggers a failure after a while and verifies that, after completion, the * state defined with either the {@link OperatorState} or the {@link Checkpointed} * interface reflects the "exactly once" semantics. + * + * The test throttles the input until at least two checkpoints are completed, to make sure that + * the recovery does not fall back to "square one" (which would naturally lead to correct + * results without testing the checkpointing). */ @SuppressWarnings("serial") public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase { @@ -63,17 +61,24 @@ public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase { * Runs the following program: * *
    -	 *     [ (source)->(filter)->(map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
    +	 *     [ (source)->(filter)] -> [ (map) -> (map) ] -> [ (groupBy/reduce)->(sink) ]
     	 * 
    */ @Override public void testProgram(StreamExecutionEnvironment env) { assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); + final long failurePosMin = (long) (0.4 * NUM_STRINGS / PARALLELISM); + final long failurePosMax = (long) (0.7 * NUM_STRINGS / PARALLELISM); + + final long failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); stream - // -------------- first vertex, chained to the source ---------------- + // first vertex, chained to the source + // this filter throttles the flow until at least one checkpoint + // is complete, to make sure this program does not run without .filter(new StringRichFilterFunction()) // -------------- seconds vertex - one-to-one connected ---------------- @@ -83,12 +88,16 @@ public void testProgram(StreamExecutionEnvironment env) { // -------------- third vertex - reducer and the sink ---------------- .partitionByHash("prefix") - .flatMap(new OnceFailingAggregator(NUM_STRINGS)) + .flatMap(new OnceFailingAggregator(failurePos)) .addSink(new ValidatingSink()); } @Override public void postSubmit() { + + assertTrue("Test inconclusive: failure occurred before first checkpoint", + OnceFailingAggregator.wasCheckpointedBeforeFailure); + long filterSum = 0; for (long l : StringRichFilterFunction.counts) { filterSum += l; @@ -189,14 +198,15 @@ public void restoreState(Integer state) { } } - private static class StringRichFilterFunction extends RichFilterFunction implements Checkpointed { + private static class StringRichFilterFunction extends RichFilterFunction + implements Checkpointed { static final long[] counts = new long[PARALLELISM]; - + private long count; - + @Override - public boolean filter(String value) { + public boolean filter(String value) throws Exception { count++; return value.length() < 100; // should be always true } @@ -271,35 +281,34 @@ public void close() throws IOException { } private static class OnceFailingAggregator extends RichFlatMapFunction - implements Checkpointed> { + implements Checkpointed>, CheckpointNotifier { + static boolean wasCheckpointedBeforeFailure = false; + private static volatile boolean hasFailed = false; private final HashMap aggregationMap = new HashMap(); - private final long numElements; - private long failurePos; private long count; + private boolean wasCheckpointed; + - OnceFailingAggregator(long numElements) { - this.numElements = numElements; + OnceFailingAggregator(long failurePos) { + this.failurePos = failurePos; } @Override public void open(Configuration parameters) { - long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); - long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); - - failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; count = 0; } @Override public void flatMap(PrefixCount value, Collector out) throws Exception { count++; - if (!hasFailed && count >= failurePos) { + if (!hasFailed && count >= failurePos && getRuntimeContext().getIndexOfThisSubtask() == 1) { + wasCheckpointedBeforeFailure = wasCheckpointed; hasFailed = true; throw new Exception("Test Failure"); } @@ -324,6 +333,11 @@ public HashMap snapshotState(long checkpointId, long checkp public void restoreState(HashMap state) { aggregationMap.putAll(state); } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + this.wasCheckpointed = true; + } } private static class ValidatingSink extends RichSinkFunction diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index 29933156d7d03..8920cf2fa2077 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -18,38 +18,22 @@ package org.apache.flink.test.checkpointing; - -import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.state.OperatorState; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.checkpoint.Checkpointed; -import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.test.util.ForkableFlinkMiniCluster; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; import java.io.Serializable; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - /** * Test base for fault tolerant streaming programs */ -@SuppressWarnings("serial") public abstract class StreamFaultToleranceTestBase { protected static final int NUM_TASK_MANAGERS = 2; @@ -127,6 +111,7 @@ public void runCheckpointedProgram() { // Frequently used utilities // -------------------------------------------------------------------------------------------- + @SuppressWarnings("serial") public static class PrefixCount implements Serializable { public String prefix; @@ -146,5 +131,4 @@ public String toString() { return prefix + " / " + value; } } - } From 2f0412f163f4d37605188c8cc763111e0b51f0dc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 14 Aug 2015 15:06:06 +0200 Subject: [PATCH 141/175] [FLINK-2521] [tests] Adds TestLogger class which automatically logs the currently executed test names and the reasons for a failure. Makes test bases extend TestLogger to add automatic test name logging Frames test logging statements to make them more prominent This closes #1015. --- .../common/io/SequentialFormatTestBase.java | 3 +- .../common/typeutils/ComparatorTestBase.java | 3 +- .../common/typeutils/SerializerTestBase.java | 3 +- .../configuration/ConfigurationTest.java | 4 +- .../GlobalConfigurationTest.java | 3 +- .../UnmodifiableConfigurationTest.java | 3 +- .../flink/types/parser/ParserTestBase.java | 3 +- .../org/apache/flink/util/AbstractIDTest.java | 2 +- .../flink/util/InstantiationUtilTest.java | 2 +- .../util/NumberSequenceIteratorTest.java | 2 +- .../flink/util/SimpleStringUtilsTest.java | 2 +- .../apache/flink/util/StringUtilsTest.java | 2 +- .../org/apache/flink/util/TestLogger.java | 59 +++++++++++++++++++ .../base/TuplePairComparatorTestBase.java | 3 +- flink-optimizer/pom.xml | 8 +++ .../optimizer/util/CompilerTestBase.java | 3 +- flink-runtime/pom.xml | 8 +++ .../partition/SubpartitionTestBase.java | 3 +- .../operators/testutils/DriverTestBase.java | 3 +- .../operators/testutils/TaskTestBase.java | 3 +- .../testutils/UnaryOperatorTestBase.java | 3 +- flink-test-utils/pom.xml | 6 ++ .../apache/flink/test/util/TestBaseUtils.java | 3 +- .../test/cancelling/CancellingTestBase.java | 3 +- .../StreamFaultToleranceTestBase.java | 3 +- .../ScalaAPICompletenessTestBase.scala | 3 +- .../tuple/base/PairComparatorTestBase.scala | 3 +- .../org/apache/flink/yarn/YarnTestBase.java | 3 +- 28 files changed, 124 insertions(+), 25 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/util/TestLogger.java diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java index 3e4697ab87419..91f80e8b6188b 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java @@ -33,6 +33,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper; +import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -42,7 +43,7 @@ /** * Test base for {@link org.apache.flink.api.common.io.BinaryInputFormat} and {@link org.apache.flink.api.common.io.BinaryOutputFormat}. */ -public abstract class SequentialFormatTestBase { +public abstract class SequentialFormatTestBase extends TestLogger { public class InputSplitSorter implements Comparator { @Override diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java index bc5c6b6dd31a5..782c402f1addf 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java @@ -29,6 +29,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.junit.Test; @@ -37,7 +38,7 @@ * * @param */ -public abstract class ComparatorTestBase { +public abstract class ComparatorTestBase extends TestLogger { // Same as in the NormalizedKeySorter private static final int DEFAULT_MAX_NORMALIZED_KEY_LEN = 8; diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java index 998ae1218d789..ea91e56fa7c91 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.apache.commons.lang3.SerializationException; @@ -47,7 +48,7 @@ * (JodaTime DataTime) with the default KryoSerializer used to pass this test but the * internal state would be corrupt, which becomes evident when toString is called. */ -public abstract class SerializerTestBase { +public abstract class SerializerTestBase extends TestLogger { protected abstract TypeSerializer createSerializer(); diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java index 33dde3d72c4c1..e3b75b67e8e59 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java @@ -25,18 +25,18 @@ import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.util.TestLogger; import org.junit.Test; /** * This class contains test for the configuration package. In particular, the serialization of {@link Configuration} * objects is tested. */ -public class ConfigurationTest { +public class ConfigurationTest extends TestLogger { private static final byte[] EMPTY_BYTES = new byte[0]; private static final long TOO_LONG = Integer.MAX_VALUE + 10L; private static final double TOO_LONG_DOUBLE = Double.MAX_VALUE; - /** * This test checks the serialization/deserialization of configuration objects. diff --git a/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java index f207f783d67e2..41e97b206cb36 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java @@ -27,13 +27,14 @@ import java.lang.reflect.Field; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.util.TestLogger; import org.junit.Before; import org.junit.Test; /** * This class contains tests for the global configuration (parsing configuration directory information). */ -public class GlobalConfigurationTest { +public class GlobalConfigurationTest extends TestLogger { @Before public void resetSingleton() throws SecurityException, NoSuchFieldException, IllegalArgumentException, diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java index 3ea52b89ec4ea..386d03bc4b8ac 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.flink.util.TestLogger; import org.junit.Test; import java.lang.reflect.InvocationTargetException; @@ -34,7 +35,7 @@ * This class verifies that the Unmodifiable Configuration class overrides all setter methods in * Configuration. */ -public class UnmodifiableConfigurationTest { +public class UnmodifiableConfigurationTest extends TestLogger { @Test public void testOverrideAddMethods() { diff --git a/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java b/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java index 94fe327d5d1c1..b979a38503e1a 100644 --- a/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java @@ -26,13 +26,14 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import org.apache.flink.util.TestLogger; import org.junit.Test; /** * */ -public abstract class ParserTestBase { +public abstract class ParserTestBase extends TestLogger { public abstract String[] getValidTestValues(); diff --git a/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java b/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java index 8c7e8b54ce8cd..8de3b94680891 100644 --- a/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java @@ -30,7 +30,7 @@ /** * This class contains tests for the {@link org.apache.flink.util.AbstractID} class. */ -public class AbstractIDTest { +public class AbstractIDTest extends TestLogger { /** * Tests the serialization/deserialization of an abstract ID. */ diff --git a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java index aded919360c72..f9500b00683da 100644 --- a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java @@ -35,7 +35,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class InstantiationUtilTest { +public class InstantiationUtilTest extends TestLogger { @Test public void testInstantiationOfStringValue() { diff --git a/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java b/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java index 6698167bf1fe4..edaeb7a6658a4 100644 --- a/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java @@ -24,7 +24,7 @@ import org.junit.Test; -public class NumberSequenceIteratorTest { +public class NumberSequenceIteratorTest extends TestLogger { @Test public void testSplitRegular() { diff --git a/flink-core/src/test/java/org/apache/flink/util/SimpleStringUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/SimpleStringUtilsTest.java index 9794fa371609c..98f8574ec70af 100644 --- a/flink-core/src/test/java/org/apache/flink/util/SimpleStringUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/SimpleStringUtilsTest.java @@ -28,7 +28,7 @@ import org.junit.Test; -public class SimpleStringUtilsTest { +public class SimpleStringUtilsTest extends TestLogger { @Test public void testToLowerCaseConverting() { diff --git a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java index b580fc6c8234e..3f6b16d03e3ea 100644 --- a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java @@ -24,7 +24,7 @@ import org.apache.flink.util.StringUtils; import org.junit.Test; -public class StringUtilsTest { +public class StringUtilsTest extends TestLogger { @Test public void testControlCharacters() { diff --git a/flink-core/src/test/java/org/apache/flink/util/TestLogger.java b/flink-core/src/test/java/org/apache/flink/util/TestLogger.java new file mode 100644 index 0000000000000..49d0f0fc64281 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/util/TestLogger.java @@ -0,0 +1,59 @@ +/* + * 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.util; + +import org.junit.Rule; +import org.junit.rules.TestRule; +import org.junit.rules.TestWatcher; +import org.junit.runner.Description; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Adds automatic test name logging. Every test which wants to log which test is currently + * executed and why it failed, simply has to extend this class. + */ +public class TestLogger { + protected final Logger log = LoggerFactory.getLogger(getClass()); + + @Rule + public TestRule watchman = new TestWatcher() { + + @Override + public void starting(Description description) { + log.info("================================================================================"); + log.info("Test {} is running.", description); + log.info("--------------------------------------------------------------------------------"); + } + + @Override + public void succeeded(Description description) { + log.info("--------------------------------------------------------------------------------"); + log.info("Test {} successfully run.", description); + log.info("================================================================================"); + } + + @Override + public void failed(Throwable e, Description description) { + log.error("--------------------------------------------------------------------------------"); + log.error("Test {} failed with:\n{}", description, StringUtils.stringifyException(e)); + log.error("================================================================================"); + } + }; +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java index 17f85c9b44910..1d414d8e57e2e 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.TestLogger; import org.junit.Test; /** @@ -32,7 +33,7 @@ * @param * @param */ -public abstract class TuplePairComparatorTestBase { +public abstract class TuplePairComparatorTestBase extends TestLogger { protected abstract TypePairComparator createComparator(boolean ascending); diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml index 868ca3a703270..512c415ecd991 100644 --- a/flink-optimizer/pom.xml +++ b/flink-optimizer/pom.xml @@ -41,6 +41,14 @@ under the License. ${project.version} + + org.apache.flink + flink-core + ${project.version} + test-jar + test + + org.apache.flink flink-runtime diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java index 35c50d3434d55..b4c39a5ed49c7 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java @@ -38,6 +38,7 @@ import org.apache.flink.optimizer.plan.PlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.util.OperatingSystem; +import org.apache.flink.util.TestLogger; import org.apache.flink.util.Visitor; import org.junit.Before; @@ -46,7 +47,7 @@ * of a program and to fetch the nodes in an optimizer plan that correspond * the the node in the program plan. */ -public abstract class CompilerTestBase implements java.io.Serializable { +public abstract class CompilerTestBase extends TestLogger implements java.io.Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 065467fce6118..6dc8d42b68960 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -45,6 +45,14 @@ under the License. ${project.version} + + org.apache.flink + flink-core + ${project.version} + test-jar + test + + org.apache.flink flink-java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index a41a01354cee6..cb0069b10e517 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.util.TestLogger; import org.junit.Test; import static org.junit.Assert.assertFalse; @@ -27,7 +28,7 @@ /** * Basic subpartition behaviour tests. */ -public abstract class SubpartitionTestBase { +public abstract class SubpartitionTestBase extends TestLogger { /** * Return the subpartition to be tested. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 642ac7dbc3b63..e4aad981a4ac2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -53,7 +54,7 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) -public class DriverTestBase implements PactTaskContext { +public class DriverTestBase extends TestLogger implements PactTaskContext { protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java index c7be7a5b894ad..c04d051fb387a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java @@ -36,12 +36,13 @@ import org.apache.flink.types.Record; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Assert; import java.util.List; -public abstract class TaskTestBase { +public abstract class TaskTestBase extends TestLogger { protected long memorySize = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 20edc20a2065b..9600e5a7bce88 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -40,6 +40,7 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Assert; import org.junit.runner.RunWith; @@ -51,7 +52,7 @@ import java.util.List; @RunWith(Parameterized.class) -public class UnaryOperatorTestBase implements PactTaskContext { +public class UnaryOperatorTestBase extends TestLogger implements PactTaskContext { protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024; diff --git a/flink-test-utils/pom.xml b/flink-test-utils/pom.xml index 4bb51ef5d0b02..1eb5ff2597265 100644 --- a/flink-test-utils/pom.xml +++ b/flink-test-utils/pom.xml @@ -50,6 +50,12 @@ under the License. flink-runtime ${project.version} + + org.apache.flink + flink-core + ${project.version} + test-jar + org.apache.flink flink-runtime diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java index cd6d3b21b1ef3..9068fcc213d6f 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.StreamingMode; import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.TestLogger; import org.apache.hadoop.fs.FileSystem; import org.junit.Assert; import org.slf4j.Logger; @@ -66,7 +67,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -public class TestBaseUtils { +public class TestBaseUtils extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(TestBaseUtils.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java index 1502110ece220..088ea4d6ed8bc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.client.JobCancellationException; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.slf4j.Logger; @@ -45,7 +46,7 @@ /** * */ -public abstract class CancellingTestBase { +public abstract class CancellingTestBase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(CancellingTestBase.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index 8920cf2fa2077..59f24b054d7dd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -34,7 +35,7 @@ /** * Test base for fault tolerant streaming programs */ -public abstract class StreamFaultToleranceTestBase { +public abstract class StreamFaultToleranceTestBase extends TestLogger { protected static final int NUM_TASK_MANAGERS = 2; protected static final int NUM_TASK_SLOTS = 3; diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala index 0df4589deba24..d1055d0286589 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala @@ -19,6 +19,7 @@ package org.apache.flink.api.scala.completeness import java.lang.reflect.Method +import org.apache.flink.util.TestLogger import org.junit.Assert._ import org.junit.Test @@ -32,7 +33,7 @@ import scala.language.existentials * * Note: This is inspired by the JavaAPICompletenessChecker from Spark. */ -abstract class ScalaAPICompletenessTestBase { +abstract class ScalaAPICompletenessTestBase extends TestLogger { /** * Determines whether a method is excluded by name. diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala index d3d52b0dac9f5..5d154f5ae337c 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime.tuple.base +import org.apache.flink.util.TestLogger import org.junit.Assert.assertTrue import org.junit.Assert.fail import org.apache.flink.api.common.typeutils.TypePairComparator @@ -25,7 +26,7 @@ import org.junit.Test /** * Abstract test base for PairComparators. */ -abstract class PairComparatorTestBase[T, R] { +abstract class PairComparatorTestBase[T, R] extends TestLogger { protected def createComparator(ascending: Boolean): TypePairComparator[T, R] protected def getSortedTestData: (Array[T], Array[R]) diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java index 2d227000f03e4..4399f191e96f8 100644 --- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.client.CliFrontend; import org.apache.flink.client.FlinkYarnSessionCli; import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.TestLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.Service; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -69,7 +70,7 @@ * * The test is not thread-safe. Parallel execution of tests is not possible! */ -public abstract class YarnTestBase { +public abstract class YarnTestBase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(YarnTestBase.class); protected final static PrintStream originalStdout = System.out; From 143ec4f3132974a2ed3ea8683964a65bfe288a59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Thu, 6 Aug 2015 15:42:54 +0200 Subject: [PATCH 142/175] [FLINK-2286] [streaming] Wrapped ParallelMerge into stream operator Closes #994 --- .../api/datastream/DiscretizedStream.java | 8 ++-- .../api/operators/co/CoStreamFlatMap.java | 4 ++ .../operators/windowing/ParallelMerge.java | 3 ++ .../windowing/ParallelMergeOperator.java | 43 +++++++++++++++++++ 4 files changed, 53 insertions(+), 5 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java index ba28fa4f1946f..e35592e60f32d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java @@ -30,11 +30,9 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.functions.WindowMapFunction; -import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.streaming.api.operators.StreamFlatMap; -import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap; import org.apache.flink.streaming.api.operators.windowing.EmptyWindowFilter; import org.apache.flink.streaming.api.operators.windowing.ParallelGroupedMerge; import org.apache.flink.streaming.api.operators.windowing.ParallelMerge; @@ -45,6 +43,7 @@ import org.apache.flink.streaming.api.operators.windowing.WindowPartExtractor; import org.apache.flink.streaming.api.operators.windowing.WindowPartitioner; import org.apache.flink.streaming.api.operators.windowing.WindowReducer; +import org.apache.flink.streaming.api.operators.windowing.ParallelMergeOperator; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.StreamWindowTypeInfo; import org.apache.flink.streaming.api.windowing.WindowUtils.WindowKey; @@ -147,7 +146,7 @@ protected DiscretizedStream timeReduce(ReduceFunction reduceFunction) DataStream> numOfParts, DiscretizedStream reduced, ReduceFunction reduceFunction) { - CoFlatMapFunction, Tuple2, StreamWindow> parallelMerger = isGrouped() ? new ParallelGroupedMerge() + ParallelMerge parallelMerger = isGrouped() ? new ParallelGroupedMerge() : new ParallelMerge(reduceFunction); return reduced.discretizedStream @@ -156,8 +155,7 @@ protected DiscretizedStream timeReduce(ReduceFunction reduceFunction) .transform( "CoFlatMap", reduced.discretizedStream.getType(), - new CoStreamFlatMap, Tuple2, StreamWindow>( - parallelMerger)); + new ParallelMergeOperator(parallelMerger)); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java index d2bd107d55cfd..1448ab8ebed3f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java @@ -81,4 +81,8 @@ public void processWatermark2(Watermark mark) throws Exception { output.emitWatermark(new Watermark(combinedWatermark)); } } + + protected TimestampedCollector getCollector() { + return collector; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java index cd239fc66c106..ce7d887530d5e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java @@ -139,4 +139,7 @@ public void open(Configuration conf) { this.numberOfDiscretizers = getRuntimeContext().getNumberOfParallelSubtasks(); } + Map, Integer>> getReceivedWindows() { + return receivedWindows; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java new file mode 100644 index 0000000000000..74df3ad21b10b --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java @@ -0,0 +1,43 @@ +/* + * 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.streaming.api.operators.windowing; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap; +import org.apache.flink.streaming.api.windowing.StreamWindow; + +public class ParallelMergeOperator extends CoStreamFlatMap, Tuple2, StreamWindow> { + + private ParallelMerge parallelMerge; + + public ParallelMergeOperator(ParallelMerge parallelMerge) { + super(parallelMerge); + this.parallelMerge = parallelMerge; + } + + @Override + public void close() throws Exception { + // emit remaining (partial) windows + + for (Tuple2, Integer> receivedWindow : parallelMerge.getReceivedWindows().values()) { + getCollector().collect(receivedWindow.f0); + } + + super.close(); + } +} From 67087dea698881b365debe17f847af0f192ee5cb Mon Sep 17 00:00:00 2001 From: mbalassi Date: Tue, 11 Aug 2015 15:51:30 +0200 Subject: [PATCH 143/175] [FLINK-2286] [streaming] ITCase for ParallelMerge behavior Closes #1014 --- .../graph/test/example/PageRankITCase.java | 2 +- .../windowing/ParallelMergeITCase.java | 101 ++++++++++++++++++ .../scala/table/test/PageRankTableITCase.java | 2 +- .../tez/test/PageRankBasicStepITCase.java | 2 +- .../apache/flink/test/util/TestBaseUtils.java | 11 +- .../exampleJavaPrograms/PageRankITCase.java | 2 +- .../exampleScalaPrograms/PageRankITCase.java | 2 +- 7 files changed, 111 insertions(+), 11 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java index 544cc66f01068..cde959f7f694d 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java @@ -61,7 +61,7 @@ public void before() throws Exception{ @After public void after() throws Exception{ - compareKeyValueParisWithDelta(expected, resultPath, "\t", 0.01); + compareKeyValuePairsWithDelta(expected, resultPath, "\t", 0.01); } @Test diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java new file mode 100644 index 0000000000000..b762d65a0f8c4 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java @@ -0,0 +1,101 @@ +/* + * 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.streaming.api.operators.windowing; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.helper.Time; +import org.apache.flink.streaming.util.StreamingProgramTestBase; +import org.apache.flink.util.Collector; +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * Tests that {@link ParallelMerge} does not swallow records of the + * last window. + */ +public class ParallelMergeITCase extends StreamingProgramTestBase { + + protected String textPath; + protected String resultPath; + protected final String input = "To be, or not to be,--that is the question:--" + + "Whether 'tis nobler in the mind to suffer"; + + @Override + protected void preSubmit() throws Exception { + textPath = createTempFile("text.txt", input); + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + List resultLines = new ArrayList<>(); + readAllResultLines(resultLines, resultPath); + + // check that result lines are not swallowed, as every element is expected to be in the + // last time window we either get the right output or no output at all + if (resultLines.isEmpty()){ + Assert.fail(); + } + } + + @Override + protected void testProgram() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream text = env.fromElements(input); + + DataStream> counts = + text.flatMap(new Tokenizer()) + .window(Time.of(1000, TimeUnit.MILLISECONDS)) + .groupBy(0) + .sum(1) + .flatten(); + + counts.writeAsText(resultPath); + + try { + env.execute(); + } catch (RuntimeException e){ + // might happen at closing the active window + // do nothing + } + } + + public static final class Tokenizer implements FlatMapFunction> { + private static final long serialVersionUID = 1L; + + @Override + public void flatMap(String value, Collector> out) + throws Exception { + String[] tokens = value.toLowerCase().split("\\W+"); + + for (String token : tokens) { + if (token.length() > 0) { + out.collect(Tuple2.of(token, 1)); + } + } + } + } +} diff --git a/flink-staging/flink-table/src/test/java/org/apache/flink/api/scala/table/test/PageRankTableITCase.java b/flink-staging/flink-table/src/test/java/org/apache/flink/api/scala/table/test/PageRankTableITCase.java index 5353b732103f9..18166146c6132 100644 --- a/flink-staging/flink-table/src/test/java/org/apache/flink/api/scala/table/test/PageRankTableITCase.java +++ b/flink-staging/flink-table/src/test/java/org/apache/flink/api/scala/table/test/PageRankTableITCase.java @@ -61,7 +61,7 @@ protected void testProgram() throws Exception { @Override protected void postSubmit() throws Exception { - compareKeyValueParisWithDelta(expectedResult, resultPath, " ", 0.01); + compareKeyValuePairsWithDelta(expectedResult, resultPath, " ", 0.01); } @Parameters diff --git a/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/PageRankBasicStepITCase.java b/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/PageRankBasicStepITCase.java index 511c2cb2adc8d..9a203feec438a 100644 --- a/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/PageRankBasicStepITCase.java +++ b/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/PageRankBasicStepITCase.java @@ -49,6 +49,6 @@ protected void testProgram() throws Exception { @Override protected void postSubmit() throws Exception { - compareKeyValueParisWithDelta(expectedResult, resultPath, " ", 0.001); + compareKeyValuePairsWithDelta(expectedResult, resultPath, " ", 0.001); } } diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java index 9068fcc213d6f..c28347c62f61f 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java @@ -321,16 +321,15 @@ public static void checkLinesAgainstRegexp(String resultPath, String regexp){ Assert.fail(msg); } } - } - public static void compareKeyValueParisWithDelta(String expectedLines, String resultPath, - String delimiter, double maxDelta) throws Exception { - compareKeyValueParisWithDelta(expectedLines, resultPath, new String[]{}, delimiter, maxDelta); + public static void compareKeyValuePairsWithDelta(String expectedLines, String resultPath, + String delimiter, double maxDelta) throws Exception { + compareKeyValuePairsWithDelta(expectedLines, resultPath, new String[]{}, delimiter, maxDelta); } - public static void compareKeyValueParisWithDelta(String expectedLines, String resultPath, - String[] excludePrefixes, String delimiter, double maxDelta) throws Exception { + public static void compareKeyValuePairsWithDelta(String expectedLines, String resultPath, + String[] excludePrefixes, String delimiter, double maxDelta) throws Exception { ArrayList list = new ArrayList(); readAllResultLines(list, resultPath, excludePrefixes, false); diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java index 1c66c3e781018..2d1519d598f36 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java @@ -63,7 +63,7 @@ public void before() throws Exception{ @After public void after() throws Exception{ - compareKeyValueParisWithDelta(expected, resultPath, " ", 0.01); + compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01); } @Test diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java index f9c2566f9ce7e..6b9e550012e35 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java @@ -67,7 +67,7 @@ public void before() throws Exception{ @After public void after() throws Exception{ - compareKeyValueParisWithDelta(expected, resultPath, " ", 0.01); + compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01); } @Test From a9d55d3ea7a876510cbb643ff62d777d438291b9 Mon Sep 17 00:00:00 2001 From: Rucongzhang Date: Sun, 16 Aug 2015 16:44:00 +0800 Subject: [PATCH 144/175] [flink-2532] [streaming] fix variable name in StreamWindow Closes #1025 --- .../flink/streaming/api/windowing/StreamWindow.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java index ee2ea06f2d437..5a63940896e90 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java @@ -153,24 +153,24 @@ public static List> split(StreamWindow window, int n) { if (n > numElements) { return split(window, numElements); } else { - List> split = new ArrayList>(); + List> splitsList = new ArrayList>(); int splitSize = numElements / n; int index = -1; StreamWindow currentSubWindow = new StreamWindow(window.windowID, n); - split.add(currentSubWindow); + splitsList.add(currentSubWindow); for (X element : window) { index++; - if (index == splitSize && split.size() < n) { + if (index == splitSize && splitsList.size() < n) { currentSubWindow = new StreamWindow(window.windowID, n); - split.add(currentSubWindow); + splitsList.add(currentSubWindow); index = 0; } currentSubWindow.add(element); } - return split; + return splitsList; } } From 6eae11f34943ea835eaae38651c64c98666e01d7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 18 Aug 2015 15:00:42 +0200 Subject: [PATCH 145/175] [tests] Hardens TaskManagerRegistrationTest.testTaskManagerResumesConnectAfterJobManagerFailure test case by checking testing actor's complete mailbox for a Terminated message --- .../TaskManagerRegistrationTest.java | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java index 6d8e6569f5ec3..df650199ae162 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java @@ -23,6 +23,7 @@ import akka.actor.InvalidActorNameException; import akka.actor.Kill; import akka.actor.Props; +import akka.actor.Terminated; import akka.actor.UntypedActor; import akka.pattern.Patterns; import akka.testkit.JavaTestKit; @@ -38,6 +39,7 @@ import org.apache.flink.runtime.messages.RegistrationMessages.RefuseRegistration; import org.apache.flink.runtime.messages.TaskManagerMessages; import org.apache.flink.runtime.testingUtils.TestingTaskManager; +import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -60,7 +62,7 @@ * when connecting to the JobManager, and when the JobManager * is unreachable. */ -public class TaskManagerRegistrationTest { +public class TaskManagerRegistrationTest extends TestLogger { private static final Option NONE_STRING = Option.empty(); @@ -310,11 +312,22 @@ protected void run() { stopActor(fakeJobManager1); // wait for the killing to be completed - new Within(new FiniteDuration(2, TimeUnit.SECONDS)) { + final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.SECONDS); + + new Within(timeout) { @Override protected void run() { - expectTerminated(fakeJobManager1); + Object message = null; + + // we might also receive RegisterTaskManager and Heartbeat messages which + // are queued up in the testing actor's mailbox + while(message == null || !(message instanceof Terminated)) { + message = receiveOne(timeout); + } + + Terminated terminatedMessage = (Terminated) message; + assertEquals(fakeJobManager1, terminatedMessage.actor()); } }; From 09fdfda7f25cf95426bc43ca33ed7bb39c7d353a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 18 Aug 2015 11:11:13 +0200 Subject: [PATCH 146/175] [tests] Rename 'compiler' tests to 'optimizer' tests for consistent naming --- .../flink/test/iterative/MultipleSolutionSetJoinsITCase.java | 2 +- .../{compiler => optimizer}/examples/KMeansSingleStepTest.java | 2 +- .../examples/RelationalQueryCompilerTest.java | 2 +- .../{compiler => optimizer}/examples/WordCountCompilerTest.java | 2 +- .../iterations/ConnectedComponentsCoGroupTest.java | 2 +- .../{compiler => optimizer}/iterations/IterativeKMeansTest.java | 2 +- .../iterations/MultipleJoinsWithSolutionSetCompilerTest.java | 2 +- .../iterations/PageRankCompilerTest.java | 2 +- .../plandump => optimizer/jsonplan}/DumpCompiledPlanTest.java | 2 +- .../plandump => optimizer/jsonplan}/PreviewPlanDumpTest.java | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/examples/KMeansSingleStepTest.java (99%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/examples/RelationalQueryCompilerTest.java (99%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/examples/WordCountCompilerTest.java (99%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/iterations/ConnectedComponentsCoGroupTest.java (99%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/iterations/IterativeKMeansTest.java (99%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/iterations/MultipleJoinsWithSolutionSetCompilerTest.java (99%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler => optimizer}/iterations/PageRankCompilerTest.java (98%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler/plandump => optimizer/jsonplan}/DumpCompiledPlanTest.java (98%) rename flink-tests/src/test/java/org/apache/flink/test/{compiler/plandump => optimizer/jsonplan}/PreviewPlanDumpTest.java (98%) diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java index 8eea9f37ae780..e6e91f6d51618 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java @@ -26,7 +26,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.test.compiler.iterations.MultipleJoinsWithSolutionSetCompilerTest; +import org.apache.flink.test.optimizer.iterations.MultipleJoinsWithSolutionSetCompilerTest; import org.apache.flink.test.util.JavaProgramTestBase; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java index ec532bee1aa53..ab8ff45f8c403 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.examples; +package org.apache.flink.test.optimizer.examples; import static org.junit.Assert.*; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java index bc53810bebc4b..f58486bb48e28 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.examples; +package org.apache.flink.test.optimizer.examples; import java.util.Arrays; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java index 6cfef9c073409..ce713832f615f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.examples; +package org.apache.flink.test.optimizer.examples; import java.util.Arrays; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java index 10f2b5c638766..de5fde0bbc667 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.iterations; +package org.apache.flink.test.optimizer.iterations; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/IterativeKMeansTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/IterativeKMeansTest.java index bd4b6be0d132a..378527073b603 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/IterativeKMeansTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.iterations; +package org.apache.flink.test.optimizer.iterations; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java index aea448fd614e0..f17b28a21b0ca 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.iterations; +package org.apache.flink.test.optimizer.iterations; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java index a3b757251e1fa..9b172706568e1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java @@ -17,7 +17,7 @@ */ -package org.apache.flink.test.compiler.iterations; +package org.apache.flink.test.optimizer.iterations; import static org.apache.flink.api.java.aggregation.Aggregations.SUM; import static org.junit.Assert.fail; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java index a9811243e7663..082532e4b4479 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.plandump; +package org.apache.flink.test.optimizer.jsonplan; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/PreviewPlanDumpTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/PreviewPlanDumpTest.java rename to flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java index b3483339e1343..49fe6d8cd5b45 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/PreviewPlanDumpTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.compiler.plandump; +package org.apache.flink.test.optimizer.jsonplan; import java.util.List; From d2e88ffdabb9d817f3897acd3ca3e566cf86450a Mon Sep 17 00:00:00 2001 From: HuangWHWHW <404823056@qq.com> Date: Wed, 5 Aug 2015 10:50:59 +0800 Subject: [PATCH 147/175] [FLINK-2477] [streaming] Add tests for SocketClientSink This closes #977 --- .../api/functions/SocketClientSinkTest.java | 128 ++++++++++++++++++ 1 file changed, 128 insertions(+) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/SocketClientSinkTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/SocketClientSinkTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/SocketClientSinkTest.java new file mode 100644 index 0000000000000..b6b6181082474 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/SocketClientSinkTest.java @@ -0,0 +1,128 @@ +/* + * 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.streaming.api.functions; + +import java.io.IOException; +import java.net.Socket; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.sink.SocketClientSink; +import org.apache.flink.streaming.util.serialization.SerializationSchema; + +import org.junit.Assert; +import org.junit.Test; + +import static org.junit.Assert.*; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.ServerSocket; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Tests for the {@link org.apache.flink.streaming.api.functions.sink.SocketClientSink}. + */ +public class SocketClientSinkTest{ + + final AtomicReference error = new AtomicReference(); + private final String host = "127.0.0.1"; + private int port; + private String access; + private String value; + public SocketServer.ServerThread th; + + public SocketClientSinkTest() { + } + + class SocketServer extends Thread { + + private ServerSocket server; + private Socket sk; + private BufferedReader rdr; + + private SocketServer() { + try { + this.server = new ServerSocket(0); + port = server.getLocalPort(); + } catch (Exception e) { + error.set(e); + } + } + + public void run() { + try { + sk = server.accept(); + access = "Connected"; + th = new ServerThread(sk); + th.start(); + } catch (Exception e) { + error.set(e); + } + } + + class ServerThread extends Thread { + Socket sk; + + public ServerThread(Socket sk) { + this.sk = sk; + } + + public void run() { + try { + rdr = new BufferedReader(new InputStreamReader(sk + .getInputStream())); + value = rdr.readLine(); + } catch (IOException e) { + error.set(e); + } + } + } + } + + @Test + public void testSocketSink() throws Exception{ + + SocketServer server = new SocketServer(); + server.start(); + + SerializationSchema simpleSchema = new SerializationSchema() { + @Override + public byte[] serialize(String element) { + return element.getBytes(); + } + }; + + SocketClientSink simpleSink = new SocketClientSink(host, port, simpleSchema); + simpleSink.open(new Configuration()); + simpleSink.invoke("testSocketSinkInvoke"); + simpleSink.close(); + + server.join(); + th.join(); + + if (error.get() != null) { + Throwable t = error.get(); + t.printStackTrace(); + fail("Error in spawned thread: " + t.getMessage()); + } + + assertEquals("Connected", this.access); + assertEquals("testSocketSinkInvoke", value); + } +} \ No newline at end of file From 240e8895c6e1d5ea6f67370d276cb58fd0ecddb8 Mon Sep 17 00:00:00 2001 From: Pieter-Jan Van Aeken Date: Mon, 10 Aug 2015 14:06:52 +0200 Subject: [PATCH 148/175] [FLINK-1962] Add Gelly Scala API This closes #1004 --- .../api/scala/ExecutionEnvironment.scala | 4 + flink-staging/flink-gelly-scala/pom.xml | 198 +++++ .../flink/graph/scala/EdgesFunction.scala | 35 + .../scala/EdgesFunctionWithVertexValue.scala | 33 + .../org/apache/flink/graph/scala/Graph.scala | 734 ++++++++++++++++++ .../flink/graph/scala/NeighborsFunction.scala | 37 + .../NeighborsFunctionWithVertexValue.scala | 40 + .../apache/flink/graph/scala/package.scala | 30 + .../graph/scala/utils/EdgeToTuple3Map.scala | 31 + .../graph/scala/utils/VertexToTuple2Map.scala | 31 + .../graph/scala/test/TestGraphUtils.scala | 55 ++ .../scala/test/operations/DegreesITCase.scala | 88 +++ .../operations/GraphMutationsITCase.scala | 171 ++++ .../operations/GraphOperationsITCase.scala | 238 ++++++ .../test/operations/JoinWithEdgesITCase.scala | 170 ++++ .../operations/JoinWithVerticesITCase.scala | 93 +++ .../test/operations/MapEdgesITCase.scala | 102 +++ .../test/operations/MapVerticesITCase.scala | 99 +++ .../ReduceOnEdgesMethodsITCase.scala | 173 +++++ .../ReduceOnNeighborMethodsITCase.scala | 144 ++++ .../java/org/apache/flink/graph/Graph.java | 179 +++++ flink-staging/pom.xml | 1 + 22 files changed, 2686 insertions(+) create mode 100644 flink-staging/flink-gelly-scala/pom.xml create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala create mode 100644 flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala create mode 100644 flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index d53c54cdaae6c..17311e9f9253a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -64,6 +64,10 @@ import scala.reflect.ClassTag */ class ExecutionEnvironment(javaEnv: JavaEnv) { + /** + * @return the Java Execution environment. + */ + def getJavaEnv: JavaEnv = javaEnv /** * Gets the config object. */ diff --git a/flink-staging/flink-gelly-scala/pom.xml b/flink-staging/flink-gelly-scala/pom.xml new file mode 100644 index 0000000000000..390dbb84a9bbd --- /dev/null +++ b/flink-staging/flink-gelly-scala/pom.xml @@ -0,0 +1,198 @@ + + + + + flink-staging + org.apache.flink + 0.10-SNAPSHOT + .. + + 4.0.0 + + flink-gelly-scala + + jar + + + + org.apache.flink + flink-scala + ${project.version} + + + org.apache.flink + flink-clients + ${project.version} + + + org.apache.flink + flink-gelly + ${project.version} + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + com.google.guava + guava + ${guava.version} + + + + + + + + net.alchim31.maven + scala-maven-plugin + 3.1.4 + + + + scala-compile-first + process-resources + + compile + + + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + -Xms128m + -Xmx512m + + + + org.scalamacros + paradise_${scala.version} + ${scala.macros.version} + + + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + true + + org.scala-ide.sdt.core.scalanature + org.eclipse.jdt.core.javanature + + + org.scala-ide.sdt.core.scalabuilder + + + org.scala-ide.sdt.launching.SCALA_CONTAINER + org.eclipse.jdt.launching.JRE_CONTAINER + + + org.scala-lang:scala-library + org.scala-lang:scala-compiler + + + **/*.scala + **/*.java + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.7 + + + + add-source + generate-sources + + add-source + + + + src/main/scala + + + + + + add-test-source + generate-test-sources + + add-test-source + + + + src/test/scala + + + + + + + + org.scalastyle + scalastyle-maven-plugin + 0.5.0 + + + + check + + + + + false + true + true + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + ${project.basedir}/../../tools/maven/scalastyle-config.xml + ${project.basedir}/scalastyle-output.xml + UTF-8 + + + + + + \ No newline at end of file diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala new file mode 100644 index 0000000000000..70a5fdf181ad3 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala @@ -0,0 +1,35 @@ +/* + * 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.graph.scala + +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.graph.Edge +import org.apache.flink.util.Collector + +abstract class EdgesFunction[K, EV, T] extends org.apache.flink.graph.EdgesFunction[K, EV, T] { + + def iterateEdges(edges: Iterable[(K, Edge[K, EV])], out: Collector[T]) + + override def iterateEdges(edges: java.lang.Iterable[Tuple2[K, Edge[K, EV]]], out: + Collector[T]): Unit = { + val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(edges) + .map(jtuple => (jtuple.f0, jtuple.f1)) + iterateEdges(scalaIterable, out) + } +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala new file mode 100644 index 0000000000000..82589b6fbf95d --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala @@ -0,0 +1,33 @@ +/* + * 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.graph.scala + +import org.apache.flink.graph.{Edge, Vertex} +import org.apache.flink.util.Collector + +abstract class EdgesFunctionWithVertexValue[K, VV, EV, T] extends org.apache.flink.graph +.EdgesFunctionWithVertexValue[K, VV, EV, T] { + @throws(classOf[Exception]) + def iterateEdges(v: Vertex[K, VV], edges: Iterable[Edge[K, EV]], out: Collector[T]) + + override def iterateEdges(v: Vertex[K, VV], edges: java.lang.Iterable[Edge[K, EV]], out: + Collector[T]) = { + iterateEdges(v, scala.collection.JavaConversions.iterableAsScalaIterable(edges), out) + } +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala new file mode 100644 index 0000000000000..738fd903def97 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala @@ -0,0 +1,734 @@ +/* + * 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.graph.scala + +import org.apache.flink.api.common.functions.{FilterFunction, MapFunction} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.{tuple => jtuple} +import org.apache.flink.api.scala._ +import org.apache.flink.graph._ +import org.apache.flink.graph.gsa.{ApplyFunction, GSAConfiguration, GatherFunction, SumFunction} +import org.apache.flink.graph.spargel.{MessagingFunction, VertexCentricConfiguration, VertexUpdateFunction} +import org.apache.flink.{graph => jg} + +import _root_.scala.collection.JavaConverters._ +import _root_.scala.reflect.ClassTag + +object Graph { + def fromDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV: + TypeInformation : ClassTag](vertices: DataSet[Vertex[K, VV]], edges: DataSet[Edge[K, EV]], + env: ExecutionEnvironment): Graph[K, VV, EV] = { + wrapGraph(jg.Graph.fromDataSet[K, VV, EV](vertices.javaSet, edges.javaSet, env.getJavaEnv)) + } + + def fromCollection[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV: + TypeInformation : ClassTag](vertices: Seq[Vertex[K, VV]], edges: Seq[Edge[K, EV]], env: + ExecutionEnvironment): Graph[K, VV, EV] = { + wrapGraph(jg.Graph.fromCollection[K, VV, EV](vertices.asJavaCollection, edges + .asJavaCollection, env.getJavaEnv)) + } +} + +/** + * Represents a graph consisting of {@link Edge edges} and {@link Vertex vertices}. + * @param jgraph the underlying java api Graph. + * @tparam K the key type for vertex and edge identifiers + * @tparam VV the value type for vertices + * @tparam EV the value type for edges + * @see org.apache.flink.graph.Edge + * @see org.apache.flink.graph.Vertex + */ +final class Graph[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV: +TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { + + private[flink] def getWrappedGraph = jgraph + + + private[flink] def clean[F <: AnyRef](f: F, checkSerializable: Boolean = true): F = { + if (jgraph.getContext.getConfig.isClosureCleanerEnabled) { + ClosureCleaner.clean(f, checkSerializable) + } + ClosureCleaner.ensureSerializable(f) + f + } + + /** + * @return the vertex DataSet. + */ + def getVertices = wrap(jgraph.getVertices) + + /** + * @return the edge DataSet. + */ + def getEdges = wrap(jgraph.getEdges) + + /** + * @return the vertex DataSet as Tuple2. + */ + def getVerticesAsTuple2(): DataSet[(K, VV)] = { + wrap(jgraph.getVerticesAsTuple2).map(jtuple => (jtuple.f0, jtuple.f1)) + } + + /** + * @return the edge DataSet as Tuple3. + */ + def getEdgesAsTuple3(): DataSet[(K, K, EV)] = { + wrap(jgraph.getEdgesAsTuple3).map(jtuple => (jtuple.f0, jtuple.f1, jtuple.f2)) + } + + /** + * Apply a function to the attribute of each vertex in the graph. + * + * @param mapper the map function to apply. + * @return a new graph + */ + def mapVertices[NV: TypeInformation : ClassTag](mapper: MapFunction[Vertex[K, VV], NV]): + Graph[K, NV, EV] = { + new Graph[K, NV, EV](jgraph.mapVertices[NV]( + mapper, + createTypeInformation[Vertex[K, NV]] + )) + } + + /** + * Apply a function to the attribute of each vertex in the graph. + * + * @param fun the map function to apply. + * @return a new graph + */ + def mapVertices[NV: TypeInformation : ClassTag](fun: Vertex[K, VV] => NV): Graph[K, NV, EV] = { + val mapper: MapFunction[Vertex[K, VV], NV] = new MapFunction[Vertex[K, VV], NV] { + val cleanFun = clean(fun) + + def map(in: Vertex[K, VV]): NV = cleanFun(in) + } + new Graph[K, NV, EV](jgraph.mapVertices[NV](mapper, createTypeInformation[Vertex[K, NV]])) + } + + /** + * Apply a function to the attribute of each edge in the graph. + * + * @param mapper the map function to apply. + * @return a new graph + */ + def mapEdges[NV: TypeInformation : ClassTag](mapper: MapFunction[Edge[K, EV], NV]): Graph[K, + VV, NV] = { + new Graph[K, VV, NV](jgraph.mapEdges[NV]( + mapper, + createTypeInformation[Edge[K, NV]] + )) + } + + /** + * Apply a function to the attribute of each edge in the graph. + * + * @param fun the map function to apply. + * @return a new graph + */ + def mapEdges[NV: TypeInformation : ClassTag](fun: Edge[K, EV] => NV): Graph[K, VV, NV] = { + val mapper: MapFunction[Edge[K, EV], NV] = new MapFunction[Edge[K, EV], NV] { + val cleanFun = clean(fun) + + def map(in: Edge[K, EV]): NV = cleanFun(in) + } + new Graph[K, VV, NV](jgraph.mapEdges[NV](mapper, createTypeInformation[Edge[K, NV]])) + } + + /** + * Joins the vertex DataSet of this graph with an input DataSet and applies + * a UDF on the resulted values. + * + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. + * @return a new graph where the vertex values have been updated. + */ + def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper: MapFunction[ + (VV, T), VV]): Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() { + override def map(value: jtuple.Tuple2[VV, T]): VV = { + mapper.map((value.f0, value.f1)) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1, + scalatuple._2)).javaSet + wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper)) + } + + /** + * Joins the vertex DataSet of this graph with an input DataSet and applies + * a UDF on the resulted values. + * + * @param inputDataSet the DataSet to join with. + * @param fun the UDF map function to apply. + * @return a new graph where the vertex values have been updated. + */ + def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (VV, T) => VV): + Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() { + val cleanFun = clean(fun) + + override def map(value: jtuple.Tuple2[VV, T]): VV = { + cleanFun(value.f0, value.f1) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1, + scalatuple._2)).javaSet + wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper)) + } + + /** + * Joins the edge DataSet with an input DataSet on a composite key of both + * source and target and applies a UDF on the resulted values. + * + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. + * @tparam T the return type + * @return a new graph where the edge values have been updated. + */ + def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], mapper: MapFunction[ + (EV, T), EV]): Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() { + override def map(value: jtuple.Tuple2[EV, T]): EV = { + mapper.map((value.f0, value.f1)) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1, + scalatuple._2, scalatuple._3)).javaSet + wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper)) + } + + /** + * Joins the edge DataSet with an input DataSet on a composite key of both + * source and target and applies a UDF on the resulted values. + * + * @param inputDataSet the DataSet to join with. + * @param fun the UDF map function to apply. + * @tparam T the return type + * @return a new graph where the edge values have been updated. + */ + def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], fun: (EV, T) => EV): + Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() { + val cleanFun = clean(fun) + + override def map(value: jtuple.Tuple2[EV, T]): EV = { + cleanFun(value.f0, value.f1) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1, + scalatuple._2, scalatuple._3)).javaSet + wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper)) + } + + /** + * Joins the edge DataSet with an input DataSet on the source key of the + * edges and the first attribute of the input DataSet and applies a UDF on + * the resulted values. In case the inputDataSet contains the same key more + * than once, only the first value will be considered. + * + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. + * @tparam T the return type + * @return a new graph where the edge values have been updated. + */ + def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper: + MapFunction[(EV, T), EV]): Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() { + override def map(value: jtuple.Tuple2[EV, T]): EV = { + mapper.map((value.f0, value.f1)) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1, + scalatuple._2)).javaSet + wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper)) + } + + /** + * Joins the edge DataSet with an input DataSet on the source key of the + * edges and the first attribute of the input DataSet and applies a UDF on + * the resulted values. In case the inputDataSet contains the same key more + * than once, only the first value will be considered. + * + * @param inputDataSet the DataSet to join with. + * @param fun the UDF map function to apply. + * @tparam T the return type + * @return a new graph where the edge values have been updated. + */ + def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) => + EV): Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() { + val cleanFun = clean(fun) + + override def map(value: jtuple.Tuple2[EV, T]): EV = { + cleanFun(value.f0, value.f1) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1, + scalatuple._2)).javaSet + wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper)) + } + + /** + * Joins the edge DataSet with an input DataSet on the target key of the + * edges and the first attribute of the input DataSet and applies a UDF on + * the resulted values. Should the inputDataSet contain the same key more + * than once, only the first value will be considered. + * + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. + * @tparam T the return type + * @return a new graph where the edge values have been updated. + */ + def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper: + MapFunction[(EV, T), EV]): Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() { + override def map(value: jtuple.Tuple2[EV, T]): EV = { + mapper.map((value.f0, value.f1)) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1, + scalatuple._2)).javaSet + wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper)) + } + + /** + * Joins the edge DataSet with an input DataSet on the target key of the + * edges and the first attribute of the input DataSet and applies a UDF on + * the resulted values. Should the inputDataSet contain the same key more + * than once, only the first value will be considered. + * + * @param inputDataSet the DataSet to join with. + * @param fun the UDF map function to apply. + * @tparam T the return type + * @return a new graph where the edge values have been updated. + */ + def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) => + EV): Graph[K, VV, EV] = { + val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() { + val cleanFun = clean(fun) + + override def map(value: jtuple.Tuple2[EV, T]): EV = { + cleanFun(value.f0, value.f1) + } + } + val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1, + scalatuple._2)).javaSet + wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper)) + } + + /** + * Apply filtering functions to the graph and return a sub-graph that + * satisfies the predicates for both vertices and edges. + * + * @param vertexFilter the filter function for vertices. + * @param edgeFilter the filter function for edges. + * @return the resulting sub-graph. + */ + def subgraph(vertexFilter: FilterFunction[Vertex[K, VV]], edgeFilter: FilterFunction[Edge[K, + EV]]) = { + wrapGraph(jgraph.subgraph(vertexFilter, edgeFilter)) + } + + /** + * Apply filtering functions to the graph and return a sub-graph that + * satisfies the predicates for both vertices and edges. + * + * @param vertexFilterFun the filter function for vertices. + * @param edgeFilterFun the filter function for edges. + * @return the resulting sub-graph. + */ + def subgraph(vertexFilterFun: Vertex[K, VV] => Boolean, edgeFilterFun: Edge[K, EV] => + Boolean) = { + val vertexFilter = new FilterFunction[Vertex[K, VV]] { + val cleanVertexFun = clean(vertexFilterFun) + + override def filter(value: Vertex[K, VV]): Boolean = cleanVertexFun(value) + } + + val edgeFilter = new FilterFunction[Edge[K, EV]] { + val cleanEdgeFun = clean(edgeFilterFun) + + override def filter(value: Edge[K, EV]): Boolean = cleanEdgeFun(value) + } + + wrapGraph(jgraph.subgraph(vertexFilter, edgeFilter)) + } + + /** + * Apply a filtering function to the graph and return a sub-graph that + * satisfies the predicates only for the vertices. + * + * @param vertexFilter the filter function for vertices. + * @return the resulting sub-graph. + */ + def filterOnVertices(vertexFilter: FilterFunction[Vertex[K, VV]]) = { + wrapGraph(jgraph.filterOnVertices(vertexFilter)) + } + + /** + * Apply a filtering function to the graph and return a sub-graph that + * satisfies the predicates only for the vertices. + * + * @param vertexFilterFun the filter function for vertices. + * @return the resulting sub-graph. + */ + def filterOnVertices(vertexFilterFun: Vertex[K, VV] => Boolean) = { + val vertexFilter = new FilterFunction[Vertex[K, VV]] { + val cleanVertexFun = clean(vertexFilterFun) + + override def filter(value: Vertex[K, VV]): Boolean = cleanVertexFun(value) + } + + wrapGraph(jgraph.filterOnVertices(vertexFilter)) + } + + /** + * Apply a filtering function to the graph and return a sub-graph that + * satisfies the predicates only for the edges. + * + * @param edgeFilter the filter function for edges. + * @return the resulting sub-graph. + */ + def filterOnEdges(edgeFilter: FilterFunction[Edge[K, EV]]) = { + wrapGraph(jgraph.filterOnEdges(edgeFilter)) + } + + /** + * Apply a filtering function to the graph and return a sub-graph that + * satisfies the predicates only for the edges. + * + * @param edgeFilterFun the filter function for edges. + * @return the resulting sub-graph. + */ + def filterOnEdges(edgeFilterFun: Edge[K, EV] => Boolean) = { + val edgeFilter = new FilterFunction[Edge[K, EV]] { + val cleanEdgeFun = clean(edgeFilterFun) + + override def filter(value: Edge[K, EV]): Boolean = cleanEdgeFun(value) + } + + wrapGraph(jgraph.filterOnEdges(edgeFilter)) + } + + /** + * Return the in-degree of all vertices in the graph + * + * @return A DataSet of Tuple2 + */ + def inDegrees(): DataSet[(K, Long)] = { + wrap(jgraph.inDegrees).map(javatuple => (javatuple.f0, javatuple.f1)) + } + + /** + * Return the out-degree of all vertices in the graph + * + * @return A DataSet of Tuple2 + */ + def outDegrees(): DataSet[(K, Long)] = { + wrap(jgraph.outDegrees).map(javatuple => (javatuple.f0, javatuple.f1)) + } + + /** + * Return the degree of all vertices in the graph + * + * @return A DataSet of Tuple2 + */ + def getDegrees(): DataSet[(K, Long)] = { + wrap(jgraph.getDegrees).map(javatuple => (javatuple.f0, javatuple.f1)) + } + + /** + * This operation adds all inverse-direction edges to the graph. + * + * @return the undirected graph. + */ + def getUndirected(): Graph[K, VV, EV] = { + new Graph(jgraph.getUndirected) + } + + /** + * Reverse the direction of the edges in the graph + * + * @return a new graph with all edges reversed + * @throws UnsupportedOperationException + */ + def reverse(): Graph[K, VV, EV] = { + new Graph(jgraph.reverse()) + } + + /** + * Compute an aggregate over the edges of each vertex. The function applied + * on the edges has access to the vertex value. + * + * @param edgesFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @tparam T the output type + * @return a dataset of a T + */ + def groupReduceOnEdges[T: TypeInformation : ClassTag](edgesFunction: + EdgesFunctionWithVertexValue[K, VV, EV, + T], direction: EdgeDirection): + DataSet[T] = { + wrap(jgraph.groupReduceOnEdges(edgesFunction, direction, createTypeInformation[T])) + } + + /** + * Compute an aggregate over the edges of each vertex. The function applied + * on the edges has access to the vertex value. + * + * @param edgesFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @tparam T the output type + * @return a dataset of a T + */ + def groupReduceOnEdges[T: TypeInformation : ClassTag](edgesFunction: EdgesFunction[K, EV, T], + direction: EdgeDirection): DataSet[T] = { + wrap(jgraph.groupReduceOnEdges(edgesFunction, direction, createTypeInformation[T])) + } + + /** + * Compute an aggregate over the neighbors (edges and vertices) of each + * vertex. The function applied on the neighbors has access to the vertex + * value. + * + * @param neighborsFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @tparam T the output type + * @return a dataset of a T + */ + def groupReduceOnNeighbors[T: TypeInformation : ClassTag](neighborsFunction: + NeighborsFunctionWithVertexValue[K, + VV, EV, T], direction: + EdgeDirection): DataSet[T] = { + wrap(jgraph.groupReduceOnNeighbors(neighborsFunction, direction, createTypeInformation[T])) + } + + /** + * Compute an aggregate over the neighbors (edges and vertices) of each + * vertex. + * + * @param neighborsFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @tparam T the output type + * @return a dataset of a T + */ + def groupReduceOnNeighbors[T: TypeInformation : ClassTag](neighborsFunction: + NeighborsFunction[K, VV, EV, T], + direction: EdgeDirection): + DataSet[T] = { + wrap(jgraph.groupReduceOnNeighbors(neighborsFunction, direction, createTypeInformation[T])) + } + + /** + * @return a long integer representing the number of vertices + */ + def numberOfVertices(): Long = { + jgraph.numberOfVertices() + } + + /** + * @return a long integer representing the number of edges + */ + def numberOfEdges(): Long = { + jgraph.numberOfEdges() + } + + /** + * @return The IDs of the vertices as DataSet + */ + def getVertexIds(): DataSet[K] = { + wrap(jgraph.getVertexIds) + } + + /** + * @return The IDs of the edges as DataSet + */ + def getEdgeIds(): DataSet[(K, K)] = { + wrap(jgraph.getEdgeIds).map(jtuple => (jtuple.f0, jtuple.f1)) + } + + /** + * Adds the input vertex to the graph. If the vertex already + * exists in the graph, it will not be added again. + * + * @param vertex the vertex to be added + * @return the new graph containing the existing vertices as well as the one just added + */ + def addVertex(vertex: Vertex[K, VV]) = { + wrapGraph(jgraph.addVertex(vertex)) + } + + /** + * Adds the given edge to the graph. If the source and target vertices do + * not exist in the graph, they will also be added. + * + * @param source the source vertex of the edge + * @param target the target vertex of the edge + * @param edgeValue the edge value + * @return the new graph containing the existing vertices and edges plus the + * newly added edge + */ + def addEdge(source: Vertex[K, VV], target: Vertex[K, VV], edgeValue: EV) = { + wrapGraph(jgraph.addEdge(source, target, edgeValue)) + } + + /** + * Removes the given vertex and its edges from the graph. + * + * @param vertex the vertex to remove + * @return the new graph containing the existing vertices and edges without + * the removed vertex and its edges + */ + def removeVertex(vertex: Vertex[K, VV]) = { + wrapGraph(jgraph.removeVertex(vertex)) + } + + /** + * Removes all edges that match the given edge from the graph. + * + * @param edge the edge to remove + * @return the new graph containing the existing vertices and edges without + * the removed edges + */ + def removeEdge(edge: Edge[K, EV]) = { + wrapGraph(jgraph.removeEdge(edge)) + } + + /** + * Performs union on the vertices and edges sets of the input graphs + * removing duplicate vertices but maintaining duplicate edges. + * + * @param graph the graph to perform union with + * @return a new graph + */ + def union(graph: Graph[K, VV, EV]) = { + wrapGraph(jgraph.union(graph.getWrappedGraph)) + } + + /** + * Compute an aggregate over the neighbor values of each + * vertex. + * + * @param reduceNeighborsFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value) + */ + def reduceOnNeighbors(reduceNeighborsFunction: ReduceNeighborsFunction[VV], direction: + EdgeDirection): DataSet[(K, VV)] = { + wrap(jgraph.reduceOnNeighbors(reduceNeighborsFunction, direction)).map(jtuple => (jtuple + .f0, jtuple.f1)) + } + + /** + * Compute an aggregate over the edge values of each vertex. + * + * @param reduceEdgesFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @return a Dataset containing one value per vertex(vertex key, aggegate edge value) + * @throws IllegalArgumentException + */ + def reduceOnEdges(reduceEdgesFunction: ReduceEdgesFunction[EV], direction: EdgeDirection): + DataSet[(K, EV)] = { + wrap(jgraph.reduceOnEdges(reduceEdgesFunction, direction)).map(jtuple => (jtuple.f0, + jtuple.f1)) + } + + def run(algorithm: GraphAlgorithm[K, VV, EV]) = { + wrapGraph(jgraph.run(algorithm)) + } + + /** + * Runs a Vertex-Centric iteration on the graph. + * No configuration options are provided. + * + * @param vertexUpdateFunction the vertex update function + * @param messagingFunction the messaging function + * @param maxIterations maximum number of iterations to perform + * + * @return the updated Graph after the vertex-centric iteration has converged or + * after maximumNumberOfIterations. + */ + def runVertexCentricIteration[M](vertexUpdateFunction: VertexUpdateFunction[K, VV, M], + messagingFunction: MessagingFunction[K, VV, M, EV], + maxIterations: Int): Graph[K, VV, EV] = { + wrapGraph(jgraph.runVertexCentricIteration(vertexUpdateFunction, messagingFunction, + maxIterations)) + } + + /** + * Runs a Vertex-Centric iteration on the graph with configuration options. + * + * @param vertexUpdateFunction the vertex update function + * @param messagingFunction the messaging function + * @param maxIterations maximum number of iterations to perform + * @param parameters the iteration configuration parameters + * + * @return the updated Graph after the vertex-centric iteration has converged or + * after maximumNumberOfIterations. + */ + def runVertexCentricIteration[M](vertexUpdateFunction: VertexUpdateFunction[K, VV, M], + messagingFunction: MessagingFunction[K, VV, M, EV], + maxIterations: Int, parameters: VertexCentricConfiguration): + Graph[K, VV, EV] = { + wrapGraph(jgraph.runVertexCentricIteration(vertexUpdateFunction, messagingFunction, + maxIterations, parameters)) + } + + /** + * Runs a Gather-Sum-Apply iteration on the graph. + * No configuration options are provided. + * + * @param gatherFunction the gather function collects information about adjacent + * vertices and edges + * @param sumFunction the sum function aggregates the gathered information + * @param applyFunction the apply function updates the vertex values with the aggregates + * @param maxIterations maximum number of iterations to perform + * @tparam M the intermediate type used between gather, sum and apply + * + * @return the updated Graph after the gather-sum-apply iteration has converged or + * after maximumNumberOfIterations. + */ + def runGatherSumApplyIteration[M](gatherFunction: GatherFunction[VV, EV, M], sumFunction: + SumFunction[VV, EV, M], applyFunction: ApplyFunction[K, VV, M], maxIterations: Int): Graph[K, + VV, EV] = { + wrapGraph(jgraph.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction, + maxIterations)) + } + + /** + * Runs a Gather-Sum-Apply iteration on the graph with configuration options. + * + * @param gatherFunction the gather function collects information about adjacent + * vertices and edges + * @param sumFunction the sum function aggregates the gathered information + * @param applyFunction the apply function updates the vertex values with the aggregates + * @param maxIterations maximum number of iterations to perform + * @param parameters the iteration configuration parameters + * @tparam M the intermediate type used between gather, sum and apply + * + * @return the updated Graph after the gather-sum-apply iteration has converged or + * after maximumNumberOfIterations. + */ + def runGatherSumApplyIteration[M](gatherFunction: GatherFunction[VV, EV, M], sumFunction: + SumFunction[VV, EV, M], applyFunction: ApplyFunction[K, VV, M], maxIterations: Int, + parameters: GSAConfiguration): Graph[K, VV, EV] = { + wrapGraph(jgraph.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction, + maxIterations, parameters)) + } +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala new file mode 100644 index 0000000000000..ca15dabdccace --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala @@ -0,0 +1,37 @@ +/* + * 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.graph.scala + +import org.apache.flink.api.java.tuple.Tuple3 +import org.apache.flink.graph.{Edge, Vertex} +import org.apache.flink.util.Collector + + +abstract class NeighborsFunction[K, VV, EV, T] extends org.apache.flink.graph +.NeighborsFunction[K, VV, EV, T] { + + def iterateNeighbors(neighbors: Iterable[(K, Edge[K, EV], Vertex[K, VV])], out: Collector[T]) + + override def iterateNeighbors(neighbors: java.lang.Iterable[Tuple3[K, Edge[K, EV], Vertex[K, + VV]]], out: Collector[T]) = { + val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(neighbors) + .map(jtuple => (jtuple.f0, jtuple.f1, jtuple.f2)) + iterateNeighbors(scalaIterable, out) + } +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala new file mode 100644 index 0000000000000..cefc277c00ad2 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala @@ -0,0 +1,40 @@ +/* + * 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.graph.scala + +import java.lang + +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.graph.{Edge, Vertex} +import org.apache.flink.util.Collector + + +abstract class NeighborsFunctionWithVertexValue[K, VV, EV, T] extends org.apache.flink.graph +.NeighborsFunctionWithVertexValue[K, VV, EV, T] { + + def iterateNeighbors(vertex: Vertex[K, VV], neighbors: Iterable[(Edge[K, EV], Vertex[K, VV]) + ], out: Collector[T]): Unit + + override def iterateNeighbors(vertex: Vertex[K, VV], neighbors: lang.Iterable[Tuple2[Edge[K, + EV], Vertex[K, VV]]], out: Collector[T]): Unit = { + val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(neighbors) + .map(jtuple => (jtuple.f0, jtuple.f1)) + iterateNeighbors(vertex, scalaIterable, out) + } +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala new file mode 100644 index 0000000000000..159a1003554fe --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala @@ -0,0 +1,30 @@ +/* + * 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.graph + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.graph.{Graph => JGraph} + +import _root_.scala.reflect.ClassTag + + +package object scala { + private[flink] def wrapGraph[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, + EV: TypeInformation : ClassTag](javagraph: JGraph[K, VV, EV]) = new Graph[K, VV, EV](javagraph) +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala new file mode 100644 index 0000000000000..0d7d2afffc08b --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala @@ -0,0 +1,31 @@ +/* + * 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.graph.scala.utils + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.graph.Edge + +class EdgeToTuple3Map[K, EV] extends MapFunction[Edge[K, EV], (K, K, EV)] { + + private val serialVersionUID: Long = 1L + + override def map(value: Edge[K, EV]): (K, K, EV) = { + (value.getSource, value.getTarget, value.getValue) + } +} diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala new file mode 100644 index 0000000000000..de77832ec67f1 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala @@ -0,0 +1,31 @@ +/* + * 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.graph.scala.utils + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.graph.Vertex + +class VertexToTuple2Map[K, VV] extends MapFunction[Vertex[K, VV], (K, VV)] { + + private val serialVersionUID: Long = 1L + + override def map(value: Vertex[K, VV]): (K, VV) = { + (value.getId, value.getValue) + } +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala new file mode 100644 index 0000000000000..1c2cf54d33a56 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala @@ -0,0 +1,55 @@ +/* + * 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.graph.scala.test + +import org.apache.flink.api.scala._ +import org.apache.flink.graph.{Edge, Vertex} + +object TestGraphUtils { + + def getLongLongVertexData(env: ExecutionEnvironment): DataSet[Vertex[Long, Long]] = { + return env.fromCollection(getLongLongVertices) + } + + def getLongLongEdgeData(env: ExecutionEnvironment): DataSet[Edge[Long, Long]] = { + return env.fromCollection(getLongLongEdges) + } + + def getLongLongVertices: List[Vertex[Long, Long]] = { + List( + new Vertex[Long, Long](1L, 1L), + new Vertex[Long, Long](2L, 2L), + new Vertex[Long, Long](3L, 3L), + new Vertex[Long, Long](4L, 4L), + new Vertex[Long, Long](5L, 5L) + ) + } + + def getLongLongEdges: List[Edge[Long, Long]] = { + List( + new Edge[Long, Long](1L, 2L, 12L), + new Edge[Long, Long](1L, 3L, 13L), + new Edge[Long, Long](2L, 3L, 23L), + new Edge[Long, Long](3L, 4L, 34L), + new Edge[Long, Long](3L, 5L, 35L), + new Edge[Long, Long](4L, 5L, 45L), + new Edge[Long, Long](5L, 1L, 51L) + ) + } +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala new file mode 100644 index 0000000000000..98dbbe9254e7c --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala @@ -0,0 +1,88 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.scala._ +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class DegreesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testInDegrees { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.inDegrees().writeAsCsv(resultPath) + env.execute + expectedResult = "1,1\n" + "2,1\n" + "3,2\n" + "4,1\n" + "5,2\n" + } + + @Test + @throws(classOf[Exception]) + def testOutDegrees { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.outDegrees().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2\n" + "2,1\n" + "3,2\n" + "4,1\n" + "5,1\n" + } + + @Test + @throws(classOf[Exception]) + def testGetDegrees { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.getDegrees().writeAsCsv(resultPath) + env.execute + expectedResult = "1,3\n" + "2,2\n" + "3,4\n" + "4,2\n" + "5,3\n" + } +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala new file mode 100644 index 0000000000000..687b0a7581f8a --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala @@ -0,0 +1,171 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.scala._ +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.graph.{Edge, Vertex} +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class GraphMutationsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testAddVertex { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + + val newgraph = graph.addVertex(new Vertex[Long, Long](6L, 6L)) + newgraph.getVerticesAsTuple2.writeAsCsv(resultPath) + env.execute + expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n" + } + + @Test + @throws(classOf[Exception]) + def testAddVertexExisting { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.addVertex(new Vertex[Long, Long](1L, 1L)) + newgraph.getVerticesAsTuple2.writeAsCsv(resultPath) + env.execute + expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + } + + @Test + @throws(classOf[Exception]) + def testAddVertexNoEdges { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.addVertex(new Vertex[Long, Long](6L, 6L)) + newgraph.getVerticesAsTuple2.writeAsCsv(resultPath) + env.execute + expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n" + } + + @Test + @throws(classOf[Exception]) + def testRemoveVertex { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.removeVertex(new Vertex[Long, Long](5L, 5L)) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + } + + @Test + @throws(classOf[Exception]) + def testRemoveInvalidVertex { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.removeVertex(new Vertex[Long, Long](6L, 6L)) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," + + "45\n" + "5,1,51\n" + } + + @Test + @throws(classOf[Exception]) + def testAddEdge { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.addEdge(new Vertex[Long, Long](6L, 6L), new Vertex[Long, Long](1L, + 1L), 61L) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," + + "45\n" + "5,1,51\n" + "6,1,61\n" + } + + @Test + @throws(classOf[Exception]) + def testAddExistingEdge { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.addEdge(new Vertex[Long, Long](1L, 1L), new Vertex[Long, Long](2L, + 2L), 12L) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5," + + "35\n" + "4,5,45\n" + "5,1,51\n" + } + + @Test + @throws(classOf[Exception]) + def testRemoveEdge { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.removeEdge(new Edge[Long, Long](5L, 1L, 51L)) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n" + } + + @Test + @throws(classOf[Exception]) + def testRemoveInvalidEdge { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val newgraph = graph.removeEdge(new Edge[Long, Long](6L, 1L, 61L)) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," + + "45\n" + "5,1,51\n" + } +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala new file mode 100644 index 0000000000000..d49e56559c777 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala @@ -0,0 +1,238 @@ +/* + * 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. + */ + +import org.apache.flink.api.common.functions.FilterFunction +import org.apache.flink.api.scala._ +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.graph.{Edge, Vertex} +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class GraphOperationsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testUndirected { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.getUndirected().getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "2,1,12\n" + "1,3,13\n" + "3,1,13\n" + "2,3,23\n" + "3,2," + + "23\n" + "3,4,34\n" + "4,3,34\n" + "3,5,35\n" + "5,3,35\n" + "4,5,45\n" + "5,4,45\n" + + "5,1,51\n" + "1,5,51\n" + } + + @Test + @throws(classOf[Exception]) + def testReverse { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.reverse().getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "2,1,12\n" + "3,1,13\n" + "3,2,23\n" + "4,3,34\n" + "5,3,35\n" + "5,4," + + "45\n" + "1,5,51\n" + } + + @Test + @throws(classOf[Exception]) + def testSubGraph { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.subgraph(new FilterFunction[Vertex[Long, Long]] { + @throws(classOf[Exception]) + def filter(vertex: Vertex[Long, Long]): Boolean = { + return (vertex.getValue > 2) + } + }, new FilterFunction[Edge[Long, Long]] { + + @throws(classOf[Exception]) + override def filter(edge: Edge[Long, Long]): Boolean = { + return (edge.getValue > 34) + } + }).getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "3,5,35\n" + "4,5,45\n" + } + + @Test + @throws(classOf[Exception]) + def testSubGraphSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.subgraph( + vertex => vertex.getValue > 2, + edge => edge.getValue > 34 + ).getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "3,5,35\n" + "4,5,45\n" + } + + @Test + @throws(classOf[Exception]) + def testFilterOnVertices { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.filterOnVertices(new FilterFunction[Vertex[Long, Long]] { + @throws(classOf[Exception]) + def filter(vertex: Vertex[Long, Long]): Boolean = { + vertex.getValue > 2 + } + }).getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n" + } + + @Test + @throws(classOf[Exception]) + def testFilterOnVerticesSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.filterOnVertices( + vertex => vertex.getValue > 2 + ).getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n" + } + + @Test + @throws(classOf[Exception]) + def testFilterOnEdges { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.filterOnEdges(new FilterFunction[Edge[Long, Long]] { + @throws(classOf[Exception]) + def filter(edge: Edge[Long, Long]): Boolean = { + edge.getValue > 34 + } + }).getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "3,5,35\n" + "4,5,45\n" + "5,1,51\n" + } + + @Test + @throws(classOf[Exception]) + def testFilterOnEdgesSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.filterOnEdges( + edge => edge.getValue > 34 + ).getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "3,5,35\n" + "4,5,45\n" + "5,1,51\n" + } + + @Test + @throws(classOf[Exception]) + def testNumberOfVertices { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + env.fromElements(graph.numberOfVertices).writeAsText(resultPath) + env.execute + expectedResult = "5" + } + + @Test + @throws(classOf[Exception]) + def testNumberOfEdges { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + env.fromElements(graph.numberOfEdges).writeAsText(resultPath) + env.execute + expectedResult = "7" + } + + @Test + @throws(classOf[Exception]) + def testVertexIds { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.getVertexIds.writeAsText(resultPath) + env.execute + expectedResult = "1\n2\n3\n4\n5\n" + } + + @Test + @throws(classOf[Exception]) + def testEdgesIds { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.getEdgeIds.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2\n" + "1,3\n" + "2,3\n" + "3,4\n" + "3,5\n" + "4,5\n" + "5,1\n" + } + + @Test + @throws(classOf[Exception]) + def testUnion { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]]( + new Vertex[Long, Long](6L, 6L) + ) + val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]]( + new Edge[Long, Long](6L, 1L, 61L) + ) + + val newgraph = graph.union(Graph.fromCollection(vertices, edges, env)) + newgraph.getEdgesAsTuple3.writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," + + "45\n" + "5,1,51\n" + "6,1,61\n" + } +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala new file mode 100644 index 0000000000000..e19463e0b3b05 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala @@ -0,0 +1,170 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.scala._ +import org.apache.flink.graph.Edge +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.graph.scala.utils.EdgeToTuple3Map +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class JoinWithEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testWithEdgesInputDataset { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithEdges(graph.getEdges.map(new + EdgeToTuple3Map[Long, Long]), new AddValuesMapper) + result.getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,46\n" + "3,4,68\n" + "3,5,70\n" + "4,5," + + "90\n" + "5,1,102\n" + } + + @Test + @throws(classOf[Exception]) + def testWithEdgesInputDatasetSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithEdges(graph.getEdges.map(new + EdgeToTuple3Map[Long, Long]), (originalValue: Long, tupleValue: Long) => + originalValue + tupleValue) + result.getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,46\n" + "3,4,68\n" + "3,5,70\n" + "4,5," + + "90\n" + "5,1,102\n" + } + + @Test + @throws(classOf[Exception]) + def testWithEdgesOnSource { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnSource[Long](graph.getEdges + .map(new ProjectSourceAndValueMapper), (originalValue: Long, tupleValue: Long) => + originalValue + tupleValue) + result.getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,24\n" + "1,3,25\n" + "2,3,46\n" + "3,4,68\n" + "3,5,69\n" + "4,5," + + "90\n" + "5,1,102\n" + } + + @Test + @throws(classOf[Exception]) + def testWithEdgesOnSourceSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnSource[Long](graph.getEdges + .map(new ProjectSourceAndValueMapper), (originalValue: Long, tupleValue: Long) => + originalValue + tupleValue) + result.getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,24\n" + "1,3,25\n" + "2,3,46\n" + "3,4,68\n" + "3,5,69\n" + "4,5," + + "90\n" + "5,1,102\n" + } + + @Test + @throws(classOf[Exception]) + def testWithEdgesOnTarget { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnTarget[Long](graph.getEdges + .map(new ProjectTargetAndValueMapper), (originalValue: Long, tupleValue: Long) => + originalValue + tupleValue) + result.getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,36\n" + "3,4,68\n" + "3,5,70\n" + "4,5," + + "80\n" + "5,1,102\n" + } + + @Test + @throws(classOf[Exception]) + def testWithEdgesOnTargetSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnTarget[Long](graph.getEdges + .map(new ProjectTargetAndValueMapper), (originalValue: Long, tupleValue: Long) => + originalValue + tupleValue) + result.getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,36\n" + "3,4,68\n" + "3,5,70\n" + "4,5," + + "80\n" + "5,1,102\n" + } + + + final class AddValuesMapper extends MapFunction[(Long, Long), Long] { + @throws(classOf[Exception]) + def map(tuple: (Long, Long)): Long = { + tuple._1 + tuple._2 + } + } + + final class ProjectSourceAndValueMapper extends MapFunction[Edge[Long, Long], (Long, Long)] { + @throws(classOf[Exception]) + def map(edge: Edge[Long, Long]): (Long, Long) = { + (edge.getSource, edge.getValue) + } + } + + final class ProjectTargetAndValueMapper extends MapFunction[Edge[Long, Long], (Long, Long)] { + @throws(classOf[Exception]) + def map(edge: Edge[Long, Long]): (Long, Long) = { + (edge.getTarget, edge.getValue) + } + } + +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala new file mode 100644 index 0000000000000..4b8f3542455f1 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala @@ -0,0 +1,93 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.scala._ +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.graph.scala.utils.VertexToTuple2Map +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class JoinWithVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testJoinWithVertexSet { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result: Graph[Long, Long, Long] = graph.joinWithVertices(graph.getVertices.map(new + VertexToTuple2Map[Long, Long]), new AddValuesMapper) + result.getVerticesAsTuple2().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2\n" + "2,4\n" + "3,6\n" + "4,8\n" + "5,10\n" + } + + @Test + @throws(classOf[Exception]) + def testJoinWithVertexSetSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val tupleSet = graph.getVertices.map(new VertexToTuple2Map[Long, Long]) + val result: Graph[Long, Long, Long] = graph.joinWithVertices[Long](tupleSet, + (originalvalue: Long, tuplevalue: Long) => originalvalue + tuplevalue) + result.getVerticesAsTuple2().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2\n" + "2,4\n" + "3,6\n" + "4,8\n" + "5,10\n" + } + + + final class AddValuesMapper extends MapFunction[(Long, Long), Long] { + @throws(classOf[Exception]) + def map(tuple: (Long, Long)): Long = { + tuple._1 + tuple._2 + } + } + +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala new file mode 100644 index 0000000000000..7e5ad14659de8 --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala @@ -0,0 +1,102 @@ +/* + * 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.graph.scala.test.operations + + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.scala._ +import org.apache.flink.graph.Edge +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class MapEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testWithSameValue { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.mapEdges(new AddOneMapper) + .getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,13\n" + + "1,3,14\n" + "" + + "2,3,24\n" + + "3,4,35\n" + + "3,5,36\n" + + "4,5,46\n" + + "5,1,52\n" + } + + @Test + @throws(classOf[Exception]) + def testWithSameValueSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.mapEdges(edge => edge.getValue + 1) + .getEdgesAsTuple3().writeAsCsv(resultPath) + env.execute + expectedResult = "1,2,13\n" + + "1,3,14\n" + "" + + "2,3,24\n" + + "3,4,35\n" + + "3,5,36\n" + + "4,5,46\n" + + "5,1,52\n" + } + + final class AddOneMapper extends MapFunction[Edge[Long, Long], Long] { + @throws(classOf[Exception]) + def map(edge: Edge[Long, Long]): Long = { + edge.getValue + 1 + } + } + +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala new file mode 100644 index 0000000000000..a22cfbd3f93ca --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala @@ -0,0 +1,99 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.scala._ +import org.apache.flink.graph.Vertex +import org.apache.flink.graph.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class MapVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends +MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testWithSameValue { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.mapVertices(new AddOneMapper) + .getVerticesAsTuple2().writeAsCsv(resultPath) + env.execute + + expectedResult = "1,2\n" + + "2,3\n" + + "3,4\n" + + "4,5\n" + + "5,6\n"; + } + + @Test + @throws(classOf[Exception]) + def testWithSameValueSugar { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.mapVertices(vertex => vertex.getValue + 1) + .getVerticesAsTuple2().writeAsCsv(resultPath) + env.execute + + expectedResult = "1,2\n" + + "2,3\n" + + "3,4\n" + + "4,5\n" + + "5,6\n"; + } + + final class AddOneMapper extends MapFunction[Vertex[Long, Long], Long] { + @throws(classOf[Exception]) + def map(vertex: Vertex[Long, Long]): Long = { + vertex.getValue + 1 + } + } + +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala new file mode 100644 index 0000000000000..6ed383aae8a8a --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala @@ -0,0 +1,173 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.scala._ +import org.apache.flink.graph._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.graph.scala.{EdgesFunction, EdgesFunctionWithVertexValue, Graph} +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.apache.flink.util.Collector +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class ReduceOnEdgesMethodsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) + extends MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testAllNeighborsWithValueGreaterThanFour { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result = graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour, + EdgeDirection.ALL) + result.writeAsCsv(resultPath) + env.execute + + + expectedResult = "5,1\n" + "5,3\n" + "5,4" + } + + + @Test + @throws(classOf[Exception]) + def testAllNeighbors { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result = graph.groupReduceOnEdges(new SelectNeighbors, EdgeDirection.ALL) + result.writeAsCsv(resultPath) + env.execute + + + expectedResult = "1,2\n" + "1,3\n" + "1,5\n" + "2,1\n" + "2,3\n" + "3,1\n" + "3,2\n" + + "3,4\n" + "3,5\n" + "4,3\n" + "4,5\n" + "5,1\n" + "5,3\n" + "5,4" + } + + @Test + @throws(classOf[Exception]) + def testLowestWeightOutNeighborNoValue { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val verticesWithLowestOutNeighbor: DataSet[(Long, Long)] = graph.reduceOnEdges(new + SelectMinWeightNeighborNoValue, EdgeDirection.OUT) + verticesWithLowestOutNeighbor.writeAsCsv(resultPath) + env.execute + expectedResult = "1,12\n" + "2,23\n" + "3,34\n" + "4,45\n" + "5,51\n" + } + + @Test + @throws(classOf[Exception]) + def testLowestWeightInNeighborNoValue { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val verticesWithLowestOutNeighbor: DataSet[(Long, Long)] = graph.reduceOnEdges(new + SelectMinWeightNeighborNoValue, EdgeDirection.IN) + verticesWithLowestOutNeighbor.writeAsCsv(resultPath) + env.execute + expectedResult = "1,51\n" + "2,12\n" + "3,13\n" + "4,34\n" + "5,35\n" + } + + @Test + @throws(classOf[Exception]) + def testMaxWeightAllNeighbors { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val verticesWithMaxEdgeWeight: DataSet[(Long, Long)] = graph.reduceOnEdges(new + SelectMaxWeightNeighborNoValue, EdgeDirection.ALL) + verticesWithMaxEdgeWeight.writeAsCsv(resultPath) + env.execute + expectedResult = "1,51\n" + "2,23\n" + "3,35\n" + "4,45\n" + "5,51\n" + } + + final class SelectNeighborsValueGreaterThanFour extends EdgesFunctionWithVertexValue[Long, + Long, Long, (Long, Long)] { + @throws(classOf[Exception]) + override def iterateEdges(v: Vertex[Long, Long], edges: Iterable[Edge[Long, Long]], out: + Collector[(Long, Long)]): Unit = { + for (edge <- edges) { + if (v.getValue > 4) { + if (v.getId == edge.getTarget) { + out.collect((v.getId, edge.getSource)) + } + else { + out.collect((v.getId, edge.getTarget)) + } + } + } + } + } + + final class SelectNeighbors extends EdgesFunction[Long, Long, (Long, Long)] { + @throws(classOf[Exception]) + override def iterateEdges(edges: Iterable[(Long, Edge[Long, Long])], out: Collector[ + (Long, Long)]) { + for (edge <- edges) { + if (edge._1.equals(edge._2.getTarget)) { + out.collect(new Tuple2[Long, Long](edge._1, edge._2.getSource)) + } + else { + out.collect(new Tuple2[Long, Long](edge._1, edge._2.getTarget)) + } + } + } + } + + final class SelectMinWeightNeighborNoValue extends ReduceEdgesFunction[Long] { + override def reduceEdges(firstEdgeValue: Long, secondEdgeValue: Long): Long = { + Math.min(firstEdgeValue, secondEdgeValue) + } + } + + final class SelectMaxWeightNeighborNoValue extends ReduceEdgesFunction[Long] { + override def reduceEdges(firstEdgeValue: Long, secondEdgeValue: Long): Long = { + Math.max(firstEdgeValue, secondEdgeValue) + } + } + +} diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala new file mode 100644 index 0000000000000..52e6d7a66868a --- /dev/null +++ b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala @@ -0,0 +1,144 @@ +/* + * 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.graph.scala.test.operations + +import org.apache.flink.api.scala._ +import org.apache.flink.graph.scala.test.TestGraphUtils +import org.apache.flink.graph.scala.{NeighborsFunctionWithVertexValue, _} +import org.apache.flink.graph.{Edge, EdgeDirection, ReduceNeighborsFunction, Vertex} +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.apache.flink.util.Collector +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +@RunWith(classOf[Parameterized]) +class ReduceOnNeighborMethodsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) + extends MultipleProgramsTestBase(mode) { + + private var resultPath: String = null + private var expectedResult: String = null + + var tempFolder: TemporaryFolder = new TemporaryFolder() + + @Rule + def getFolder(): TemporaryFolder = { + tempFolder; + } + + @Before + @throws(classOf[Exception]) + def before { + resultPath = tempFolder.newFile.toURI.toString + } + + @After + @throws(classOf[Exception]) + def after { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + @throws(classOf[Exception]) + def testSumOfAllNeighborsNoValue { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.reduceOnNeighbors(new SumNeighbors, EdgeDirection.ALL).writeAsCsv(resultPath) + env.execute + expectedResult = "1,10\n" + "2,4\n" + "3,12\n" + "4,8\n" + "5,8\n" + } + + @Test + @throws(classOf[Exception]) + def testSumOfOutNeighborsNoValue { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + graph.reduceOnNeighbors(new SumNeighbors, EdgeDirection.OUT).writeAsCsv(resultPath) + env.execute + expectedResult = "1,5\n" + "2,3\n" + "3,9\n" + "4,5\n" + "5,1\n" + } + + @Test + @throws(classOf[Exception]) + def testSumOfAllNeighbors { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result = graph.groupReduceOnNeighbors(new SumAllNeighbors, EdgeDirection.ALL) + result.writeAsCsv(resultPath) + env.execute + expectedResult = "1,11\n" + "2,6\n" + "3,15\n" + "4,12\n" + "5,13\n" + } + + @Test + @throws(classOf[Exception]) + def testSumOfInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo = { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils + .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env) + val result = graph.groupReduceOnNeighbors(new + SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo, EdgeDirection.IN) + result.writeAsCsv(resultPath) + env.execute + expectedResult = "3,59\n" + "3,118\n" + "4,204\n" + "4,102\n" + "5,570\n" + "5,285" + } + + final class SumNeighbors extends ReduceNeighborsFunction[Long] { + override def reduceNeighbors(firstNeighbor: Long, secondNeighbor: Long): Long = { + firstNeighbor + secondNeighbor + } + } + + final class SumAllNeighbors extends NeighborsFunctionWithVertexValue[Long, Long, Long, (Long, + Long)] { + @throws(classOf[Exception]) + def iterateNeighbors(vertex: Vertex[Long, Long], neighbors: Iterable[(Edge[Long, Long], + Vertex[Long, Long])], out: Collector[(Long, Long)]) { + var sum: Long = 0 + for (neighbor <- neighbors) { + sum += neighbor._2.getValue + } + out.collect((vertex.getId, sum + vertex.getValue)) + } + } + + final class SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo extends + NeighborsFunction[Long, Long, Long, (Long, Long)] { + @throws(classOf[Exception]) + def iterateNeighbors(neighbors: Iterable[(Long, Edge[Long, Long], Vertex[Long, Long])], + out: Collector[(Long, Long)]) { + var sum: Long = 0 + var next: (Long, Edge[Long, Long], Vertex[Long, Long]) = null + val neighborsIterator: Iterator[(Long, Edge[Long, Long], Vertex[Long, Long])] = + neighbors.iterator + while (neighborsIterator.hasNext) { + next = neighborsIterator.next + sum += next._3.getValue * next._2.getValue + } + if (next._1 > 2) { + out.collect(new Tuple2[Long, Long](next._1, sum)) + out.collect(new Tuple2[Long, Long](next._1, sum * 2)) + } + } + } + +} diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java index ff279491d9711..8552c01a31b54 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java @@ -383,6 +383,17 @@ public Graph mapVertices(final MapFunction, NV> ma TypeInformation> returnType = (TypeInformation>) new TupleTypeInfo( Vertex.class, keyType, valueType); + return mapVertices(mapper, returnType); + } + + /** + * Apply a function to the attribute of each vertex in the graph. + * + * @param mapper the map function to apply. + * @param returnType the explicit return type. + * @return a new graph + */ + public Graph mapVertices(final MapFunction, NV> mapper, TypeInformation> returnType) { DataSet> mappedVertices = vertices.map( new MapFunction, Vertex>() { public Vertex map(Vertex value) throws Exception { @@ -411,6 +422,18 @@ public Graph mapEdges(final MapFunction, NV> mapper) TypeInformation> returnType = (TypeInformation>) new TupleTypeInfo( Edge.class, keyType, keyType, valueType); + return mapEdges(mapper, returnType); + } + + /** + * Apply a function to the attribute of each edge in the graph. + * + * @param mapper the map function to apply. + * @param returnType the explicit return type. + * @return a new graph + */ + @SuppressWarnings({ "unchecked", "rawtypes" }) + public Graph mapEdges(final MapFunction, NV> mapper, TypeInformation> returnType) { DataSet> mappedEdges = edges.map( new MapFunction, Edge>() { public Edge map(Edge value) throws Exception { @@ -752,6 +775,38 @@ public DataSet groupReduceOnEdges(EdgesFunctionWithVertexValue + * the output type + * @param typeInfo the explicit return type. + * @return a dataset of a T + * @throws IllegalArgumentException + */ + public DataSet groupReduceOnEdges(EdgesFunctionWithVertexValue edgesFunction, + EdgeDirection direction, TypeInformation typeInfo) throws IllegalArgumentException { + + switch (direction) { + case IN: + return vertices.coGroup(edges).where(0).equalTo(1) + .with(new ApplyCoGroupFunction(edgesFunction)).returns(typeInfo); + case OUT: + return vertices.coGroup(edges).where(0).equalTo(0) + .with(new ApplyCoGroupFunction(edgesFunction)).returns(typeInfo); + case ALL: + return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode())) + .where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges(edgesFunction)).returns(typeInfo); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + /** * Compute an aggregate over the edges of each vertex. The function applied * on the edges only has access to the vertex id (not the vertex value). @@ -785,6 +840,40 @@ public DataSet groupReduceOnEdges(EdgesFunction edgesFunction, } } + /** + * Compute an aggregate over the edges of each vertex. The function applied + * on the edges only has access to the vertex id (not the vertex value). + * + * @param edgesFunction + * the function to apply to the neighborhood + * @param direction + * the edge direction (in-, out-, all-) + * @param + * the output type + * @param typeInfo the explicit return type. + * @return a dataset of T + * @throws IllegalArgumentException + */ + public DataSet groupReduceOnEdges(EdgesFunction edgesFunction, + EdgeDirection direction, TypeInformation typeInfo) throws IllegalArgumentException { + + switch (direction) { + case IN: + return edges.map(new ProjectVertexIdMap(1)) + .withForwardedFields("f1->f0") + .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)).returns(typeInfo); + case OUT: + return edges.map(new ProjectVertexIdMap(0)) + .withForwardedFields("f0") + .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)).returns(typeInfo); + case ALL: + return edges.flatMap(new EmitOneEdgePerNode()) + .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)).returns(typeInfo); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + private static final class ProjectVertexIdMap implements MapFunction< Edge, Tuple2>> { @@ -1410,6 +1499,51 @@ public DataSet groupReduceOnNeighbors(NeighborsFunctionWithVertexValue the output type + * @param typeInfo the explicit return type. + * @return a dataset of a T + * @throws IllegalArgumentException + */ + public DataSet groupReduceOnNeighbors(NeighborsFunctionWithVertexValue neighborsFunction, + EdgeDirection direction, TypeInformation typeInfo) throws IllegalArgumentException { + switch (direction) { + case IN: + // create pairs + DataSet, Vertex>> edgesWithSources = edges + .join(this.vertices).where(0).equalTo(0); + return vertices.coGroup(edgesWithSources) + .where(0).equalTo("f0.f1") + .with(new ApplyNeighborCoGroupFunction(neighborsFunction)).returns(typeInfo); + case OUT: + // create pairs + DataSet, Vertex>> edgesWithTargets = edges + .join(this.vertices).where(1).equalTo(0); + return vertices.coGroup(edgesWithTargets) + .where(0).equalTo("f0.f0") + .with(new ApplyNeighborCoGroupFunction(neighborsFunction)).returns(typeInfo); + case ALL: + // create pairs + DataSet, Vertex>> edgesWithNeighbors = edges + .flatMap(new EmitOneEdgeWithNeighborPerNode()) + .join(this.vertices).where(1).equalTo(0) + .with(new ProjectEdgeWithNeighbor()); + + return vertices.coGroup(edgesWithNeighbors) + .where(0).equalTo(0) + .with(new ApplyCoGroupFunctionOnAllNeighbors(neighborsFunction)).returns(typeInfo); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + + /** * Compute an aggregate over the neighbors (edges and vertices) of each * vertex. The function applied on the neighbors only has access to the @@ -1454,6 +1588,51 @@ public DataSet groupReduceOnNeighbors(NeighborsFunction nei } } + /** + * Compute an aggregate over the neighbors (edges and vertices) of each + * vertex. The function applied on the neighbors only has access to the + * vertex id (not the vertex value). + * + * @param neighborsFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @param the output type + * @param typeInfo the explicit return type. + * @return a dataset of a T + * @throws IllegalArgumentException + */ + public DataSet groupReduceOnNeighbors(NeighborsFunction neighborsFunction, + EdgeDirection direction, TypeInformation typeInfo) throws IllegalArgumentException { + switch (direction) { + case IN: + // create pairs + DataSet, Vertex>> edgesWithSources = edges + .join(this.vertices).where(0).equalTo(0) + .with(new ProjectVertexIdJoin(1)) + .withForwardedFieldsFirst("f1->f0"); + return edgesWithSources.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)).returns(typeInfo); + case OUT: + // create pairs + DataSet, Vertex>> edgesWithTargets = edges + .join(this.vertices).where(1).equalTo(0) + .with(new ProjectVertexIdJoin(0)) + .withForwardedFieldsFirst("f0"); + return edgesWithTargets.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)).returns(typeInfo); + case ALL: + // create pairs + DataSet, Vertex>> edgesWithNeighbors = edges + .flatMap(new EmitOneEdgeWithNeighborPerNode()) + .join(this.vertices).where(1).equalTo(0) + .with(new ProjectEdgeWithNeighbor()); + + return edgesWithNeighbors.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)).returns(typeInfo); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + private static final class ApplyNeighborGroupReduceFunction implements GroupReduceFunction, Vertex>, T>, ResultTypeQueryable { diff --git a/flink-staging/pom.xml b/flink-staging/pom.xml index b3aec14c87e8f..a0cda670054ed 100644 --- a/flink-staging/pom.xml +++ b/flink-staging/pom.xml @@ -46,6 +46,7 @@ under the License. flink-table flink-ml flink-language-binding + flink-gelly-scala From 8a59f87bd4b3ec15d9bafae8e7de6d8a55e070b1 Mon Sep 17 00:00:00 2001 From: HuangWHWHW <404823056@qq.com> Date: Sun, 16 Aug 2015 11:56:18 +0800 Subject: [PATCH 149/175] [FLINK-2529] [runtime] Remove some unused code in Execution This closes #1022 --- .../java/org/apache/flink/runtime/executiongraph/Execution.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index aa0f98148548a..64c4d470f540d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -497,8 +497,6 @@ else if (numConsumers == 0) { @Override public Boolean call() throws Exception { try { - final ExecutionGraph consumerGraph = consumerVertex.getExecutionGraph(); - consumerVertex.scheduleForExecution( consumerVertex.getExecutionGraph().getScheduler(), consumerVertex.getExecutionGraph().isQueuedSchedulingAllowed()); From 8602b7af0d048139ba90f36162e590da1050891d Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 17 Aug 2015 18:21:20 +0200 Subject: [PATCH 150/175] [tests] Minor cleanup removing duplicate mock invokables in various tests. --- .../runtime/io/disk/iomanager/IOManagerITCase.java | 4 ++-- .../iomanager/IOManagerPerformanceBenchmark.java | 6 ++++-- .../memory/MemoryManagerLazyAllocationTest.java | 13 ++----------- .../flink/runtime/memory/MemoryManagerTest.java | 12 +----------- .../flink/runtime/memory/MemorySegmentTest.java | 3 ++- 5 files changed, 11 insertions(+), 27 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java index a0a83562f529d..52908d350d374 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Random; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.junit.Assert; import org.junit.After; import org.junit.Before; @@ -34,7 +35,6 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.memory.MemoryManagerTest; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; /** @@ -82,7 +82,7 @@ public void afterTest() throws Exception { @SuppressWarnings("unchecked") public void parallelChannelsTest() throws Exception { final Random rnd = new Random(SEED); - final AbstractInvokable memOwner = new MemoryManagerTest.DummyInvokable(); + final AbstractInvokable memOwner = new DummyInvokable(); FileIOChannel.ID[] ids = new FileIOChannel.ID[NUM_CHANNELS]; BlockChannelWriter[] writers = new BlockChannelWriter[NUM_CHANNELS]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java index c71e181068078..3bdc9bd6a9dbb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java @@ -31,17 +31,19 @@ import java.nio.channels.FileChannel; import java.util.List; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.IntValue; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.flink.core.memory.InputViewDataInputStreamWrapper; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.memory.MemoryManagerTest; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; + import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -62,7 +64,7 @@ public class IOManagerPerformanceBenchmark { private static final int NUM_INTS_WRITTEN = 100000000; - private static final AbstractInvokable memoryOwner = new MemoryManagerTest.DummyInvokable(); + private static final AbstractInvokable memoryOwner = new DummyInvokable(); private DefaultMemoryManager memManager; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java index e077d08edc823..fc88207a70df3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java @@ -22,6 +22,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -195,15 +197,4 @@ private boolean allMemorySegmentsFreed(List memSegs) { } return true; } - - /** - * Utility class to serve as owner for the memory. - */ - public static final class DummyInvokable extends AbstractInvokable { - @Override - public void registerInputOutput() {} - - @Override - public void invoke() throws Exception {} - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java index c21135740d35f..c0f32ca1dd9c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.junit.Assert; import org.junit.After; import org.junit.Before; @@ -194,15 +195,4 @@ private boolean allMemorySegmentsFreed(List memSegs) { } return true; } - - /** - * Utility class to serve as owner for the memory. - */ - public static final class DummyInvokable extends AbstractInvokable { - @Override - public void registerInputOutput() {} - - @Override - public void invoke() throws Exception {} - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java index f9adb94ee0ec3..990dffa6bc1df 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.util.Random; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.junit.Assert; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; @@ -52,7 +53,7 @@ public class MemorySegmentTest { public void setUp() throws Exception{ try { this.manager = new DefaultMemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, true); - this.segment = manager.allocatePages(new MemoryManagerTest.DummyInvokable(), 1).get(0); + this.segment = manager.allocatePages(new DummyInvokable(), 1).get(0); this.random = new Random(RANDOM_SEED); } catch (Exception e) { e.printStackTrace(); From baaa3dacabdfa87c3c11882810e05dc384f24038 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 19 Aug 2015 16:16:31 +0200 Subject: [PATCH 151/175] [tests] Replaces Scala mixins for the Job/TaskManager with classes which extend the respective classes. Adds proper logger registration for sub-classes of FlinkUntypedActor. --- .../flink/runtime/akka/FlinkUntypedActor.java | 2 +- .../flink/runtime/jobmanager/JobManager.scala | 2 - .../runtime/testingUtils/TestingCluster.scala | 9 ++-- .../testingUtils/TestingJobManager.scala | 49 ++++++++++++++++--- .../testingUtils/TestingMemoryArchivist.scala | 12 ++--- .../test/util/ForkableFlinkMiniCluster.scala | 14 ++---- 6 files changed, 57 insertions(+), 31 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java index bba2aeb988d02..1456758763482 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java @@ -39,7 +39,7 @@ * a leader session ID option which is returned by getLeaderSessionID. */ public abstract class FlinkUntypedActor extends UntypedActor { - protected static Logger LOG = LoggerFactory.getLogger(FlinkUntypedActor.class); + protected Logger LOG = LoggerFactory.getLogger(getClass()); /** * This method is called by Akka if a new message has arrived for the actor. It logs the diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 5c0f468c1b1b2..92688fa65989b 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -25,7 +25,6 @@ import java.util.{UUID, Collections} import akka.actor.Status.{Failure, Success} import akka.actor._ -import _root_.akka.pattern.ask import grizzled.slf4j.Logger @@ -69,7 +68,6 @@ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps import scala.collection.JavaConverters._ -import scala.collection.JavaConversions._ /** * The job manager is responsible for receiving Flink jobs, scheduling the tasks, gathering the diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index f5a506dc5fe08..057ffeba93d1b 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -22,7 +22,7 @@ import akka.actor.{ActorRef, Props, ActorSystem} import akka.testkit.CallingThreadDispatcher import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.runtime.StreamingMode -import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager} +import org.apache.flink.runtime.jobmanager.JobManager import org.apache.flink.runtime.minicluster.FlinkMiniCluster import org.apache.flink.runtime.net.NetUtils import org.apache.flink.runtime.taskmanager.TaskManager @@ -80,11 +80,11 @@ class TestingCluster(userConfiguration: Configuration, timeout, archiveCount) = JobManager.createJobManagerComponents(configuration) - val testArchiveProps = Props(new MemoryArchivist(archiveCount) with TestingMemoryArchivist) + val testArchiveProps = Props(new TestingMemoryArchivist(archiveCount)) val archive = actorSystem.actorOf(testArchiveProps, JobManager.ARCHIVE_NAME) val jobManagerProps = Props( - new JobManager( + new TestingJobManager( configuration, executionContext, instanceManager, @@ -94,8 +94,7 @@ class TestingCluster(userConfiguration: Configuration, executionRetries, delayBetweenRetries, timeout, - streamingMode) - with TestingJobManager) + streamingMode)) val dispatcherJobManagerProps = if (synchronousDispatcher) { // disable asynchronous futures (e.g. accumulator update in Heartbeat) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 8a7297b30370a..987af4044e87e 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -21,10 +21,14 @@ package org.apache.flink.runtime.testingUtils import akka.actor.{Cancellable, Terminated, ActorRef} import akka.pattern.{ask, pipe} import org.apache.flink.api.common.JobID -import org.apache.flink.runtime.FlinkActor +import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.{StreamingMode, FlinkActor} import org.apache.flink.runtime.execution.ExecutionState +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager +import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobgraph.JobStatus import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged import org.apache.flink.runtime.messages.Messages.Disconnect import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat @@ -32,15 +36,46 @@ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged -import scala.concurrent.Future +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import scala.language.postfixOps -/** Mixin for [[TestingJobManager]] to support testing messages - */ -trait TestingJobManager extends FlinkActor { - that: JobManager => +/** JobManager implementation extended by testing messages + * + * @param flinkConfiguration + * @param executionContext + * @param instanceManager + * @param scheduler + * @param libraryCacheManager + * @param archive + * @param defaultExecutionRetries + * @param delayBetweenRetries + * @param timeout + * @param mode + */ +class TestingJobManager( + flinkConfiguration: Configuration, + executionContext: ExecutionContext, + instanceManager: InstanceManager, + scheduler: Scheduler, + libraryCacheManager: BlobLibraryCacheManager, + archive: ActorRef, + defaultExecutionRetries: Int, + delayBetweenRetries: Long, + timeout: FiniteDuration, + mode: StreamingMode) + extends JobManager( + flinkConfiguration, + executionContext, + instanceManager, + scheduler, + libraryCacheManager, + archive, + defaultExecutionRetries, + delayBetweenRetries, + timeout, + mode) { import scala.collection.JavaConverters._ import context._ @@ -60,7 +95,7 @@ trait TestingJobManager extends FlinkActor { var disconnectDisabled = false - abstract override def handleMessage: Receive = { + override def handleMessage: Receive = { handleTestingMessage orElse super.handleMessage } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala index f105cccb72135..2ccddfa7598d3 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala @@ -22,13 +22,13 @@ import org.apache.flink.runtime.{FlinkActor} import org.apache.flink.runtime.jobmanager.MemoryArchivist import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{ExecutionGraphNotFound, ExecutionGraphFound, RequestExecutionGraph} -/** - * Mixin for the [[MemoryArchivist]] to support testing messages - */ -trait TestingMemoryArchivist extends FlinkActor { - self: MemoryArchivist => +/** Memory archivist extended by testing messages + * + * @param maxEntries number of maximum number of archived jobs + */ +class TestingMemoryArchivist(maxEntries: Int) extends MemoryArchivist(maxEntries) { - abstract override def handleMessage: Receive = { + override def handleMessage: Receive = { handleTestingMessage orElse super.handleMessage } diff --git a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala index e83c7a67b481a..faf84245c390f 100644 --- a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala +++ b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala @@ -22,10 +22,7 @@ import akka.actor.{Props, ActorRef, ActorSystem} import akka.pattern.Patterns._ import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.runtime.StreamingMode -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.instance.AkkaActorGateway -import org.apache.flink.runtime.jobmanager.web.WebInfoServer -import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager} +import org.apache.flink.runtime.jobmanager.JobManager import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingJobManager, @@ -95,14 +92,12 @@ class ForkableFlinkMiniCluster( archiveCount) = JobManager.createJobManagerComponents(configuration) val testArchiveProps = Props( - new MemoryArchivist( - archiveCount) - with TestingMemoryArchivist) + new TestingMemoryArchivist(archiveCount)) val archiver = actorSystem.actorOf(testArchiveProps, JobManager.ARCHIVE_NAME) val jobManagerProps = Props( - new JobManager( + new TestingJobManager( configuration, executionContext, instanceManager, @@ -112,8 +107,7 @@ class ForkableFlinkMiniCluster( executionRetries, delayBetweenRetries, timeout, - streamingMode) - with TestingJobManager) + streamingMode)) val jobManager = actorSystem.actorOf(jobManagerProps, JobManager.JOB_MANAGER_NAME) From bac21bf5d77c8e15c608ecbf006d29e7af1dd68a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 23 Jul 2015 15:12:38 +0200 Subject: [PATCH 152/175] [FLINK-2398][api-breaking] Introduce StreamGraphGenerator This decouples the building of the StreamGraph from the API methods. Before the methods would build the StreamGraph as they go. Now the API methods build a hierachy of StreamTransformation nodes. From these a StreamGraph is generated upon execution. This also introduces some API breaking changes: - The result of methods that create sinks is now DataStreamSink instead of DataStream - Iterations cannot have feedback edges with differing parallelism - "Preserve partitioning" is not the default for feedback edges. The previous option for this is removed. - You can close an iteration several times, no need for a union. - Strict checking of whether partitioning and parallelism work together. I.e. if upstream and downstream parallelism don't match it is not legal to have Forward partitioning anymore. This was not very transparent: When you went from low parallelism to high dop some downstream operators would never get any input. When you went from high parallelism to low dop you would get skew in the downstream operators because all elements that would be forwarded to an operator that is not "there" go to another operator. This requires insertion of global() or rebalance() in some places. For example with most sources which have parallelism one. This also makes StreamExecutionEnvironment.execute() behave consistently across different execution environments (local, remote ...): The list of operators to be executed are cleared after execute is called. --- docs/apis/streaming_guide.md | 33 +- .../connectors/rabbitmq/RMQTopology.java | 6 +- .../api/datastream/ConnectedDataStream.java | 61 +- .../streaming/api/datastream/DataStream.java | 604 ++++++++---------- .../api/datastream/DataStreamSink.java | 66 +- .../api/datastream/DataStreamSource.java | 23 +- .../api/datastream/DiscretizedStream.java | 17 +- .../api/datastream/GroupedDataStream.java | 9 - .../api/datastream/IterativeDataStream.java | 149 ++--- .../api/datastream/KeyedDataStream.java | 43 +- .../SingleOutputStreamOperator.java | 109 ++-- .../api/datastream/SplitDataStream.java | 22 +- .../api/datastream/WindowedDataStream.java | 6 +- .../temporal/StreamCrossOperator.java | 43 +- .../temporal/StreamJoinOperator.java | 71 +- .../datastream/temporal/TemporalOperator.java | 4 +- .../environment/LocalStreamEnvironment.java | 5 +- .../environment/RemoteStreamEnvironment.java | 8 +- .../environment/StreamContextEnvironment.java | 7 +- .../StreamExecutionEnvironment.java | 178 ++++-- .../environment/StreamPlanEnvironment.java | 5 +- .../functions/source/FileSourceFunction.java | 8 + .../streaming/api/graph/JSONGenerator.java | 28 +- .../streaming/api/graph/StreamConfig.java | 23 - .../streaming/api/graph/StreamGraph.java | 383 +++++------ .../api/graph/StreamGraphGenerator.java | 530 +++++++++++++++ .../flink/streaming/api/graph/StreamLoop.java | 122 ---- .../flink/streaming/api/graph/StreamNode.java | 27 +- .../api/graph/StreamingJobGraphGenerator.java | 35 +- .../api/graph/WindowingOptimizer.java | 10 +- .../CoFeedbackTransformation.java | 122 ++++ .../FeedbackTransformation.java | 124 ++++ .../OneInputTransformation.java | 117 ++++ .../PartitionTransformation.java | 80 +++ .../transformations/SelectTransformation.java | 83 +++ .../transformations/SinkTransformation.java | 106 +++ .../transformations/SourceTransformation.java | 70 ++ .../transformations/SplitTransformation.java | 84 +++ .../transformations/StreamTransformation.java | 320 ++++++++++ .../TwoInputTransformation.java | 116 ++++ .../transformations/UnionTransformation.java | 81 +++ .../partitioner/BroadcastPartitioner.java | 17 +- .../partitioner/CustomPartitionerWrapper.java | 11 +- .../partitioner/ForwardPartitioner.java | 46 ++ .../partitioner/GlobalPartitioner.java | 20 +- ...sPartitioner.java => HashPartitioner.java} | 24 +- .../partitioner/RebalancePartitioner.java | 14 +- .../partitioner/ShufflePartitioner.java | 14 +- .../partitioner/StreamPartitioner.java | 25 +- .../api/ChainedRuntimeContextTest.java | 12 +- .../flink/streaming/api/CoStreamTest.java | 10 +- .../flink/streaming/api/DataStreamTest.java | 226 ++++--- .../flink/streaming/api/IterateTest.java | 459 +++++++------ .../streaming/api/OutputSplitterTest.java | 11 +- .../flink/streaming/api/PartitionerTest.java | 56 +- .../api/StreamExecutionEnvironmentTest.java | 34 +- .../flink/streaming/api/TypeFillTest.java | 3 +- .../streaming/api/WindowCrossJoinTest.java | 10 +- .../api/collector/DirectedOutputTest.java | 7 +- .../api/complex/ComplexIntegrationTest.java | 3 - .../api/graph/SlotAllocationTest.java | 5 +- .../api/graph/StreamGraphGeneratorTest.java | 179 ++++++ .../graph/StreamingJobGraphGeneratorTest.java | 9 +- .../api/operators/StreamProjectTest.java | 8 +- .../api/operators/co/CoStreamFlatMapTest.java | 19 +- .../api/operators/co/SelfConnectionTest.java | 43 +- .../operators/windowing/WindowingITCase.java | 62 +- .../outputformat/CsvOutputFormatITCase.java | 6 +- .../outputformat/TextOutputFormatITCase.java | 6 +- .../api/state/StatefulOperatorTest.java | 6 +- .../api/streamtask/StreamVertexTest.java | 15 +- .../streaming/graph/TranslationTest.java | 6 +- .../partitioner/ForwardPartitionerTest.java | 8 +- ...onerTest.java => HashPartitionerTest.java} | 24 +- ...est.java => RebalancePartitionerTest.java} | 4 +- .../streaming/timestamp/TimestampITCase.java | 21 +- .../streaming/util/EvenOddOutputSelector.java | 31 + .../flink/streaming/util/NoOpIntMap.java | 28 + .../apache/flink/streaming/util/NoOpSink.java | 26 + .../streaming/util/ReceiveCheckNoOpSink.java | 42 ++ .../streaming/util/TestStreamEnvironment.java | 9 +- .../streaming/api/scala/DataStream.scala | 24 +- .../api/scala/StreamCrossOperator.scala | 10 +- .../api/scala/StreamJoinOperator.scala | 26 +- .../streaming/api/scala/DataStreamTest.scala | 195 +++--- .../api/scala/OutputFormatTestPrograms.scala | 4 +- .../api/scala/StateTestPrograms.scala | 12 +- .../StreamingScalaAPICompletenessTest.scala | 11 + .../accumulators/AccumulatorLiveITCase.java | 19 +- .../classloading/jar/StreamingProgram.java | 2 +- 90 files changed, 4004 insertions(+), 1796 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java delete mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java rename flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/{FieldsPartitioner.java => HashPartitioner.java} (79%) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java rename flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/{FieldsPartitionerTest.java => HashPartitionerTest.java} (74%) rename flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/{DistributePartitionerTest.java => RebalancePartitionerTest.java} (95%) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index 35f6147794627..cd5acc31d179c 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -290,25 +290,25 @@ instance (with its fields set to new values). The object reuse mode will lead to because fewer objects are created, but the user has to manually take care of what they are doing with the object references. -### Partitioning +### Data Shipping Strategies -Partitioning controls how individual data points of a stream are distributed among the parallel instances of the transformation operators. This also controls the ordering of the records in the `DataStream`. There is partial ordering guarantee for the outputs with respect to the partitioning scheme (outputs produced from each partition are guaranteed to arrive in the order they were produced). +The data shipping strategy controls how individual elements of a stream are distributed among the parallel instances of a transformation operator. This also controls the ordering of the records in the `DataStream`. There is partial ordering guarantee for the outputs with respect to the shipping strategy (outputs produced from each partition are guaranteed to arrive in the order they were produced). -There are several partitioning types supported in Flink Streaming: +These are the supported shipping strategies: - * *Forward (default)*: Forward partitioning directs the output data to the next operator on the same machine (if possible) avoiding expensive network I/O. If there are more processing nodes than inputs or vice versa the load is distributed among the extra nodes in a round-robin fashion. This is the default partitioner. + * *Forward*: Forward shipping directs the output data to the next operator on the same machine, avoiding expensive network I/O. It can only be used when the parallelism of the input operations matches the parallelism of the downstream operation. This is the default shipping strategy if no strategy is specified and if the parallelism allows it. Usage: `dataStream.forward()` - * *Shuffle*: Shuffle partitioning randomly partitions the output data stream to the next operator using uniform distribution. Use this only when it is important that the partitioning is randomised. If you only care about an even load use *Rebalance*. + * *Shuffle*: Shuffle randomly partitions the output data stream to the next operator using uniform distribution. Use this only when it is important that the partitioning is randomised. If you only care about an even load use *Rebalance*. Usage: `dataStream.shuffle()` - * *Rebalance*: Rebalance partitioning directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution. + * *Rebalance*: Rebalance directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution. This is the default strategy if no strategy is defined and forward shipping is not possible because the parallelism of operations differs. Usage: `dataStream.rebalance()` - * *Field/Key Partitioning*: Field/Key partitioning partitions the output data stream based on the hash code of a selected key of the tuples. Data points with the same key are directed to the same operator instance. + * *Field/Key Partitioning*: Field/Key partitioning partitions the output data stream based on the hash code of a selected key of the tuples. Data points with the same key are directed to the same operator instance. Usage: `dataStream.partitionByHash(fields…)` -* *Field/Key Grouping*: Field/Key grouping takes partitioning one step further and seperates the elements to disjoint groups based on the hash code. These groups are processed separately by the next downstream operator. +* *Field/Key Grouping*: Field/Key grouping takes field/key partitioning one step further and seperates the elements into disjoint groups based on the hash code. These groups are processed separately by the next downstream operator. Usage: `dataStream.groupBy(fields…)` - * *Broadcast*: Broadcast partitioning sends the output data stream to all parallel instances of the next operator. + * *Broadcast*: Broadcast shipping sends the output data stream to all parallel instances of the next operator. Usage: `dataStream.broadcast()` - * *Global*: All data points are directed to the first instance of the operator. + * *Global*: All elements are directed to the first downstream instance of the operator. Usage: `dataStream.global()` Custom partitioning can also be used by giving a Partitioner function and a single field key to partition on, similarly to the batch API. @@ -330,17 +330,15 @@ val result = in -By default *Forward* partitioning is used. - -Partitioning does not remain in effect after a transformation, so it needs to be set again for subsequent operations. +The shipping strategy does not remain in effect after a transformation, so it needs to be set again for subsequent operations. ### Connecting to the outside world -The user is expected to connect to the outside world through the source and the sink interfaces. +The user is expected to connect to the outside world through the source and the sink interfaces. #### Sources -Sources can by created by using `StreamExecutionEnvironment.addSource(sourceFunction)`. +Sources can by created by using `StreamExecutionEnvironment.addSource(sourceFunction)`. Either use one of the source functions that come with Flink or write a custom source by implementing the `SourceFunction` interface. By default, sources run with parallelism of 1. To create parallel sources the user's source function needs to implement @@ -534,7 +532,8 @@ dataStream.fold("", new FoldFunction() { Union -

    Union of two or more data streams creating a new stream containing all the elements from all the streams.

    +

    Union of two or more data streams creating a new stream containing all the elements from all the streams. Node: If you union a data stream + with itself you will still only get each element once.

    {% highlight java %} dataStream.union(otherStream1, otherStream2, …) {% endhighlight %} @@ -1484,7 +1483,7 @@ Setting parallelism for operators works exactly the same way as in the batch Fli ### Buffer timeout -By default, data points are not transferred on the network one-by-one, which would cause unnecessary network traffic, but are buffered in the output buffers. The size of the output buffers can be set in the Flink config files. While this method is good for optimizing throughput, it can cause latency issues when the incoming stream is not fast enough. +By default, elements are not transferred on the network one-by-one, which would cause unnecessary network traffic, but are buffered in the output buffers. The size of the output buffers can be set in the Flink config files. While this method is good for optimizing throughput, it can cause latency issues when the incoming stream is not fast enough. To tackle this issue the user can call `env.setBufferTimeout(timeoutMillis)` on the execution environment (or on individual operators) to set a maximum wait time for the buffers to fill up. After this time, the buffers are flushed automatically even if they are not full. The default value for this timeout is 100 ms, which should be appropriate for most use-cases. Usage: diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java index 0f0623563c484..1f858629c1d74 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java @@ -17,7 +17,7 @@ package org.apache.flink.streaming.connectors.rabbitmq; -import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; @@ -29,11 +29,11 @@ public static void main(String[] args) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1); @SuppressWarnings("unused") - DataStream dataStream1 = env.addSource( + DataStreamSink dataStream1 = env.addSource( new RMQSource("localhost", "hello", new SimpleStringSchema())).print(); @SuppressWarnings("unused") - DataStream dataStream2 = env.fromElements("one", "two", "three", "four", "five", + DataStreamSink dataStream2 = env.fromElements("one", "two", "three", "four", "five", "q").addSink( new RMQSink("localhost", "hello", new StringToByteSerializer())); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java index 079f1363d1d6f..8609a3041ea78 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java @@ -27,12 +27,12 @@ import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.co.CoReduceFunction; import org.apache.flink.streaming.api.functions.co.CoWindowFunction; -import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap; import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce; import org.apache.flink.streaming.api.operators.co.CoStreamMap; import org.apache.flink.streaming.api.operators.co.CoStreamWindow; +import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; @@ -49,7 +49,6 @@ public class ConnectedDataStream { protected StreamExecutionEnvironment environment; - protected StreamGraph jobGraphBuilder; protected DataStream dataStream1; protected DataStream dataStream2; @@ -57,13 +56,13 @@ public class ConnectedDataStream { protected KeySelector keySelector1; protected KeySelector keySelector2; - protected ConnectedDataStream(DataStream input1, DataStream input2) { - this.jobGraphBuilder = input1.streamGraph; - this.environment = input1.environment; - this.dataStream1 = input1.copy(); - + protected ConnectedDataStream(StreamExecutionEnvironment env, DataStream input1, DataStream input2) { + this.environment = env; + if (input1 != null) { + this.dataStream1 = input1; + } if (input2 != null) { - this.dataStream2 = input2.copy(); + this.dataStream2 = input2; } if ((input1 instanceof GroupedDataStream) && (input2 instanceof GroupedDataStream)) { @@ -78,7 +77,6 @@ protected ConnectedDataStream(DataStream input1, DataStream input2) { } protected ConnectedDataStream(ConnectedDataStream coDataStream) { - this.jobGraphBuilder = coDataStream.jobGraphBuilder; this.environment = coDataStream.environment; this.dataStream1 = coDataStream.getFirst(); this.dataStream2 = coDataStream.getSecond(); @@ -105,7 +103,7 @@ public StreamExecutionEnvironment getExecutionEnvironment() { * @return The first DataStream. */ public DataStream getFirst() { - return dataStream1.copy(); + return dataStream1; } /** @@ -114,7 +112,7 @@ public DataStream getFirst() { * @return The second DataStream. */ public DataStream getSecond() { - return dataStream2.copy(); + return dataStream2; } /** @@ -150,7 +148,7 @@ public TypeInformation getType2() { * @return The grouped {@link ConnectedDataStream} */ public ConnectedDataStream groupBy(int keyPosition1, int keyPosition2) { - return new ConnectedDataStream(dataStream1.groupBy(keyPosition1), + return new ConnectedDataStream(this.environment, dataStream1.groupBy(keyPosition1), dataStream2.groupBy(keyPosition2)); } @@ -167,7 +165,7 @@ public ConnectedDataStream groupBy(int keyPosition1, int keyPosition2) * @return The grouped {@link ConnectedDataStream} */ public ConnectedDataStream groupBy(int[] keyPositions1, int[] keyPositions2) { - return new ConnectedDataStream(dataStream1.groupBy(keyPositions1), + return new ConnectedDataStream(environment, dataStream1.groupBy(keyPositions1), dataStream2.groupBy(keyPositions2)); } @@ -185,7 +183,7 @@ public ConnectedDataStream groupBy(int[] keyPositions1, int[] keyPosit * @return The grouped {@link ConnectedDataStream} */ public ConnectedDataStream groupBy(String field1, String field2) { - return new ConnectedDataStream(dataStream1.groupBy(field1), + return new ConnectedDataStream(environment, dataStream1.groupBy(field1), dataStream2.groupBy(field2)); } @@ -204,7 +202,7 @@ public ConnectedDataStream groupBy(String field1, String field2) { * @return The grouped {@link ConnectedDataStream} */ public ConnectedDataStream groupBy(String[] fields1, String[] fields2) { - return new ConnectedDataStream(dataStream1.groupBy(fields1), + return new ConnectedDataStream(environment, dataStream1.groupBy(fields1), dataStream2.groupBy(fields2)); } @@ -222,7 +220,7 @@ public ConnectedDataStream groupBy(String[] fields1, String[] fields2) */ public ConnectedDataStream groupBy(KeySelector keySelector1, KeySelector keySelector2) { - return new ConnectedDataStream(dataStream1.groupBy(keySelector1), + return new ConnectedDataStream(environment, dataStream1.groupBy(keySelector1), dataStream2.groupBy(keySelector2)); } @@ -239,7 +237,7 @@ public ConnectedDataStream groupBy(KeySelector keySelector1, * @return The partitioned {@link ConnectedDataStream} */ public ConnectedDataStream partitionByHash(int keyPosition1, int keyPosition2) { - return new ConnectedDataStream(dataStream1.partitionByHash(keyPosition1), + return new ConnectedDataStream(environment, dataStream1.partitionByHash(keyPosition1), dataStream2.partitionByHash(keyPosition2)); } @@ -254,7 +252,7 @@ public ConnectedDataStream partitionByHash(int keyPosition1, int keyPo * @return The partitioned {@link ConnectedDataStream} */ public ConnectedDataStream partitionByHash(int[] keyPositions1, int[] keyPositions2) { - return new ConnectedDataStream(dataStream1.partitionByHash(keyPositions1), + return new ConnectedDataStream(environment, dataStream1.partitionByHash(keyPositions1), dataStream2.partitionByHash(keyPositions2)); } @@ -272,7 +270,7 @@ public ConnectedDataStream partitionByHash(int[] keyPositions1, int[] * @return The partitioned {@link ConnectedDataStream} */ public ConnectedDataStream partitionByHash(String field1, String field2) { - return new ConnectedDataStream(dataStream1.partitionByHash(field1), + return new ConnectedDataStream(environment, dataStream1.partitionByHash(field1), dataStream2.partitionByHash(field2)); } @@ -290,7 +288,7 @@ public ConnectedDataStream partitionByHash(String field1, String field * @return The partitioned {@link ConnectedDataStream} */ public ConnectedDataStream partitionByHash(String[] fields1, String[] fields2) { - return new ConnectedDataStream(dataStream1.partitionByHash(fields1), + return new ConnectedDataStream(environment, dataStream1.partitionByHash(fields1), dataStream2.partitionByHash(fields2)); } @@ -306,7 +304,7 @@ public ConnectedDataStream partitionByHash(String[] fields1, String[] */ public ConnectedDataStream partitionByHash(KeySelector keySelector1, KeySelector keySelector2) { - return new ConnectedDataStream(dataStream1.partitionByHash(keySelector1), + return new ConnectedDataStream(environment, dataStream1.partitionByHash(keySelector1), dataStream2.partitionByHash(keySelector2)); } @@ -477,15 +475,22 @@ protected TwoInputStreamOperator getReduceOperator( public SingleOutputStreamOperator transform(String functionName, TypeInformation outTypeInfo, TwoInputStreamOperator operator) { - @SuppressWarnings({ "unchecked", "rawtypes" }) - SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator( - environment, outTypeInfo, operator); + // read the output type of the input Transforms to coax out errors about MissinTypeInfo + dataStream1.getType(); + dataStream2.getType(); - dataStream1.streamGraph.addCoOperator(returnStream.getId(), operator, getType1(), - getType2(), outTypeInfo, functionName); + TwoInputTransformation transform = new TwoInputTransformation( + dataStream1.getTransformation(), + dataStream2.getTransformation(), + functionName, + operator, + outTypeInfo, + environment.getParallelism()); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator(environment, transform); - dataStream1.connectGraph(dataStream1, returnStream.getId(), 1); - dataStream1.connectGraph(dataStream2, returnStream.getId(), 2); + getExecutionEnvironment().addOperator(transform); return returnStream; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 10ed5e3ab46a3..4f8ec1851d2be 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -17,13 +17,13 @@ package org.apache.flink.streaming.api.datastream; -import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import com.google.common.collect.Lists; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichFilterFunction; @@ -39,7 +39,6 @@ import org.apache.flink.api.java.io.TextOutputFormat; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.typeutils.MissingTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.core.fs.Path; @@ -51,12 +50,15 @@ import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.sink.SocketClientSink; -import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.streaming.api.operators.StreamFlatMap; import org.apache.flink.streaming.api.operators.StreamMap; import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.api.transformations.StreamTransformation; +import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.streaming.api.windowing.helper.Count; import org.apache.flink.streaming.api.windowing.helper.Delta; import org.apache.flink.streaming.api.windowing.helper.FullStream; @@ -66,8 +68,9 @@ import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; -import org.apache.flink.streaming.runtime.partitioner.FieldsPartitioner; +import org.apache.flink.streaming.runtime.partitioner.HashPartitioner; import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; @@ -79,84 +82,29 @@ /** * A DataStream represents a stream of elements of the same type. A DataStream * can be transformed into another DataStream by applying a transformation as - * for example + * for example: *
      - *
    • {@link DataStream#map},
    • - *
    • {@link DataStream#filter}, or
    • + *
    • {@link DataStream#map}, + *
    • {@link DataStream#filter}, or *
    * - * @param - * The type of the DataStream, i.e., the type of the elements of the - * DataStream. + * @param The type of the elements in this Stream */ -public class DataStream { +public class DataStream { - protected static Integer counter = 0; protected final StreamExecutionEnvironment environment; - protected final Integer id; - protected int parallelism; - protected List selectedNames; - protected StreamPartitioner partitioner; - @SuppressWarnings("rawtypes") - protected TypeInformation typeInfo; - protected List> unionedStreams; - - protected Integer iterationID = null; - protected Long iterationWaitTime = null; - protected final StreamGraph streamGraph; - private boolean typeUsed; + protected final StreamTransformation transformation; /** * Create a new {@link DataStream} in the given execution environment with * partitioning set to forward by default. - * - * @param environment - * StreamExecutionEnvironment - * @param typeInfo - * Type of the datastream - */ - public DataStream(StreamExecutionEnvironment environment, TypeInformation typeInfo) { - if (environment == null) { - throw new NullPointerException("context is null"); - } - - counter++; - this.id = counter; - this.environment = environment; - this.parallelism = environment.getParallelism(); - this.streamGraph = environment.getStreamGraph(); - this.selectedNames = new ArrayList(); - this.partitioner = new RebalancePartitioner(true); - this.typeInfo = typeInfo; - this.unionedStreams = new ArrayList>(); - this.unionedStreams.add(this); - } - - /** - * Create a new DataStream by creating a copy of another DataStream - * - * @param dataStream - * The DataStream that will be copied. + * + * @param environment The StreamExecutionEnvironment */ - public DataStream(DataStream dataStream) { - this.environment = dataStream.environment; - this.id = dataStream.id; - this.parallelism = dataStream.parallelism; - this.selectedNames = new ArrayList(dataStream.selectedNames); - this.partitioner = dataStream.partitioner.copy(); - this.streamGraph = dataStream.streamGraph; - this.typeInfo = dataStream.typeInfo; - this.iterationID = dataStream.iterationID; - this.iterationWaitTime = dataStream.iterationWaitTime; - this.unionedStreams = new ArrayList>(); - this.unionedStreams.add(this); - - int size = dataStream.unionedStreams.size(); - for (int i = 1; i < size; i++) { - this.unionedStreams.add(new DataStream(dataStream.unionedStreams.get(i))); - } - + public DataStream(StreamExecutionEnvironment environment, StreamTransformation transformation) { + this.environment = Preconditions.checkNotNull(environment, "Execution Environment must not be null."); + this.transformation = Preconditions.checkNotNull(transformation, "Stream Transformation must not be null."); } /** @@ -165,7 +113,7 @@ public DataStream(DataStream dataStream) { * @return ID of the DataStream */ public Integer getId() { - return id; + return transformation.getId(); } /** @@ -174,15 +122,7 @@ public Integer getId() { * @return The parallelism set for this operator. */ public int getParallelism() { - return this.parallelism; - } - - public StreamPartitioner getPartitioner() { - return this.partitioner; - } - - public List getSelectedNames(){ - return selectedNames; + return transformation.getParallelism(); } /** @@ -190,51 +130,26 @@ public List getSelectedNames(){ * * @return The type of the datastream. */ - @SuppressWarnings("unchecked") - public TypeInformation getType() { - if (typeInfo instanceof MissingTypeInfo) { - MissingTypeInfo typeInfo = (MissingTypeInfo) this.typeInfo; - throw new InvalidTypesException( - "The return type of function '" - + typeInfo.getFunctionName() - + "' could not be determined automatically, due to type erasure. " - + "You can give type information hints by using the returns(...) method on the result of " - + "the transformation call, or by letting your function implement the 'ResultTypeQueryable' " - + "interface.", typeInfo.getTypeException()); - } - typeUsed = true; - return this.typeInfo; + public TypeInformation getType() { + return transformation.getOutputType(); } /** - * Tries to fill in the type information. Type information can be filled in - * later when the program uses a type hint. This method checks whether the - * type information has ever been accessed before and does not allow - * modifications if the type was accessed already. This ensures consistency - * by making sure different parts of the operation do not assume different - * type information. - * - * @param typeInfo - * The type information to fill in. - * - * @throws IllegalStateException - * Thrown, if the type information has been accessed before. + * Invokes the {@link org.apache.flink.api.java.ClosureCleaner} + * on the given function if closure cleaning is enabled in the {@link ExecutionConfig}. + * + * @return The cleaned Function */ - protected void fillInType(TypeInformation typeInfo) { - if (typeUsed) { - throw new IllegalStateException( - "TypeInformation cannot be filled in for the type after it has been used. " - + "Please make sure that the type info hints are the first call after the transformation function, " - + "before any access to types or semantic properties, etc."); - } - streamGraph.setOutType(id, typeInfo); - this.typeInfo = typeInfo; - } - protected F clean(F f) { return getExecutionEnvironment().clean(f); } + /** + * Returns the {@link StreamExecutionEnvironment} that was used to create this + * {@link DataStream} + * + * @return The Execution Environment + */ public StreamExecutionEnvironment getExecutionEnvironment() { return environment; } @@ -252,16 +167,27 @@ public ExecutionConfig getExecutionConfig() { * The DataStreams to union output with. * @return The {@link DataStream}. */ - public DataStream union(DataStream... streams) { - DataStream returnStream = this.copy(); + public DataStream union(DataStream... streams) { + List> unionedTransforms = Lists.newArrayList(); + unionedTransforms.add(this.transformation); + + Collection> thisPredecessors = this.getTransformation().getTransitivePredecessors(); + + for (DataStream newStream : streams) { + if (!(newStream.getParallelism() == this.getParallelism())) { + throw new UnsupportedClassVersionError( + "DataStream can only be unioned with DataStreams of the same parallelism. " + + "This Stream: " + this.getTransformation() + + ", other stream: " + newStream.getTransformation()); + } + Collection> predecessors = newStream.getTransformation().getTransitivePredecessors(); - for (DataStream stream : streams) { - for (DataStream ds : stream.unionedStreams) { - validateUnion(ds.getId()); - returnStream.unionedStreams.add(ds.copy()); + if (predecessors.contains(this.transformation) || thisPredecessors.contains(newStream.getTransformation())) { + throw new UnsupportedOperationException("A DataStream cannot be unioned with itself"); } + unionedTransforms.add(newStream.getTransformation()); } - return returnStream; + return new DataStream(this.environment, new UnionTransformation(unionedTransforms)); } /** @@ -275,12 +201,8 @@ public DataStream union(DataStream... streams) { * for directing the tuples. * @return The {@link SplitDataStream} */ - public SplitDataStream split(OutputSelector outputSelector) { - for (DataStream ds : this.unionedStreams) { - streamGraph.addOutputSelector(ds.getId(), clean(outputSelector)); - } - - return new SplitDataStream(this); + public SplitDataStream split(OutputSelector outputSelector) { + return new SplitDataStream(this, clean(outputSelector)); } /** @@ -293,8 +215,8 @@ public SplitDataStream split(OutputSelector outputSelector) { * The DataStream with which this stream will be connected. * @return The {@link ConnectedDataStream}. */ - public ConnectedDataStream connect(DataStream dataStream) { - return new ConnectedDataStream(this, dataStream); + public ConnectedDataStream connect(DataStream dataStream) { + return new ConnectedDataStream(environment, this, dataStream); } /** @@ -306,8 +228,8 @@ public ConnectedDataStream connect(DataStream dataStream) { * The KeySelector to be used for extracting the key for partitioning * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream) */ - public KeyedDataStream keyBy(KeySelector key){ - return new KeyedDataStream(this, clean(key)); + public KeyedDataStream keyBy(KeySelector key){ + return new KeyedDataStream(this, clean(key)); } /** @@ -318,11 +240,11 @@ public KeyedDataStream keyBy(KeySelector key){ * will be grouped. * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream) */ - public KeyedDataStream keyBy(int... fields) { + public KeyedDataStream keyBy(int... fields) { if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) { - return keyBy(new KeySelectorUtil.ArrayKeySelector(fields)); + return keyBy(new KeySelectorUtil.ArrayKeySelector(fields)); } else { - return keyBy(new Keys.ExpressionKeys(fields, getType())); + return keyBy(new Keys.ExpressionKeys(fields, getType())); } } @@ -337,12 +259,12 @@ public KeyedDataStream keyBy(int... fields) { * partitioned. * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream) **/ - public KeyedDataStream keyBy(String... fields) { - return keyBy(new Keys.ExpressionKeys(fields, getType())); + public KeyedDataStream keyBy(String... fields) { + return keyBy(new Keys.ExpressionKeys(fields, getType())); } - private KeyedDataStream keyBy(Keys keys) { - return new KeyedDataStream(this, clean(KeySelectorUtil.getSelectorForKeys(keys, + private KeyedDataStream keyBy(Keys keys) { + return new KeyedDataStream(this, clean(KeySelectorUtil.getSelectorForKeys(keys, getType(), getExecutionConfig()))); } @@ -356,11 +278,11 @@ private KeyedDataStream keyBy(Keys keys) { * will be partitioned. * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream) */ - public GroupedDataStream groupBy(int... fields) { + public GroupedDataStream groupBy(int... fields) { if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) { - return groupBy(new KeySelectorUtil.ArrayKeySelector(fields)); + return groupBy(new KeySelectorUtil.ArrayKeySelector(fields)); } else { - return groupBy(new Keys.ExpressionKeys(fields, getType())); + return groupBy(new Keys.ExpressionKeys(fields, getType())); } } @@ -369,7 +291,10 @@ public GroupedDataStream groupBy(int... fields) { * is either the name of a public field or a getter method with parentheses * of the {@link DataStream}S underlying type. A dot can be used to drill * down into objects, as in {@code "field1.getInnerField2()" }. This method - * returns an {@link GroupedDataStream}.

    This operator also affects the + * returns an {@link GroupedDataStream}. + * + *

    + * This operator also affects the * partitioning of the stream, by forcing values with the same key to go to * the same processing instance. * @@ -378,14 +303,15 @@ public GroupedDataStream groupBy(int... fields) { * grouped. * @return The grouped {@link DataStream} **/ - public GroupedDataStream groupBy(String... fields) { - return groupBy(new Keys.ExpressionKeys(fields, getType())); + public GroupedDataStream groupBy(String... fields) { + return groupBy(new Keys.ExpressionKeys(fields, getType())); } /** * Groups the elements of a {@link DataStream} by the key extracted by the * {@link KeySelector} to be used with grouped operators like - * {@link GroupedDataStream#reduce(ReduceFunction)}. + * {@link GroupedDataStream#reduce(org.apache.flink.api.common.functions.ReduceFunction)}. + * *

    * This operator also affects the partitioning of the stream, by forcing * values with the same key to go to the same processing instance. @@ -395,12 +321,12 @@ public GroupedDataStream groupBy(String... fields) { * the values * @return The grouped {@link DataStream} */ - public GroupedDataStream groupBy(KeySelector keySelector) { - return new GroupedDataStream(this, clean(keySelector)); + public GroupedDataStream groupBy(KeySelector keySelector) { + return new GroupedDataStream(this, clean(keySelector)); } - private GroupedDataStream groupBy(Keys keys) { - return new GroupedDataStream(this, clean(KeySelectorUtil.getSelectorForKeys(keys, + private GroupedDataStream groupBy(Keys keys) { + return new GroupedDataStream(this, clean(KeySelectorUtil.getSelectorForKeys(keys, getType(), getExecutionConfig()))); } @@ -412,14 +338,13 @@ private GroupedDataStream groupBy(Keys keys) { * * @param fields The tuple fields that should be used for partitioning * @return The partitioned DataStream - * Specifies how elements will be distributed to parallel instances of downstream operations. * */ - public DataStream partitionByHash(int... fields) { + public DataStream partitionByHash(int... fields) { if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) { - return partitionByHash(new KeySelectorUtil.ArrayKeySelector(fields)); + return partitionByHash(new KeySelectorUtil.ArrayKeySelector(fields)); } else { - return partitionByHash(new Keys.ExpressionKeys(fields, getType())); + return partitionByHash(new Keys.ExpressionKeys(fields, getType())); } } @@ -431,11 +356,10 @@ public DataStream partitionByHash(int... fields) { * * @param fields The tuple fields that should be used for partitioning * @return The partitioned DataStream - * Specifies how elements will be distributed to parallel instances of downstream operations. * */ - public DataStream partitionByHash(String... fields) { - return partitionByHash(new Keys.ExpressionKeys(fields, getType())); + public DataStream partitionByHash(String... fields) { + return partitionByHash(new Keys.ExpressionKeys(fields, getType())); } /** @@ -444,19 +368,21 @@ public DataStream partitionByHash(String... fields) { * effects the how the outputs will be distributed between the parallel * instances of the next processing operator. * - * @param keySelector + * @param keySelector The function that extracts the key from an element in the Stream * @return The partitioned DataStream - * Specifies how elements will be distributed to parallel instances of downstream operations. */ - public DataStream partitionByHash(KeySelector keySelector) { - return setConnectionType(new FieldsPartitioner(clean(keySelector))); + public DataStream partitionByHash(KeySelector keySelector) { + return setConnectionType(new HashPartitioner(clean(keySelector))); } //private helper method for partitioning - private DataStream partitionByHash(Keys keys) { - return setConnectionType( - new FieldsPartitioner( - clean(KeySelectorUtil.getSelectorForKeys(keys, getType(), getExecutionConfig())))); + private DataStream partitionByHash(Keys keys) { + KeySelector keySelector = clean(KeySelectorUtil.getSelectorForKeys( + keys, + getType(), + getExecutionConfig())); + + return setConnectionType(new HashPartitioner(keySelector)); } /** @@ -469,8 +395,8 @@ private DataStream partitionByHash(Keys keys) { * @param field The field index on which the DataStream is to partitioned. * @return The partitioned DataStream. */ - public DataStream partitionCustom(Partitioner partitioner, int field) { - Keys.ExpressionKeys outExpressionKeys = new Keys.ExpressionKeys(new int[]{field}, getType()); + public DataStream partitionCustom(Partitioner partitioner, int field) { + Keys.ExpressionKeys outExpressionKeys = new Keys.ExpressionKeys(new int[]{field}, getType()); return partitionCustom(partitioner, outExpressionKeys); } @@ -484,8 +410,8 @@ public DataStream partitionCustom(Partitioner partitioner, int field * @param field The field index on which the DataStream is to partitioned. * @return The partitioned DataStream. */ - public DataStream partitionCustom(Partitioner partitioner, String field) { - Keys.ExpressionKeys outExpressionKeys = new Keys.ExpressionKeys(new String[]{field}, getType()); + public DataStream partitionCustom(Partitioner partitioner, String field) { + Keys.ExpressionKeys outExpressionKeys = new Keys.ExpressionKeys(new String[]{field}, getType()); return partitionCustom(partitioner, outExpressionKeys); } @@ -505,16 +431,16 @@ public DataStream partitionCustom(Partitioner partitioner, String fi * @return The partitioned DataStream. * @see KeySelector */ - public DataStream partitionCustom(Partitioner partitioner, KeySelector keySelector) { - return setConnectionType(new CustomPartitionerWrapper(clean(partitioner), clean(keySelector))); + public DataStream partitionCustom(Partitioner partitioner, KeySelector keySelector) { + return setConnectionType(new CustomPartitionerWrapper(clean(partitioner), clean(keySelector))); } // private helper method for custom partitioning - private DataStream partitionCustom(Partitioner partitioner, Keys keys) { - KeySelector keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig()); + private DataStream partitionCustom(Partitioner partitioner, Keys keys) { + KeySelector keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig()); return setConnectionType( - new CustomPartitionerWrapper( + new CustomPartitionerWrapper( clean(partitioner), clean(keySelector))); } @@ -529,8 +455,8 @@ private DataStream partitionCustom(Partitioner partitioner, Keys broadcast() { - return setConnectionType(new BroadcastPartitioner()); + public DataStream broadcast() { + return setConnectionType(new BroadcastPartitioner()); } /** @@ -543,8 +469,8 @@ public DataStream broadcast() { * * @return The DataStream with shuffle partitioning set. */ - public DataStream shuffle() { - return setConnectionType(new ShufflePartitioner()); + public DataStream shuffle() { + return setConnectionType(new ShufflePartitioner()); } /** @@ -558,8 +484,8 @@ public DataStream shuffle() { * * @return The DataStream with forward partitioning set. */ - public DataStream forward() { - return setConnectionType(new RebalancePartitioner(true)); + public DataStream forward() { + return setConnectionType(new ForwardPartitioner()); } /** @@ -573,8 +499,8 @@ public DataStream forward() { * * @return The DataStream with rebalance partitioning set. */ - public DataStream rebalance() { - return setConnectionType(new RebalancePartitioner(false)); + public DataStream rebalance() { + return setConnectionType(new RebalancePartitioner()); } /** @@ -585,8 +511,8 @@ public DataStream rebalance() { * * @return The DataStream with shuffle partitioning set. */ - public DataStream global() { - return setConnectionType(new GlobalPartitioner()); + public DataStream global() { + return setConnectionType(new GlobalPartitioner()); } /** @@ -607,7 +533,7 @@ public DataStream global() { *

    * The iteration edge will be partitioned the same way as the first input of * the iteration head unless it is changed in the - * {@link IterativeDataStream#closeWith(DataStream, boolean)} call. + * {@link IterativeDataStream#closeWith(DataStream)} call. *

    * By default a DataStream with iteration will never terminate, but the user * can use the maxWaitTime parameter to set a max waiting time for the @@ -616,8 +542,8 @@ public DataStream global() { * * @return The iterative data stream created. */ - public IterativeDataStream iterate() { - return new IterativeDataStream(this, 0); + public IterativeDataStream iterate() { + return new IterativeDataStream(this, 0); } /** @@ -638,7 +564,7 @@ public IterativeDataStream iterate() { *

    * The iteration edge will be partitioned the same way as the first input of * the iteration head unless it is changed in the - * {@link IterativeDataStream#closeWith(DataStream, boolean)} call. + * {@link IterativeDataStream#closeWith(DataStream)} call. *

    * By default a DataStream with iteration will never terminate, but the user * can use the maxWaitTime parameter to set a max waiting time for the @@ -651,8 +577,8 @@ public IterativeDataStream iterate() { * * @return The iterative data stream created. */ - public IterativeDataStream iterate(long maxWaitTimeMillis) { - return new IterativeDataStream(this, maxWaitTimeMillis); + public IterativeDataStream iterate(long maxWaitTimeMillis) { + return new IterativeDataStream(this, maxWaitTimeMillis); } /** @@ -669,12 +595,12 @@ public IterativeDataStream iterate(long maxWaitTimeMillis) { * output type * @return The transformed {@link DataStream}. */ - public SingleOutputStreamOperator map(MapFunction mapper) { + public SingleOutputStreamOperator map(MapFunction mapper) { TypeInformation outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(), Utils.getCallLocationName(), true); - return transform("Map", outType, new StreamMap(clean(mapper))); + return transform("Map", outType, new StreamMap(clean(mapper))); } /** @@ -693,12 +619,12 @@ public IterativeDataStream iterate(long maxWaitTimeMillis) { * output type * @return The transformed {@link DataStream}. */ - public SingleOutputStreamOperator flatMap(FlatMapFunction flatMapper) { + public SingleOutputStreamOperator flatMap(FlatMapFunction flatMapper) { TypeInformation outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper), getType(), Utils.getCallLocationName(), true); - return transform("Flat Map", outType, new StreamFlatMap(clean(flatMapper))); + return transform("Flat Map", outType, new StreamFlatMap(clean(flatMapper))); } @@ -716,15 +642,17 @@ public IterativeDataStream iterate(long maxWaitTimeMillis) { * DataStream. * @return The filtered DataStream. */ - public SingleOutputStreamOperator filter(FilterFunction filter) { - return transform("Filter", getType(), new StreamFilter(clean(filter))); + public SingleOutputStreamOperator filter(FilterFunction filter) { + return transform("Filter", getType(), new StreamFilter(clean(filter))); } /** * Initiates a Project transformation on a {@link Tuple} {@link DataStream}.
    - * Note: Only Tuple DataStreams can be projected.
    The - * transformation projects each Tuple of the DataSet onto a (sub)set of + * Note: Only Tuple DataStreams can be projected. + * + *

    + * The transformation projects each Tuple of the DataSet onto a (sub)set of * fields. * * @param fieldIndexes @@ -737,7 +665,7 @@ public IterativeDataStream iterate(long maxWaitTimeMillis) { * @see DataStream */ public SingleOutputStreamOperator project(int... fieldIndexes) { - return new StreamProjection(this.copy(), fieldIndexes).projectTupleX(); + return new StreamProjection(this, fieldIndexes).projectTupleX(); } @@ -762,15 +690,16 @@ public IterativeDataStream iterate(long maxWaitTimeMillis) { * cross transformation. * */ - public StreamCrossOperator cross(DataStream dataStreamToCross) { - return new StreamCrossOperator(this, dataStreamToCross); + public StreamCrossOperator cross(DataStream dataStreamToCross) { + return new StreamCrossOperator(this, dataStreamToCross); } /** * Initiates a temporal Join transformation.
    * A temporal Join transformation joins the elements of two - * {@link DataStream}s on key equality over a specified time window.
    - * + * {@link DataStream}s on key equality over a specified time window. + * + *

    * This method returns a {@link StreamJoinOperator} on which the * {@link StreamJoinOperator#onWindow(long, java.util.concurrent.TimeUnit)} * should be called to define the window, and then the @@ -779,7 +708,7 @@ public StreamCrossOperator cross(DataStream dataStreamToCro * the join keys. *

    * The user can also use the - * {@link StreamJoinOperator.JoinedStream#with(org.apache.flink.api.common.functions.JoinFunction)} + * {@link org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator.JoinPredicate.JoinedStream#with} * to apply a custom join function. * * @param dataStreamToJoin @@ -788,8 +717,8 @@ public StreamCrossOperator cross(DataStream dataStreamToCro * Join transformation. * */ - public StreamJoinOperator join(DataStream dataStreamToJoin) { - return new StreamJoinOperator(this, dataStreamToJoin); + public StreamJoinOperator join(DataStream dataStreamToJoin) { + return new StreamJoinOperator(this, dataStreamToJoin); } /** @@ -798,30 +727,46 @@ public StreamJoinOperator join(DataStream dataStreamToJoin) * {@link WindowedDataStream#mapWindow} or aggregations on preset * chunks(windows) of the data stream. To define windows a * {@link WindowingHelper} such as {@link Time}, {@link Count}, - * {@link Delta} and {@link FullStream} can be used.

    When applied - * to a grouped data stream, the windows (evictions) and slide sizes - * (triggers) will be computed on a per group basis.

    For more - * advanced control over the trigger and eviction policies please refer to - * {@link #window(TriggerPolicy, EvictionPolicy)}

    For example to create a - * sum every 5 seconds in a tumbling fashion:
    - * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).sum(field)}

    To - * create sliding windows use the - * {@link WindowedDataStream#every(WindowingHelper)}

    The same - * example with 3 second slides:
    + * {@link Delta} and {@link FullStream} can be used. + * + *

    + * When applied to a grouped data stream, the windows (evictions) and slide sizes + * (triggers) will be computed on a per group basis. + * + *

    + * For more advanced control over the trigger and eviction policies please refer to + * {@link #window(TriggerPolicy, EvictionPolicy)} + * + *

    + * For example, to create a sum every 5 seconds in a tumbling fashion: + * + *

    +	 * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).sum(field)}
    +	 * 
    + * + *

    + * To create sliding windows use the + * {@link WindowedDataStream#every(WindowingHelper)}, for example with 3 second slides:
    + * + *

     	 * 
    -	 * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).every(Time.of(3,
    -	 *       TimeUnit.SECONDS)).sum(field)}
    +	 * {@code
    +	 * ds.window(Time.of(5, TimeUnit.SECONDS)).every(Time.of(3, TimeUnit.SECONDS)).sum(field)
    +	 * }
    +	 *
    +	 * 
    * * @param policyHelper * Any {@link WindowingHelper} such as {@link Time}, * {@link Count}, {@link Delta} {@link FullStream} to define the * window size. + * * @return A {@link WindowedDataStream} providing further operations. */ @SuppressWarnings({ "rawtypes", "unchecked" }) - public WindowedDataStream window(WindowingHelper policyHelper) { + public WindowedDataStream window(WindowingHelper policyHelper) { policyHelper.setExecutionConfig(getExecutionConfig()); - return new WindowedDataStream(this, policyHelper); + return new WindowedDataStream(this, policyHelper); } /** @@ -829,8 +774,10 @@ public WindowedDataStream window(WindowingHelper policyHelper) { * and {@link EvictionPolicy}. Windowing can be used to apply transformation * like {@link WindowedDataStream#reduceWindow}, * {@link WindowedDataStream#mapWindow} or aggregations on preset - * chunks(windows) of the data stream.

    For most common use-cases - * please refer to {@link #window(WindowingHelper)} + * chunks(windows) of the data stream. + * + *

    + * For most common use-cases please refer to {@link #window(WindowingHelper)} * * @param trigger * The {@link TriggerPolicy} that will determine how often the @@ -840,8 +787,8 @@ public WindowedDataStream window(WindowingHelper policyHelper) { * elements in each time window. * @return A {@link WindowedDataStream} providing further operations. */ - public WindowedDataStream window(TriggerPolicy trigger, EvictionPolicy eviction) { - return new WindowedDataStream(this, trigger, eviction); + public WindowedDataStream window(TriggerPolicy trigger, EvictionPolicy eviction) { + return new WindowedDataStream(this, trigger, eviction); } /** @@ -851,38 +798,44 @@ public WindowedDataStream window(TriggerPolicy trigger, EvictionPolicy * @return A {@link WindowedDataStream} providing further operations. */ @SuppressWarnings("rawtypes") - public WindowedDataStream every(WindowingHelper policyHelper) { + public WindowedDataStream every(WindowingHelper policyHelper) { policyHelper.setExecutionConfig(getExecutionConfig()); return window(FullStream.window()).every(policyHelper); } /** - * Writes a DataStream to the standard output stream (stdout).
    + * Writes a DataStream to the standard output stream (stdout). + * + *

    * For each element of the DataStream the result of * {@link Object#toString()} is written. * * @return The closed DataStream. */ - public DataStreamSink print() { - PrintSinkFunction printFunction = new PrintSinkFunction(); + public DataStreamSink print() { + PrintSinkFunction printFunction = new PrintSinkFunction(); return addSink(printFunction); } /** - * Writes a DataStream to the standard output stream (stderr).
    + * Writes a DataStream to the standard output stream (stderr). + * + *

    * For each element of the DataStream the result of * {@link Object#toString()} is written. * * @return The closed DataStream. */ - public DataStreamSink printToErr() { - PrintSinkFunction printFunction = new PrintSinkFunction(true); + public DataStreamSink printToErr() { + PrintSinkFunction printFunction = new PrintSinkFunction(true); return addSink(printFunction); } /** - * Writes a DataStream to the file specified by path in text format. For - * every element of the DataStream the result of {@link Object#toString()} + * Writes a DataStream to the file specified by path in text format. + * + *

    + * For every element of the DataStream the result of {@link Object#toString()} * is written. * * @param path @@ -890,14 +843,16 @@ public DataStreamSink printToErr() { * * @return the closed DataStream. */ - public DataStreamSink writeAsText(String path) { - return write(new TextOutputFormat(new Path(path)), 0L); + public DataStreamSink writeAsText(String path) { + return write(new TextOutputFormat(new Path(path)), 0L); } /** * Writes a DataStream to the file specified by path in text format. The - * writing is performed periodically, in every millis milliseconds. For - * every element of the DataStream the result of {@link Object#toString()} + * writing is performed periodically, in every millis milliseconds. + * + *

    + * For every element of the DataStream the result of {@link Object#toString()} * is written. * * @param path @@ -907,14 +862,16 @@ public DataStreamSink writeAsText(String path) { * * @return the closed DataStream */ - public DataStreamSink writeAsText(String path, long millis) { - TextOutputFormat tof = new TextOutputFormat(new Path(path)); + public DataStreamSink writeAsText(String path, long millis) { + TextOutputFormat tof = new TextOutputFormat(new Path(path)); return write(tof, millis); } /** - * Writes a DataStream to the file specified by path in text format. For - * every element of the DataStream the result of {@link Object#toString()} + * Writes a DataStream to the file specified by path in text format. + * + *

    + * For every element of the DataStream the result of {@link Object#toString()} * is written. * * @param path @@ -925,15 +882,17 @@ public DataStreamSink writeAsText(String path, long millis) { * * @return the closed DataStream. */ - public DataStreamSink writeAsText(String path, WriteMode writeMode) { - TextOutputFormat tof = new TextOutputFormat(new Path(path)); + public DataStreamSink writeAsText(String path, WriteMode writeMode) { + TextOutputFormat tof = new TextOutputFormat(new Path(path)); tof.setWriteMode(writeMode); return write(tof, 0L); } /** - * Writes a DataStream to the file specified by path in text format. For - * every element of the DataStream the result of {@link Object#toString()} + * Writes a DataStream to the file specified by path in text format. + * + *

    + * For every element of the DataStream the result of {@link Object#toString()} * is written. * * @param path @@ -946,15 +905,17 @@ public DataStreamSink writeAsText(String path, WriteMode writeMode) { * * @return the closed DataStream. */ - public DataStreamSink writeAsText(String path, WriteMode writeMode, long millis) { - TextOutputFormat tof = new TextOutputFormat(new Path(path)); + public DataStreamSink writeAsText(String path, WriteMode writeMode, long millis) { + TextOutputFormat tof = new TextOutputFormat(new Path(path)); tof.setWriteMode(writeMode); return write(tof, millis); } /** - * Writes a DataStream to the file specified by path in csv format. For - * every element of the DataStream the result of {@link Object#toString()} + * Writes a DataStream to the file specified by path in csv format. + * + *

    + * For every field of an element of the DataStream the result of {@link Object#toString()} * is written. This method can only be used on data streams of tuples. * * @param path @@ -963,20 +924,22 @@ public DataStreamSink writeAsText(String path, WriteMode writeMode, long mi * @return the closed DataStream */ @SuppressWarnings("unchecked") - public DataStreamSink writeAsCsv(String path) { + public DataStreamSink writeAsCsv(String path) { Preconditions.checkArgument(getType().isTupleType(), "The writeAsCsv() method can only be used on data sets of tuples."); CsvOutputFormat of = new CsvOutputFormat(new Path(path), CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER); - return write((OutputFormat) of, 0L); + return write((OutputFormat) of, 0L); } /** * Writes a DataStream to the file specified by path in csv format. The - * writing is performed periodically, in every millis milliseconds. For - * every element of the DataStream the result of {@link Object#toString()} + * writing is performed periodically, in every millis milliseconds. + * + *

    + * For every field of an element of the DataStream the result of {@link Object#toString()} * is written. This method can only be used on data streams of tuples. - * + * * @param path * the path pointing to the location the text file is written to * @param millis @@ -985,17 +948,19 @@ public DataStreamSink writeAsCsv(String path) { * @return the closed DataStream */ @SuppressWarnings("unchecked") - public DataStreamSink writeAsCsv(String path, long millis) { + public DataStreamSink writeAsCsv(String path, long millis) { Preconditions.checkArgument(getType().isTupleType(), "The writeAsCsv() method can only be used on data sets of tuples."); CsvOutputFormat of = new CsvOutputFormat(new Path(path), CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER); - return write((OutputFormat) of, millis); + return write((OutputFormat) of, millis); } /** - * Writes a DataStream to the file specified by path in csv format. For - * every element of the DataStream the result of {@link Object#toString()} + * Writes a DataStream to the file specified by path in csv format. + * + *

    + * For every field of an element of the DataStream the result of {@link Object#toString()} * is written. This method can only be used on data streams of tuples. * * @param path @@ -1007,7 +972,7 @@ public DataStreamSink writeAsCsv(String path, long millis * @return the closed DataStream */ @SuppressWarnings("unchecked") - public DataStreamSink writeAsCsv(String path, WriteMode writeMode) { + public DataStreamSink writeAsCsv(String path, WriteMode writeMode) { Preconditions.checkArgument(getType().isTupleType(), "The writeAsCsv() method can only be used on data sets of tuples."); CsvOutputFormat of = new CsvOutputFormat(new Path(path), @@ -1015,13 +980,15 @@ public DataStreamSink writeAsCsv(String path, WriteMode w if (writeMode != null) { of.setWriteMode(writeMode); } - return write((OutputFormat) of, 0L); + return write((OutputFormat) of, 0L); } /** * Writes a DataStream to the file specified by path in csv format. The - * writing is performed periodically, in every millis milliseconds. For - * every element of the DataStream the result of {@link Object#toString()} + * writing is performed periodically, in every millis milliseconds. + * + *

    + * For every field of an element of the DataStream the result of {@link Object#toString()} * is written. This method can only be used on data streams of tuples. * * @param path @@ -1035,7 +1002,7 @@ public DataStreamSink writeAsCsv(String path, WriteMode w * @return the closed DataStream */ @SuppressWarnings("unchecked") - public DataStreamSink writeAsCsv(String path, WriteMode writeMode, + public DataStreamSink writeAsCsv(String path, WriteMode writeMode, long millis) { Preconditions.checkArgument(getType().isTupleType(), "The writeAsCsv() method can only be used on data sets of tuples."); @@ -1044,7 +1011,7 @@ public DataStreamSink writeAsCsv(String path, WriteMode w if (writeMode != null) { of.setWriteMode(writeMode); } - return write((OutputFormat) of, millis); + return write((OutputFormat) of, millis); } /** @@ -1059,8 +1026,8 @@ public DataStreamSink writeAsCsv(String path, WriteMode w * schema for serialization * @return the closed DataStream */ - public DataStreamSink writeToSocket(String hostName, int port, SerializationSchema schema) { - DataStreamSink returnStream = addSink(new SocketClientSink(hostName, port, schema)); + public DataStreamSink writeToSocket(String hostName, int port, SerializationSchema schema) { + DataStreamSink returnStream = addSink(new SocketClientSink(hostName, port, schema)); returnStream.setParallelism(1); // It would not work if multiple instances would connect to the same port return returnStream; } @@ -1072,8 +1039,8 @@ public DataStreamSink writeToSocket(String hostName, int port, Serializatio * @param millis the write frequency * @return The closed DataStream */ - public DataStreamSink write(OutputFormat format, long millis) { - return addSink(new FileSinkFunctionByMillis(format, millis)); + public DataStreamSink write(OutputFormat format, long millis) { + return addSink(new FileSinkFunctionByMillis(format, millis)); } /** @@ -1090,66 +1057,35 @@ public DataStreamSink write(OutputFormat format, long millis) { * type of the return stream * @return the data stream constructed */ - public SingleOutputStreamOperator transform(String operatorName, - TypeInformation outTypeInfo, OneInputStreamOperator operator) { - DataStream inputStream = this.copy(); - @SuppressWarnings({ "unchecked", "rawtypes" }) - SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator(environment, - outTypeInfo, operator); + public SingleOutputStreamOperator transform(String operatorName, TypeInformation outTypeInfo, OneInputStreamOperator operator) { - streamGraph.addOperator(returnStream.getId(), operator, getType(), outTypeInfo, - operatorName); + // read the output type of the input Transform to coax out errors about MissingTypeInfo + transformation.getOutputType(); - connectGraph(inputStream, returnStream.getId(), 0); - - if (iterationID != null) { - //This data stream is an input to some iteration - addIterationSource(returnStream, null); - } + OneInputTransformation resultTransform = new OneInputTransformation( + this.transformation, + operatorName, + operator, + outTypeInfo, + environment.getParallelism()); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator(environment, resultTransform); + + getExecutionEnvironment().addOperator(resultTransform); return returnStream; } - - protected void addIterationSource(DataStream dataStream, TypeInformation feedbackType) { - streamGraph.addIterationHead(dataStream.getId(), iterationID, iterationWaitTime, feedbackType); - } /** * Internal function for setting the partitioner for the DataStream - * + * * @param partitioner * Partitioner to set. * @return The modified DataStream. */ - protected DataStream setConnectionType(StreamPartitioner partitioner) { - DataStream returnStream = this.copy(); - - for (DataStream stream : returnStream.unionedStreams) { - stream.partitioner = partitioner; - } - - return returnStream; - } - - /** - * Internal function for assembling the underlying - * {@link org.apache.flink.runtime.jobgraph.JobGraph} of the job. Connects - * the outputs of the given input stream to the specified output stream - * given by the outputID. - * - * @param inputStream - * input data stream - * @param outputID - * ID of the output - * @param typeNumber - * Number of the type (used at co-functions) - */ - protected void connectGraph(DataStream inputStream, Integer outputID, int typeNumber) { - for (DataStream stream : inputStream.unionedStreams) { - streamGraph.addEdge(stream.getId(), outputID, stream.partitioner, typeNumber, - inputStream.selectedNames); - } - + protected DataStream setConnectionType(StreamPartitioner partitioner) { + return new DataStream(this.getExecutionEnvironment(), new PartitionTransformation(this.getTransformation(), partitioner)); } /** @@ -1161,24 +1097,26 @@ protected void connectGraph(DataStream inputStream, Integer outputID, int * The object containing the sink's invoke function. * @return The closed DataStream. */ - public DataStreamSink addSink(SinkFunction sinkFunction) { - return new DataStreamSink((DataStream) transform("StreamSink", null, new StreamSink(clean(sinkFunction)))); - } + public DataStreamSink addSink(SinkFunction sinkFunction) { - private void validateUnion(Integer id) { - for (DataStream ds : this.unionedStreams) { - if (ds.getId().equals(id)) { - throw new RuntimeException("A DataStream cannot be merged with itself"); - } - } + // read the output type of the input Transform to coax out errors about MissingTypeInfo + transformation.getOutputType(); + + StreamSink sinkOperator = new StreamSink(clean(sinkFunction)); + + DataStreamSink sink = new DataStreamSink(this, sinkOperator); + + getExecutionEnvironment().addOperator(sink.getTransformation()); + return sink; } /** - * Creates a copy of the {@link DataStream} - * - * @return The copy + * Returns the {@link StreamTransformation} that represents the operation that logically creates + * this {@link DataStream}. + * + * @return The Transformation */ - public DataStream copy() { - return new DataStream(this); + public StreamTransformation getTransformation() { + return transformation; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java index 60dc367756953..fdf398ca7250f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java @@ -17,30 +17,66 @@ package org.apache.flink.streaming.api.datastream; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.api.transformations.SinkTransformation; /** - * Represents the end of a DataStream. + * A Stream Sink. This is used for emitting elements from a streaming topology. * - * @param - * The type of the DataStream closed by the sink. + * @param The type of the elements in the Stream */ -public class DataStreamSink extends SingleOutputStreamOperator> { +public class DataStreamSink { - protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType, - TypeInformation outTypeInfo, OneInputStreamOperator operator) { - super(environment, outTypeInfo, operator); + SinkTransformation transformation; + + @SuppressWarnings("unchecked") + protected DataStreamSink(DataStream inputStream, StreamSink operator) { + this.transformation = new SinkTransformation(inputStream.getTransformation(), "Unnamed", operator, inputStream.getExecutionEnvironment().getParallelism()); + } + + /** + * Returns the transformation that contains the actual sink operator of this sink. + */ + public SinkTransformation getTransformation() { + return transformation; } - protected DataStreamSink(DataStream dataStream) { - super(dataStream); + /** + * Sets the name of this sink. This name is + * used by the visualization and logging during runtime. + * + * @return The named sink. + */ + public DataStreamSink name(String name) { + transformation.setName(name); + return this; } - @Override - public DataStreamSink copy() { - throw new RuntimeException("Data stream sinks cannot be copied"); + /** + * Sets the parallelism for this sink. The degree must be higher than zero. + * + * @param parallelism The parallelism for this sink. + * @return The sink with set parallelism. + */ + public DataStreamSink setParallelism(int parallelism) { + transformation.setParallelism(parallelism); + return this; } + /** + * Turns off chaining for this operator so thread co-location will not be + * used as an optimization. + * + *

    + * Chaining can be turned off for the whole + * job by {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#disableOperatorChaining()} + * however it is not advised for performance considerations. + * + * @return The sink with chaining disabled + */ + public DataStreamSink disableChaining() { + this.transformation.setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER); + return this; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java index 0dd77016c9491..d2e04a78a9a97 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java @@ -19,25 +19,22 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.transformations.SourceTransformation; /** * The DataStreamSource represents the starting point of a DataStream. * - * @param - * Type of the DataStream created. + * @param Type of the elements in the DataStream created from the this source. */ -public class DataStreamSource extends SingleOutputStreamOperator> { +public class DataStreamSource extends SingleOutputStreamOperator> { boolean isParallel; - public DataStreamSource(StreamExecutionEnvironment environment, String operatorType, - TypeInformation outTypeInfo, StreamOperator operator, + public DataStreamSource(StreamExecutionEnvironment environment, + TypeInformation outTypeInfo, StreamSource operator, boolean isParallel, String sourceName) { - super(environment, outTypeInfo, operator); - - environment.getStreamGraph().addSource(getId(), operator, null, outTypeInfo, - sourceName); + super(environment, new SourceTransformation(sourceName, operator, outTypeInfo, environment.getParallelism())); this.isParallel = isParallel; if (!isParallel) { @@ -46,11 +43,11 @@ public DataStreamSource(StreamExecutionEnvironment environment, String operatorT } @Override - public DataStreamSource setParallelism(int parallelism) { + public DataStreamSource setParallelism(int parallelism) { if (parallelism > 1 && !isParallel) { - throw new IllegalArgumentException("Source: " + this.id + " is not a parallel source"); + throw new IllegalArgumentException("Source: " + transformation.getId() + " is not a parallel source"); } else { - return (DataStreamSource) super.setParallelism(parallelism); + return (DataStreamSource) super.setParallelism(parallelism); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java index e35592e60f32d..5893295f9b451 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.operators.windowing.EmptyWindowFilter; import org.apache.flink.streaming.api.operators.windowing.ParallelGroupedMerge; import org.apache.flink.streaming.api.operators.windowing.ParallelMerge; +import org.apache.flink.streaming.api.operators.windowing.ParallelMergeOperator; import org.apache.flink.streaming.api.operators.windowing.WindowFlattener; import org.apache.flink.streaming.api.operators.windowing.WindowFolder; import org.apache.flink.streaming.api.operators.windowing.WindowMapper; @@ -43,7 +44,7 @@ import org.apache.flink.streaming.api.operators.windowing.WindowPartExtractor; import org.apache.flink.streaming.api.operators.windowing.WindowPartitioner; import org.apache.flink.streaming.api.operators.windowing.WindowReducer; -import org.apache.flink.streaming.api.operators.windowing.ParallelMergeOperator; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.StreamWindowTypeInfo; import org.apache.flink.streaming.api.windowing.WindowUtils.WindowKey; @@ -96,7 +97,7 @@ public DiscretizedStream name(String name){ } public DataStream flatten() { - return discretizedStream.transform("Window Flatten", getType(), new WindowFlattener()); + return discretizedStream.transform("Window Flatten", getType(), new WindowFlattener()).setParallelism(discretizedStream.getParallelism()); } public DataStream> getDiscretizedStream() { @@ -112,7 +113,7 @@ public DiscretizedStream reduceWindow(ReduceFunction reduceFunction) { // If we merged a non-grouped reduce transformation we need to reduce // again - if (!isGrouped() && out.discretizedStream.operator instanceof WindowMerger) { + if (!isGrouped() && ((OneInputTransformation)out.discretizedStream.getTransformation()).getOperator() instanceof WindowMerger) { return out.transform(WindowTransformation.REDUCEWINDOW, "Window Reduce", out.getType(), new WindowReducer(discretizedStream.clean(reduceFunction))); } else { @@ -171,7 +172,7 @@ public DiscretizedStream mapWindow(WindowMapFunction windowMapFun TypeInformation returnType) { DiscretizedStream out = partition(transformation).transform( WindowTransformation.MAPWINDOW, "Window Map", returnType, - new WindowMapper(discretizedStream.clean(windowMapFunction))).merge(); + new WindowMapper(discretizedStream.clean(windowMapFunction))).setParallelism(discretizedStream.getParallelism()).merge(); return out; } @@ -183,6 +184,7 @@ public DiscretizedStream foldWindow(R initialValue, FoldFunction DiscretizedStream out = partition(transformation).transform( WindowTransformation.FOLDWINDOW, "Fold Window", outType, new WindowFolder(discretizedStream.clean(foldFunction), initialValue)) + .setParallelism(discretizedStream.getParallelism()) .merge(); return out; } @@ -192,7 +194,7 @@ private DiscretizedStream transform(WindowTransformation transformation, OneInputStreamOperator, StreamWindow> operator) { return wrap(discretizedStream.transform(operatorName, new StreamWindowTypeInfo(retType), - operator), transformation); + operator).setParallelism(discretizedStream.getParallelism()), transformation); } private DiscretizedStream filterEmpty(DiscretizedStream input) { @@ -246,7 +248,7 @@ private DiscretizedStream merge() { if (isPartitioned) { return wrap( discretizedStream.groupBy(new WindowKey()).transform("Window Merger", - type, new WindowMerger()), false); + type, new WindowMerger()).setParallelism(discretizedStream.getParallelism()), false); } else { return this; } @@ -327,8 +329,7 @@ private static TypeInformation getWindowMapReturnTypes( } protected DiscretizedStream copy() { - return new DiscretizedStream(discretizedStream.copy(), groupByKey, transformation, - isPartitioned); + return new DiscretizedStream(discretizedStream, groupByKey, transformation, isPartitioned); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java index da005af8352e5..72ef945d0ea15 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java @@ -52,9 +52,6 @@ public GroupedDataStream(DataStream dataStream, KeySelector keySele super(dataStream, keySelector); } - protected GroupedDataStream(GroupedDataStream dataStream) { - super(dataStream); - } /** * Applies a reduce transformation on the grouped data stream grouped on by @@ -331,10 +328,4 @@ protected GroupedDataStream(GroupedDataStream dataStream) { getType(), operator); return returnStream; } - - @Override - public GroupedDataStream copy() { - return new GroupedDataStream(this); - } - } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java index 4de368c946740..6b120135b2db0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java @@ -17,40 +17,41 @@ package org.apache.flink.streaming.api.datastream; -import java.util.List; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.typeutils.TypeInfoParser; -import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation; +import org.apache.flink.streaming.api.transformations.FeedbackTransformation; +import org.apache.flink.streaming.api.transformations.StreamTransformation; + +import java.util.Collection; /** - * The iterative data stream represents the start of an iteration in a - * {@link DataStream}. + * The iterative data stream represents the start of an iteration in a {@link DataStream}. * - * @param - * Type of the DataStream + * @param Type of the elements in this Stream */ -public class IterativeDataStream extends - SingleOutputStreamOperator> { - - protected boolean closed = false; +public class IterativeDataStream extends SingleOutputStreamOperator> { - static Integer iterationCount = 0; + // We store these so that we can create a co-iteration if we need to + private DataStream originalInput; + private long maxWaitTime; - protected IterativeDataStream(DataStream dataStream, long maxWaitTime) { - super(dataStream); + protected IterativeDataStream(DataStream dataStream, long maxWaitTime) { + super(dataStream.getExecutionEnvironment(), + new FeedbackTransformation(dataStream.getTransformation(), maxWaitTime)); + this.originalInput = dataStream; + this.maxWaitTime = maxWaitTime; setBufferTimeout(dataStream.environment.getBufferTimeout()); - iterationID = iterationCount; - iterationCount++; - iterationWaitTime = maxWaitTime; } /** * Closes the iteration. This method defines the end of the iterative - * program part that will be fed back to the start of the iteration.
    - *
    A common usage pattern for streaming iterations is to use output + * program part that will be fed back to the start of the iteration. + * + *

    + * A common usage pattern for streaming iterations is to use output * splitting to send a part of the closing data stream to the head. Refer to * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)} * for more information. @@ -58,50 +59,30 @@ protected IterativeDataStream(DataStream dataStream, long maxWaitTime) { * @param feedbackStream * {@link DataStream} that will be used as input to the iteration * head. - * @param keepPartitioning - * If true the feedback partitioning will be kept as it is (not - * changed to match the input of the iteration head) + * * @return The feedback stream. * */ @SuppressWarnings({ "unchecked", "rawtypes" }) - public DataStream closeWith(DataStream iterationTail, boolean keepPartitioning) { - - if (closed) { - throw new IllegalStateException( - "An iterative data stream can only be closed once. Use union to close with multiple stream."); + public DataStream closeWith(DataStream feedbackStream) { + + Collection> predecessors = feedbackStream.getTransformation().getTransitivePredecessors(); + + if (!predecessors.contains(this.transformation)) { + throw new UnsupportedOperationException( + "Cannot close an iteration with a feedback DataStream that does not originate from said iteration."); } - closed = true; - - streamGraph.addIterationTail((List) iterationTail.unionedStreams, iterationID, - keepPartitioning); - return iterationTail; - } - - /** - * Closes the iteration. This method defines the end of the iterative - * program part that will be fed back to the start of the iteration.
    - *
    A common usage pattern for streaming iterations is to use output - * splitting to send a part of the closing data stream to the head. Refer to - * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)} - * for more information. - * - * - * @param feedbackStream - * {@link DataStream} that will be used as input to the - * iteration head. - * @return The feedback stream. - * - */ - public DataStream closeWith(DataStream iterationTail) { - return closeWith(iterationTail,false); + ((FeedbackTransformation) getTransformation()).addFeedbackEdge(feedbackStream.getTransformation()); + + return feedbackStream; } /** * Changes the feedback type of the iteration and allows the user to apply * co-transformations on the input and feedback stream, as in a * {@link ConnectedDataStream}. + * *

    * For type safety the user needs to define the feedback type * @@ -109,7 +90,7 @@ public DataStream closeWith(DataStream iterationTail) { * String describing the type information of the feedback stream. * @return A {@link ConnectedIterativeDataStream}. */ - public ConnectedIterativeDataStream withFeedbackType(String feedbackTypeString) { + public ConnectedIterativeDataStream withFeedbackType(String feedbackTypeString) { return withFeedbackType(TypeInfoParser. parse(feedbackTypeString)); } @@ -117,6 +98,7 @@ public ConnectedIterativeDataStream withFeedbackType(String feedbackT * Changes the feedback type of the iteration and allows the user to apply * co-transformations on the input and feedback stream, as in a * {@link ConnectedDataStream}. + * *

    * For type safety the user needs to define the feedback type * @@ -124,7 +106,7 @@ public ConnectedIterativeDataStream withFeedbackType(String feedbackT * Class of the elements in the feedback stream. * @return A {@link ConnectedIterativeDataStream}. */ - public ConnectedIterativeDataStream withFeedbackType(Class feedbackTypeClass) { + public ConnectedIterativeDataStream withFeedbackType(Class feedbackTypeClass) { return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass)); } @@ -132,6 +114,7 @@ public ConnectedIterativeDataStream withFeedbackType(Class feedbac * Changes the feedback type of the iteration and allows the user to apply * co-transformations on the input and feedback stream, as in a * {@link ConnectedDataStream}. + * *

    * For type safety the user needs to define the feedback type * @@ -139,9 +122,8 @@ public ConnectedIterativeDataStream withFeedbackType(Class feedbac * The type information of the feedback stream. * @return A {@link ConnectedIterativeDataStream}. */ - public ConnectedIterativeDataStream withFeedbackType(TypeInformation feedbackType) { - return new ConnectedIterativeDataStream(new IterativeDataStream(this, - iterationWaitTime), feedbackType); + public ConnectedIterativeDataStream withFeedbackType(TypeInformation feedbackType) { + return new ConnectedIterativeDataStream(originalInput, feedbackType, maxWaitTime); } /** @@ -149,6 +131,7 @@ public ConnectedIterativeDataStream withFeedbackType(TypeInformation< * iterative part of a streaming program, where the original input of the * iteration and the feedback of the iteration are connected as in a * {@link ConnectedDataStream}. + * *

    * The user can distinguish between the two inputs using co-transformation, * thus eliminating the need for mapping the inputs and outputs to a common @@ -161,38 +144,18 @@ public ConnectedIterativeDataStream withFeedbackType(TypeInformation< */ public static class ConnectedIterativeDataStream extends ConnectedDataStream{ - private IterativeDataStream input; - private TypeInformation feedbackType; + private CoFeedbackTransformation coFeedbackTransformation; - public ConnectedIterativeDataStream(IterativeDataStream input, TypeInformation feedbackType) { - super(input, null); - this.input = input; - this.feedbackType = feedbackType; + public ConnectedIterativeDataStream(DataStream input, TypeInformation feedbackType, long waitTime) { + super(input.getExecutionEnvironment(), + input, + new DataStream(input.getExecutionEnvironment(), + new CoFeedbackTransformation(input.getParallelism(), + feedbackType, + waitTime))); + this.coFeedbackTransformation = (CoFeedbackTransformation) getSecond().getTransformation(); } - - @Override - public TypeInformation getType2() { - return feedbackType; - } - - @Override - public SingleOutputStreamOperator transform(String functionName, - TypeInformation outTypeInfo, TwoInputStreamOperator operator) { - @SuppressWarnings({ "unchecked", "rawtypes" }) - SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator( - input.environment, outTypeInfo, operator); - - input.streamGraph.addCoOperator(returnStream.getId(), operator, input.getType(), - feedbackType, outTypeInfo, functionName); - - input.connectGraph(input, returnStream.getId(), 1); - - input.addIterationSource(returnStream, feedbackType); - - return returnStream; - } - /** * Closes the iteration. This method defines the end of the iterative * program part that will be fed back to the start of the iteration as @@ -206,14 +169,16 @@ public TypeInformation getType2() { */ @SuppressWarnings({ "rawtypes", "unchecked" }) public DataStream closeWith(DataStream feedbackStream) { - if (input.closed) { - throw new IllegalStateException( - "An iterative data stream can only be closed once. Use union to close with multiple stream."); + + Collection> predecessors = feedbackStream.getTransformation().getTransitivePredecessors(); + + if (!predecessors.contains(this.coFeedbackTransformation)) { + throw new UnsupportedOperationException( + "Cannot close an iteration with a feedback DataStream that does not originate from said iteration."); } - input.closed = true; - - input.streamGraph.addIterationTail((List) feedbackStream.unionedStreams, - input.iterationID, true); + + coFeedbackTransformation.addFeedbackEdge(feedbackStream.getTransformation()); + return feedbackStream; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java index b944302eac377..7628815150c45 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java @@ -19,7 +19,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.runtime.partitioner.HashPartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; /** @@ -28,11 +32,10 @@ * are also possible on a KeyedDataStream, with the exception of partitioning methods such as shuffle, forward and groupBy. * * - * @param - * The output type of the {@link KeyedDataStream}. + * @param The type of the elements in the Keyed Stream */ -public class KeyedDataStream extends DataStream { - KeySelector keySelector; +public class KeyedDataStream extends DataStream { + KeySelector keySelector; /** * Creates a new {@link KeyedDataStream} using the given {@link KeySelector} @@ -43,35 +46,35 @@ public class KeyedDataStream extends DataStream { * @param keySelector * Function for determining state partitions */ - public KeyedDataStream(DataStream dataStream, KeySelector keySelector) { - super(dataStream.partitionByHash(keySelector)); + public KeyedDataStream(DataStream dataStream, KeySelector keySelector) { + super(dataStream.getExecutionEnvironment(), new PartitionTransformation(dataStream.getTransformation(), new HashPartitioner(keySelector))); this.keySelector = keySelector; } - protected KeyedDataStream(KeyedDataStream dataStream) { - super(dataStream); - this.keySelector = dataStream.keySelector; - } - - public KeySelector getKeySelector() { + public KeySelector getKeySelector() { return this.keySelector; } @Override - protected DataStream setConnectionType(StreamPartitioner partitioner) { + protected DataStream setConnectionType(StreamPartitioner partitioner) { throw new UnsupportedOperationException("Cannot override partitioning for KeyedDataStream."); } - @Override - public KeyedDataStream copy() { - return new KeyedDataStream(this); - } - @Override public SingleOutputStreamOperator transform(String operatorName, - TypeInformation outTypeInfo, OneInputStreamOperator operator) { + TypeInformation outTypeInfo, OneInputStreamOperator operator) { + SingleOutputStreamOperator returnStream = super.transform(operatorName, outTypeInfo,operator); - streamGraph.setKey(returnStream.getId(), keySelector); + + ((OneInputTransformation) returnStream.getTransformation()).setStateKeySelector( + keySelector); return returnStream; } + + @Override + public DataStreamSink addSink(SinkFunction sinkFunction) { + DataStreamSink result = super.addSink(sinkFunction); + result.getTransformation().setStateKeySelector(keySelector); + return result; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java index b4a99c82c5ffb..016cf5e6ef619 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java @@ -24,23 +24,23 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.api.transformations.StreamTransformation; +import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; /** * The SingleOutputStreamOperator represents a user defined transformation * applied on a {@link DataStream} with one predefined output type. - * - * @param - * Output type of the operator. - * @param - * Type of the operator. + * + * @param The type of the elements in this Stream + * @param Type of the operator. */ -public class SingleOutputStreamOperator> extends - DataStream { +public class SingleOutputStreamOperator> extends DataStream { - protected boolean isSplit; - protected StreamOperator operator; + protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, StreamTransformation transformation) { + super(environment, transformation); + } /** * Gets the name of the current data stream. This name is @@ -48,8 +48,8 @@ public class SingleOutputStreamOperator name(String name){ - streamGraph.getStreamNode(id).setOperatorName(name); + public SingleOutputStreamOperator name(String name){ + transformation.setName(name); return this; } - protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, - TypeInformation outTypeInfo, StreamOperator operator) { - super(environment, outTypeInfo); - this.isSplit = false; - this.operator = operator; - } - - @SuppressWarnings("unchecked") - protected SingleOutputStreamOperator(DataStream dataStream) { - super(dataStream); - if (dataStream instanceof SingleOutputStreamOperator) { - this.isSplit = ((SingleOutputStreamOperator) dataStream).isSplit; - this.operator = ((SingleOutputStreamOperator) dataStream).operator; - } - } - /** * Sets the parallelism for this operator. The degree must be 1 or more. * @@ -86,13 +70,12 @@ protected SingleOutputStreamOperator(DataStream dataStream) { * The parallelism for this operator. * @return The operator with set parallelism. */ - public SingleOutputStreamOperator setParallelism(int parallelism) { + public SingleOutputStreamOperator setParallelism(int parallelism) { if (parallelism < 1) { throw new IllegalArgumentException("The parallelism of an operator must be at least 1."); } - this.parallelism = parallelism; - streamGraph.setParallelism(id, parallelism); + transformation.setParallelism(parallelism); return this; } @@ -105,39 +88,34 @@ public SingleOutputStreamOperator setParallelism(int parallelism) { * The maximum time between two output flushes. * @return The operator with buffer timeout set. */ - public SingleOutputStreamOperator setBufferTimeout(long timeoutMillis) { - streamGraph.setBufferTimeout(id, timeoutMillis); + public SingleOutputStreamOperator setBufferTimeout(long timeoutMillis) { + transformation.setBufferTimeout(timeoutMillis); return this; } @SuppressWarnings("unchecked") - public SingleOutputStreamOperator broadcast() { - return (SingleOutputStreamOperator) super.broadcast(); + public SingleOutputStreamOperator broadcast() { + return (SingleOutputStreamOperator) super.broadcast(); } @SuppressWarnings("unchecked") - public SingleOutputStreamOperator shuffle() { - return (SingleOutputStreamOperator) super.shuffle(); + public SingleOutputStreamOperator shuffle() { + return (SingleOutputStreamOperator) super.shuffle(); } @SuppressWarnings("unchecked") - public SingleOutputStreamOperator forward() { - return (SingleOutputStreamOperator) super.forward(); + public SingleOutputStreamOperator forward() { + return (SingleOutputStreamOperator) super.forward(); } @SuppressWarnings("unchecked") - public SingleOutputStreamOperator rebalance() { - return (SingleOutputStreamOperator) super.rebalance(); + public SingleOutputStreamOperator rebalance() { + return (SingleOutputStreamOperator) super.rebalance(); } @SuppressWarnings("unchecked") - public SingleOutputStreamOperator global() { - return (SingleOutputStreamOperator) super.global(); - } - - @Override - public SingleOutputStreamOperator copy() { - return new SingleOutputStreamOperator(this); + public SingleOutputStreamOperator global() { + return (SingleOutputStreamOperator) super.global(); } /** @@ -149,8 +127,8 @@ public SingleOutputStreamOperator copy() { * The selected {@link ChainingStrategy} * @return The operator with the modified chaining strategy */ - private SingleOutputStreamOperator setChainingStrategy(ChainingStrategy strategy) { - this.operator.setChainingStrategy(strategy); + private SingleOutputStreamOperator setChainingStrategy(ChainingStrategy strategy) { + this.transformation.setChainingStrategy(strategy); return this; } @@ -162,7 +140,7 @@ private SingleOutputStreamOperator setChainingStrategy(ChainingStrategy * * @return The operator with chaining disabled */ - public SingleOutputStreamOperator disableChaining() { + public SingleOutputStreamOperator disableChaining() { return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER); } @@ -173,7 +151,7 @@ public SingleOutputStreamOperator disableChaining() { * * @return The operator with chaining set. */ - public SingleOutputStreamOperator startNewChain() { + public SingleOutputStreamOperator startNewChain() { return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.HEAD); } @@ -216,7 +194,7 @@ public O returns(String typeInfoString) { if (typeInfoString == null) { throw new IllegalArgumentException("Type information string must not be null."); } - return returns(TypeInfoParser.parse(typeInfoString)); + return returns(TypeInfoParser.parse(typeInfoString)); } /** @@ -243,11 +221,11 @@ public O returns(String typeInfoString) { * type information as a return type hint * @return This operator with a given return type hint. */ - public O returns(TypeInformation typeInfo) { + public O returns(TypeInformation typeInfo) { if (typeInfo == null) { throw new IllegalArgumentException("Type information must not be null."); } - fillInType(typeInfo); + transformation.setOutputType(typeInfo); @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; @@ -277,13 +255,13 @@ public O returns(TypeInformation typeInfo) { * @return This operator with a given return type hint. */ @SuppressWarnings("unchecked") - public O returns(Class typeClass) { + public O returns(Class typeClass) { if (typeClass == null) { throw new IllegalArgumentException("Type class must not be null."); } try { - TypeInformation ti = (TypeInformation) TypeExtractor.createTypeInfo(typeClass); + TypeInformation ti = (TypeInformation) TypeExtractor.createTypeInfo(typeClass); return returns(ti); } catch (InvalidTypesException e) { @@ -291,6 +269,11 @@ public O returns(Class typeClass) { } } + @Override + protected DataStream setConnectionType(StreamPartitioner partitioner) { + return new SingleOutputStreamOperator(this.getExecutionEnvironment(), new PartitionTransformation(this.getTransformation(), partitioner)); + } + /** * By default all operators in a streaming job share the same resource * group. Each resource group takes as many task manager slots as the @@ -305,8 +288,8 @@ public O returns(Class typeClass) { * * @return The operator as a part of a new resource group. */ - public SingleOutputStreamOperator startNewResourceGroup() { - streamGraph.setResourceStrategy(getId(), ResourceStrategy.NEWGROUP); + public SingleOutputStreamOperator startNewResourceGroup() { + transformation.setResourceStrategy(ResourceStrategy.NEWGROUP); return this; } @@ -319,8 +302,8 @@ public SingleOutputStreamOperator startNewResourceGroup() { * * @return The operator with isolated resource group. */ - public SingleOutputStreamOperator isolateResources() { - streamGraph.setResourceStrategy(getId(), ResourceStrategy.ISOLATE); + public SingleOutputStreamOperator isolateResources() { + transformation.setResourceStrategy(ResourceStrategy.ISOLATE); return this; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java index 6b95fe784f7d8..bc9ecfbf433a9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java @@ -17,23 +17,23 @@ package org.apache.flink.streaming.api.datastream; -import java.util.Arrays; - +import com.google.common.collect.Lists; import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.transformations.SelectTransformation; +import org.apache.flink.streaming.api.transformations.SplitTransformation; /** * The SplitDataStream represents an operator that has been split using an * {@link OutputSelector}. Named outputs can be selected using the * {@link #select} function. To apply transformation on the whole output simply * call the transformation on the SplitDataStream - * - * @param - * The type of the output. + * + * @param The type of the elements in the Stream */ public class SplitDataStream extends DataStream { - protected SplitDataStream(DataStream dataStream) { - super(dataStream); + protected SplitDataStream(DataStream dataStream, OutputSelector outputSelector) { + super(dataStream.getExecutionEnvironment(), new SplitTransformation(dataStream.getTransformation(), outputSelector)); } /** @@ -55,12 +55,8 @@ private DataStream selectOutput(String[] outputNames) { } } - DataStream returnStream = copy(); - - for (DataStream ds : returnStream.unionedStreams) { - ds.selectedNames = Arrays.asList(outputNames); - } - return returnStream; + SelectTransformation selectTransform = new SelectTransformation(this.getTransformation(), Lists.newArrayList(outputNames)); + return new DataStream(this.getExecutionEnvironment(), selectTransform); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java index 9565f4b773fc7..bf3a11a63f97e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java @@ -99,7 +99,7 @@ public class WindowedDataStream { protected EvictionPolicy userEvicter; protected WindowedDataStream(DataStream dataStream, WindowingHelper policyHelper) { - this.dataStream = dataStream.copy(); + this.dataStream = dataStream; this.triggerHelper = policyHelper; if (dataStream instanceof GroupedDataStream) { @@ -109,7 +109,7 @@ protected WindowedDataStream(DataStream dataStream, WindowingHelper po protected WindowedDataStream(DataStream dataStream, TriggerPolicy trigger, EvictionPolicy evicter) { - this.dataStream = dataStream.copy(); + this.dataStream = dataStream; this.userTrigger = trigger; this.userEvicter = evicter; @@ -120,7 +120,7 @@ protected WindowedDataStream(DataStream dataStream, TriggerPolicy trig } protected WindowedDataStream(WindowedDataStream windowedDataStream) { - this.dataStream = windowedDataStream.dataStream.copy(); + this.dataStream = windowedDataStream.dataStream; this.discretizerKey = windowedDataStream.discretizerKey; this.groupByKey = windowedDataStream.groupByKey; this.triggerHelper = windowedDataStream.triggerHelper; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java index 3dd02a3fa79f6..e0aafb737bc33 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java @@ -30,10 +30,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.functions.co.CrossWindowFunction; -import org.apache.flink.streaming.api.operators.co.CoStreamWindow; public class StreamCrossOperator extends - TemporalOperator> { + TemporalOperator>> { public StreamCrossOperator(DataStream input1, DataStream input2) { super(input1, input2); @@ -48,37 +47,42 @@ protected F clean(F f) { } @Override - protected CrossWindow createNextWindowOperator() { + protected CrossWindow> createNextWindowOperator() { CrossWindowFunction> crossWindowFunction = new CrossWindowFunction>( clean(new CrossOperator.DefaultCrossFunction())); - return new CrossWindow(this, input1.connect(input2).addGeneralWindowCombine( + return new CrossWindow>(this, input1.connect(input2).addGeneralWindowCombine( crossWindowFunction, new TupleTypeInfo>(input1.getType(), input2.getType()), windowSize, slideInterval, timeStamp1, timeStamp2)); } - public static class CrossWindow extends - SingleOutputStreamOperator, CrossWindow> implements - TemporalWindow> { + public static class CrossWindow extends + SingleOutputStreamOperator> implements + TemporalWindow> { private StreamCrossOperator op; - public CrossWindow(StreamCrossOperator op, DataStream> ds) { - super(ds); + public CrossWindow(StreamCrossOperator op, DataStream ds) { + super(ds.getExecutionEnvironment(), ds.getTransformation()); this.op = op; } - public CrossWindow every(long length, TimeUnit timeUnit) { + public CrossWindow every(long length, TimeUnit timeUnit) { return every(timeUnit.toMillis(length)); } @SuppressWarnings("unchecked") - public CrossWindow every(long length) { - ((CoStreamWindow) streamGraph.getStreamNode(id).getOperator()) - .setSlideSize(length); - return this; + public CrossWindow every(long length) { + + CrossWindowFunction> crossWindowFunction = new CrossWindowFunction>( + clean(new CrossOperator.DefaultCrossFunction())); + + return (CrossWindow) new CrossWindow>(op, op.input1.connect(op.input2).addGeneralWindowCombine( + crossWindowFunction, + new TupleTypeInfo>(op.input1.getType(), op.input2.getType()), op.windowSize, + length, op.timeStamp1, op.timeStamp2)); } /** @@ -97,13 +101,12 @@ public CrossWindow every(long length) { TypeInformation outTypeInfo = TypeExtractor.getCrossReturnTypes(function, op.input1.getType(), op.input2.getType()); - CoStreamWindow operator = new CoStreamWindow( - new CrossWindowFunction(clean(function)), op.windowSize, - op.slideInterval, op.timeStamp1, op.timeStamp2); - - streamGraph.setOperator(id, operator); + CrossWindowFunction crossWindowFunction = new CrossWindowFunction(clean(function)); - return ((SingleOutputStreamOperator) this).returns(outTypeInfo); + return new CrossWindow(op, op.input1.connect(op.input2).addGeneralWindowCombine( + crossWindowFunction, + outTypeInfo, op.windowSize, + op.slideInterval, op.timeStamp1, op.timeStamp2)); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java index e18e14bc023ab..e48d707dd65a3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java @@ -31,7 +31,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.functions.co.JoinWindowFunction; -import org.apache.flink.streaming.api.operators.co.CoStreamWindow; import org.apache.flink.streaming.util.keys.KeySelectorUtil; public class StreamJoinOperator extends @@ -156,7 +155,7 @@ private JoinPredicate(StreamJoinOperator operator, KeySelector ke * @return A streaming join operator. Call {@link JoinedStream#with} to * apply a custom wrapping */ - public JoinedStream equalTo(int... fields) { + public JoinedStream> equalTo(int... fields) { keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys(fields, type2), type2, op.input1.getExecutionEnvironment().getConfig()); return createJoinOperator(); @@ -175,7 +174,7 @@ public JoinedStream equalTo(int... fields) { * @return A streaming join operator. Call {@link JoinedStream#with} to * apply a custom wrapping */ - public JoinedStream equalTo(String... fields) { + public JoinedStream> equalTo(String... fields) { this.keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys(fields, type2), type2, op.input1.getExecutionEnvironment().getConfig()); return createJoinOperator(); @@ -198,12 +197,12 @@ public JoinedStream equalTo(String... fields) { * @return A streaming join operator. Call {@link JoinedStream#with} to * apply a custom wrapping */ - public JoinedStream equalTo(KeySelector keySelector) { + public JoinedStream> equalTo(KeySelector keySelector) { this.keys2 = keySelector; return createJoinOperator(); } - private JoinedStream createJoinOperator() { + private JoinedStream> createJoinOperator() { JoinFunction> joinFunction = new DefaultJoinFunction(); @@ -213,42 +212,44 @@ private JoinedStream createJoinOperator() { TypeInformation> outType = new TupleTypeInfo>( op.input1.getType(), op.input2.getType()); - return new JoinedStream(this, op.input1 + return new JoinedStream>(this, op.input1 .groupBy(keys1) .connect(op.input2.groupBy(keys2)) .addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2)); } - } - - public static class JoinedStream extends - SingleOutputStreamOperator, JoinedStream> { - private final JoinPredicate predicate; - - private JoinedStream(JoinPredicate predicate, DataStream> ds) { - super(ds); - this.predicate = predicate; - } - - /** - * Completes a stream join.

    The resulting operator wraps each pair - * of joining elements using the user defined {@link JoinFunction} - * - * @return The joined data stream. - */ - @SuppressWarnings("unchecked") - public SingleOutputStreamOperator with(JoinFunction joinFunction) { - - TypeInformation outType = TypeExtractor.getJoinReturnTypes(joinFunction, - predicate.op.input1.getType(), predicate.op.input2.getType()); - - CoStreamWindow operator = new CoStreamWindow( - getJoinWindowFunction(joinFunction, predicate), predicate.op.windowSize, - predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2); - - streamGraph.setOperator(id, operator); - return ((SingleOutputStreamOperator) this).returns(outType); + public static class JoinedStream extends + SingleOutputStreamOperator> { + private final JoinPredicate predicate; + + private JoinedStream(JoinPredicate predicate, DataStream ds) { + super(ds.getExecutionEnvironment(), ds.getTransformation()); + this.predicate = predicate; + } + + /** + * Completes a stream join.

    The resulting operator wraps each pair + * of joining elements using the user defined {@link JoinFunction} + * + * @return The joined data stream. + */ + @SuppressWarnings("unchecked") + public SingleOutputStreamOperator with(JoinFunction joinFunction) { + + TypeInformation outType = TypeExtractor.getJoinReturnTypes(joinFunction, + predicate.op.input1.getType(), predicate.op.input2.getType()); + + JoinWindowFunction joinWindowFunction = getJoinWindowFunction(joinFunction, predicate); + + + return new JoinedStream( + predicate, predicate.op.input1 + .groupBy(predicate.keys1) + .connect(predicate.op.input2.groupBy(predicate.keys2)) + .addGeneralWindowCombine(joinWindowFunction, outType, predicate.op.windowSize, + predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2)); + } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java index 3fe7eb73f7419..9da00f2756e92 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java @@ -40,8 +40,8 @@ public TemporalOperator(DataStream input1, DataStream input2) { if (input1 == null || input2 == null) { throw new NullPointerException(); } - this.input1 = input1.copy(); - this.input2 = input2.copy(); + this.input1 = input1; + this.input2 = input2; } /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java index 3efad93e71d94..58459b7720f65 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java @@ -43,9 +43,8 @@ public JobExecutionResult execute() throws Exception { */ @Override public JobExecutionResult execute(String jobName) throws Exception { - JobExecutionResult result = ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(jobName), getParallelism(), - getConfig().isSysoutLoggingEnabled()); - streamGraph.clear(); // clear graph to allow submitting another job via the same environment. + JobExecutionResult result = ClusterUtil.runOnMiniCluster(getStreamGraph().getJobGraph(), getParallelism(), getConfig().isSysoutLoggingEnabled()); + transformations.clear(); return result; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 75e15d77d4cbf..2f8938f8cbfe4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -84,15 +84,15 @@ public RemoteStreamEnvironment(String host, int port, String... jarFiles) { @Override public JobExecutionResult execute() throws ProgramInvocationException { - - JobGraph jobGraph = streamGraph.getJobGraph(); + JobGraph jobGraph = getStreamGraph().getJobGraph(); + transformations.clear(); return executeRemotely(jobGraph); } @Override public JobExecutionResult execute(String jobName) throws ProgramInvocationException { - - JobGraph jobGraph = streamGraph.getJobGraph(jobName); + JobGraph jobGraph = getStreamGraph().getJobGraph(jobName); + transformations.clear(); return executeRemotely(jobGraph); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java index 9bfeb2f3dae16..c2335d6bbb26f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java @@ -63,15 +63,16 @@ public JobExecutionResult execute() throws Exception { @Override public JobExecutionResult execute(String jobName) throws Exception { - currentEnvironment = null; JobGraph jobGraph; if (jobName == null) { - jobGraph = this.streamGraph.getJobGraph(); + jobGraph = this.getStreamGraph().getJobGraph(); } else { - jobGraph = this.streamGraph.getJobGraph(jobName); + jobGraph = this.getStreamGraph().getJobGraph(jobName); } + transformations.clear(); + for (File file : jars) { jobGraph.addJar(new Path(file.getAbsolutePath())); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index fba4e28d85096..a019a3150a171 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -19,6 +19,7 @@ import com.esotericsoftware.kryo.Serializer; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; @@ -60,8 +61,9 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource; import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.types.StringValue; import org.apache.flink.util.SplittableIterator; @@ -86,21 +88,25 @@ public abstract class StreamExecutionEnvironment { private ExecutionConfig config = new ExecutionConfig(); - protected static StreamExecutionEnvironment currentEnvironment; + protected List> transformations = Lists.newArrayList(); - protected StreamGraph streamGraph; + protected boolean isChainingEnabled = true; + + protected long checkpointInterval = -1; // disabled + + protected CheckpointingMode checkpointingMode = null; + + protected boolean forceCheckpointing = false; + + protected StateHandleProvider stateHandleProvider; + + /** The environment of the context (local by default, cluster if invoked through command line) */ + private static StreamExecutionEnvironmentFactory contextEnvironmentFactory; // -------------------------------------------------------------------------------------------- // Constructor and Properties // -------------------------------------------------------------------------------------------- - /** - * Constructor for creating StreamExecutionEnvironment - */ - protected StreamExecutionEnvironment() { - streamGraph = new StreamGraph(this); - } - /** * Gets the config object. */ @@ -221,10 +227,19 @@ public long getBufferTimeout() { * @return StreamExecutionEnvironment with chaining disabled. */ public StreamExecutionEnvironment disableOperatorChaining() { - streamGraph.setChaining(false); + this.isChainingEnabled = false; return this; } + /** + * Returns whether operator chaining is enabled. + * + * @return {@code true} if chaining is enabled, false otherwise. + */ + public boolean isChainingEnabled() { + return isChainingEnabled; + } + // ------------------------------------------------------------------------ // Checkpointing Settings // ------------------------------------------------------------------------ @@ -275,10 +290,9 @@ public StreamExecutionEnvironment enableCheckpointing(long interval, Checkpointi if (interval <= 0) { throw new IllegalArgumentException("the checkpoint interval must be positive"); } - - streamGraph.setCheckpointingEnabled(true); - streamGraph.setCheckpointingInterval(interval); - streamGraph.setCheckpointingMode(mode); + + this.checkpointInterval = interval; + this.checkpointingMode = mode; return this; } @@ -303,19 +317,9 @@ public StreamExecutionEnvironment enableCheckpointing(long interval, Checkpointi */ @Deprecated public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode, boolean force) { - if (mode == null) { - throw new NullPointerException("checkpoint mode must not be null"); - } - if (interval <= 0) { - throw new IllegalArgumentException("the checkpoint interval must be positive"); - } - - streamGraph.setCheckpointingEnabled(true); - streamGraph.setCheckpointingInterval(interval); - streamGraph.setCheckpointingMode(mode); - if (force) { - streamGraph.forceCheckpoint(); - } + this.enableCheckpointing(interval, mode); + + this.forceCheckpointing = force; return this; } @@ -334,11 +338,34 @@ public StreamExecutionEnvironment enableCheckpointing(long interval, Checkpointi * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.

    */ public StreamExecutionEnvironment enableCheckpointing() { - streamGraph.setCheckpointingEnabled(true); - streamGraph.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); + enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE); return this; } + /** + * Returns the checkpointing interval or -1 if checkpointing is disabled. + * + * @return The checkpointing interval or -1 + */ + public long getCheckpointInterval() { + return checkpointInterval; + } + + + /** + * Returns whether checkpointing is force-enabled. + */ + public boolean isForceCheckpointing() { + return forceCheckpointing; + } + + /** + * Returns the {@link CheckpointingMode}. + */ + public CheckpointingMode getCheckpointingMode() { + return checkpointingMode; + } + /** * Sets the {@link StateHandleProvider} used for storing operator state * checkpoints when checkpointing is enabled. @@ -348,10 +375,21 @@ public StreamExecutionEnvironment enableCheckpointing() { * */ public StreamExecutionEnvironment setStateHandleProvider(StateHandleProvider provider) { - streamGraph.setStateHandleProvider(provider); + this.stateHandleProvider = provider; return this; } + /** + * Returns the {@link org.apache.flink.runtime.state.StateHandle} + * + * @see #setStateHandleProvider(org.apache.flink.runtime.state.StateHandleProvider) + * + * @return The StateHandleProvider + */ + public StateHandleProvider getStateHandleProvider() { + return stateHandleProvider; + } + /** * Sets the number of times that failed tasks are re-executed. A value of * zero effectively disables fault tolerance. A value of {@code -1} @@ -591,7 +629,7 @@ public DataStreamSource fromCollection(Collection data, TypeInfo // must not have null elements and mixed elements FromElementsFunction.checkCollection(data, typeInfo.getTypeClass()); - + SourceFunction function; try { function = new FromElementsFunction(typeInfo.createSerializer(getConfig()), data); @@ -599,7 +637,7 @@ public DataStreamSource fromCollection(Collection data, TypeInfo catch (IOException e) { throw new RuntimeException(e.getMessage(), e); } - return addSource(function, "Collection Source", typeInfo); + return addSource(function, "Collection Source", typeInfo).setParallelism(1); } /** @@ -980,15 +1018,12 @@ public DataStreamSource createInput(InputFormat inputFormat, private DataStreamSource createInput(InputFormat inputFormat, TypeInformation typeInfo, String sourceName) { FileSourceFunction function = new FileSourceFunction(inputFormat, typeInfo); - DataStreamSource returnStream = addSource(function, sourceName).returns(typeInfo); - streamGraph.setInputFormat(returnStream.getId(), inputFormat); - return returnStream; + return addSource(function, sourceName).returns(typeInfo); } /** - * Adds a data source with a custom type information thus opening a - * {@link org.apache.flink.streaming.api.datastream.DataStream}. Only in very special cases does the user need - * to support type information. Otherwise use {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)} + * Adds a Data Source to the streaming topology. + * *

    * By default sources have a parallelism of 1. To enable parallel execution, the user defined source should * implement {@link org.apache.flink.streaming.api.functions.source.ParallelSourceFunction} or extend {@link @@ -1078,10 +1113,9 @@ public DataStreamSource addSource(SourceFunction function, Strin boolean isParallel = function instanceof ParallelSourceFunction; clean(function); - StreamOperator sourceOperator = new StreamSource(function); + StreamSource sourceOperator = new StreamSource(function); - return new DataStreamSource(this, sourceName, typeInfo, sourceOperator, - isParallel, sourceName); + return new DataStreamSource(this, typeInfo, sourceOperator, isParallel, sourceName); } // -------------------------------------------------------------------------------------------- @@ -1098,20 +1132,20 @@ public DataStreamSource addSource(SourceFunction function, Strin * executed. */ public static StreamExecutionEnvironment getExecutionEnvironment() { - if (currentEnvironment != null) { - return currentEnvironment; + if (contextEnvironmentFactory != null) { + return contextEnvironmentFactory.createExecutionEnvironment(); } + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); if (env instanceof ContextEnvironment) { ContextEnvironment ctx = (ContextEnvironment) env; - currentEnvironment = createContextEnvironment(ctx.getClient(), ctx.getJars(), + return createContextEnvironment(ctx.getClient(), ctx.getJars(), ctx.getParallelism(), ctx.isWait()); } else if (env instanceof OptimizerPlanEnvironment | env instanceof PreviewPlanEnvironment) { - currentEnvironment = new StreamPlanEnvironment(env); + return new StreamPlanEnvironment(env); } else { return createLocalEnvironment(); } - return currentEnvironment; } private static StreamExecutionEnvironment createContextEnvironment(Client client, @@ -1143,9 +1177,9 @@ public static LocalStreamEnvironment createLocalEnvironment() { * @return A local execution environment with the specified parallelism. */ public static LocalStreamEnvironment createLocalEnvironment(int parallelism) { - currentEnvironment = new LocalStreamEnvironment(); - currentEnvironment.setParallelism(parallelism); - return (LocalStreamEnvironment) currentEnvironment; + LocalStreamEnvironment env = new LocalStreamEnvironment(); + env.setParallelism(parallelism); + return env; } // TODO:fix cluster default parallelism @@ -1172,8 +1206,8 @@ public static LocalStreamEnvironment createLocalEnvironment(int parallelism) { */ public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port, String... jarFiles) { - currentEnvironment = new RemoteStreamEnvironment(host, port, jarFiles); - return currentEnvironment; + RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles); + return env; } /** @@ -1199,9 +1233,9 @@ public static StreamExecutionEnvironment createRemoteEnvironment(String host, in */ public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles) { - currentEnvironment = new RemoteStreamEnvironment(host, port, jarFiles); - currentEnvironment.setParallelism(parallelism); - return currentEnvironment; + RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles); + env.setParallelism(parallelism); + return env; } /** @@ -1239,7 +1273,11 @@ public static StreamExecutionEnvironment createRemoteEnvironment(String host, in * @return The streamgraph representing the transformations */ public StreamGraph getStreamGraph() { - return streamGraph; + if (transformations.size() <= 0) { + throw new IllegalStateException("No operators defined in streaming topology. Cannot execute."); + } + StreamGraph result = StreamGraphGenerator.generate(this, transformations); + return result; } /** @@ -1254,10 +1292,6 @@ public String getExecutionPlan() { return getStreamGraph().getStreamingPlanAsJSON(); } - protected static void initializeFromFactory(StreamExecutionEnvironmentFactory eef) { - currentEnvironment = eef.createExecutionEnvironment(); - } - /** * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig} @@ -1270,4 +1304,28 @@ public F clean(F f) { return f; } + /** + * Adds an operator to the list of operators that should be executed when calling + * {@link #execute}. + * + *

    + * When calling {@link #execute()} only the operators that where previously added to the list + * are executed. + * + *

    + * This is not meant to be used by users. The API methods that create operators must call + * this method. + */ + public void addOperator(StreamTransformation transformation) { + Preconditions.checkNotNull(transformation, "Sinks must not be null."); + this.transformations.add(transformation); + } + + // -------------------------------------------------------------------------------------------- + // Methods to control the context and local environments for execution from packaged programs + // -------------------------------------------------------------------------------------------- + + protected static void initializeContextEnvironment(StreamExecutionEnvironmentFactory ctx) { + contextEnvironmentFactory = ctx; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java index 02fccd00b644f..8c1408ec5bd73 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java @@ -24,6 +24,7 @@ import org.apache.flink.client.program.PackagedProgram.PreviewPlanEnvironment; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.streaming.api.graph.StreamGraph; public class StreamPlanEnvironment extends StreamExecutionEnvironment { @@ -55,10 +56,12 @@ public JobExecutionResult execute() throws Exception { @Override public JobExecutionResult execute(String jobName) throws Exception { - currentEnvironment = null; + StreamGraph streamGraph = getStreamGraph(); streamGraph.setJobName(jobName); + transformations.clear(); + if (env instanceof OptimizerPlanEnvironment) { ((OptimizerPlanEnvironment) env).setPlan(streamGraph); } else if (env instanceof PreviewPlanEnvironment) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java index cf08e5a560ad7..253c07619ddfe 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java @@ -134,4 +134,12 @@ public void cancel() { isRunning = false; } + + /** + * Returns the {@code InputFormat}. This is only needed because we need to set the input + * split assigner on the {@code StreamGraph}. + */ + public InputFormat getFormat() { + return format; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java index bc20fff126776..743ee4a8a195b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,7 +52,19 @@ public String getJSON() throws JSONException { JSONArray nodes = new JSONArray(); json.put("nodes", nodes); List operatorIDs = new ArrayList(streamGraph.getVertexIDs()); - Collections.sort(operatorIDs); + Collections.sort(operatorIDs, new Comparator() { + @Override + public int compare(Integer o1, Integer o2) { + // put sinks at the back + if (streamGraph.getSinkIDs().contains(o1)) { + return 1; + } else if (streamGraph.getSinkIDs().contains(o2)) { + return -1; + } else { + return o1 - o2; + } + } + }); visit(nodes, operatorIDs, new HashMap()); return json.toString(); } @@ -87,7 +100,7 @@ private void visit(JSONArray jsonArray, List toVisit, for (StreamEdge inEdge : vertex.getInEdges()) { int operator = inEdge.getSourceId(); - if (streamGraph.vertexIDtoLoop.containsKey(operator)) { + if (streamGraph.vertexIDtoLoopTimeout.containsKey(operator)) { iterationHead = operator; } } @@ -119,7 +132,7 @@ private void visitIteration(JSONArray jsonArray, List toVisit, int head toVisit.remove(vertexID); // Ignoring head and tail to avoid redundancy - if (!streamGraph.vertexIDtoLoop.containsKey(vertexID)) { + if (!streamGraph.vertexIDtoLoopTimeout.containsKey(vertexID)) { JSONObject obj = new JSONObject(); jsonArray.put(obj); decorateNode(vertexID, obj); @@ -131,7 +144,7 @@ private void visitIteration(JSONArray jsonArray, List toVisit, int head if (edgeRemapings.keySet().contains(inputID)) { decorateEdge(inEdges, vertexID, inputID, inputID); - } else if (!streamGraph.vertexIDtoLoop.containsKey(inputID)) { + } else if (!streamGraph.vertexIDtoLoopTimeout.containsKey(inputID)) { decorateEdge(iterationInEdges, vertexID, inputID, inputID); } } @@ -147,8 +160,7 @@ private void decorateEdge(JSONArray inputArray, int vertexID, int mappedInputID, JSONObject input = new JSONObject(); inputArray.put(input); input.put(ID, mappedInputID); - input.put(SHIP_STRATEGY, streamGraph.getStreamEdge(inputID, vertexID).getPartitioner() - .getStrategy()); + input.put(SHIP_STRATEGY, streamGraph.getStreamEdge(inputID, vertexID).getPartitioner()); input.put(SIDE, (inputArray.length() == 0) ? "first" : "second"); } @@ -161,8 +173,10 @@ private void decorateNode(Integer vertexID, JSONObject node) throws JSONExceptio if (streamGraph.getSourceIDs().contains(vertexID)) { node.put(PACT, "Data Source"); + } else if (streamGraph.getSinkIDs().contains(vertexID)) { + node.put(PACT, "Data Sink"); } else { - node.put(PACT, "Data Stream"); + node.put(PACT, "Operator"); } StreamOperator operator = streamGraph.getStreamNode(vertexID).getOperator(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 4f19db6febcf7..2c422d91109e6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -257,29 +257,6 @@ public long getIterationWaitTime() { return config.getLong(ITERATON_WAIT, 0); } - public void setSelectedNames(Integer output, List selected) { - if (selected == null) { - selected = new ArrayList(); - } - - try { - InstantiationUtil.writeObjectToConfig(selected, this.config, OUTPUT_NAME + output); - } catch (IOException e) { - throw new StreamTaskException("Cannot serialize OutputSelector for name \"" + output+ "\".", e); - } - } - - @SuppressWarnings("unchecked") - public List getSelectedNames(Integer output, ClassLoader cl) { - List selectedNames; - try { - selectedNames = (List) InstantiationUtil.readObjectFromConfig(this.config, OUTPUT_NAME + output, cl); - } catch (Exception e) { - throw new StreamTaskException("Cannot deserialize OutputSelector for name \"" + output + "\".", e); - } - return selectedNames == null ? new ArrayList() : selectedNames; - } - public void setNumberOfInputs(int numberOfInputs) { config.setInteger(NUMBER_OF_INPUTS, numberOfInputs); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index 65736f534b3b9..6474ae9d652f2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -23,14 +23,15 @@ import java.io.PrintWriter; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -44,11 +45,11 @@ import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.collector.selector.OutputSelector; -import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; @@ -84,11 +85,15 @@ public class StreamGraph extends StreamingPlan { private Map streamNodes; private Set sources; + private Set sinks; + private Map>> virtualSelectNodes; + private Map>> virtuaPartitionNodes; - private Map streamLoops; - protected Map vertexIDtoLoop; protected Map vertexIDtoBrokerID; + protected Map vertexIDtoLoopTimeout; private StateHandleProvider stateHandleProvider; + private Set> iterationSourceSinkPairs; + private boolean forceCheckpoint = false; public StreamGraph(StreamExecutionEnvironment environment) { @@ -104,11 +109,14 @@ public StreamGraph(StreamExecutionEnvironment environment) { * Remove all registered nodes etc. */ public void clear() { - streamNodes = new HashMap(); - streamLoops = new HashMap(); - vertexIDtoLoop = new HashMap(); - vertexIDtoBrokerID = new HashMap(); - sources = new HashSet(); + streamNodes = Maps.newHashMap(); + virtualSelectNodes = Maps.newHashMap(); + virtuaPartitionNodes = Maps.newHashMap(); + vertexIDtoBrokerID = Maps.newHashMap(); + vertexIDtoLoopTimeout = Maps.newHashMap(); + iterationSourceSinkPairs = Sets.newHashSet(); + sources = Sets.newHashSet(); + sinks = Sets.newHashSet(); } protected ExecutionConfig getExecutionConfig() { @@ -167,7 +175,7 @@ public void setCheckpointingMode(CheckpointingMode checkpointingMode) { public boolean isIterative() { - return !streamLoops.isEmpty(); + return!vertexIDtoLoopTimeout.isEmpty(); } public void addSource(Integer vertexID, StreamOperator operatorObject, @@ -176,6 +184,12 @@ public void addSource(Integer vertexID, StreamOperator operatorOb sources.add(vertexID); } + public void addSink(Integer vertexID, StreamOperator operatorObject, + TypeInformation inTypeInfo, TypeInformation outTypeInfo, String operatorName) { + addOperator(vertexID, operatorObject, inTypeInfo, outTypeInfo, operatorName); + sinks.add(vertexID); + } + public void addOperator(Integer vertexID, StreamOperator operatorObject, TypeInformation inTypeInfo, TypeInformation outTypeInfo, String operatorName) { @@ -212,224 +226,141 @@ public void addCoOperator(Integer vertexID, } } - public void addIterationHead(Integer iterationHead, Integer iterationID, long timeOut, - TypeInformation feedbackType) { - // If there is no loop object created for this iteration create one - StreamLoop loop = streamLoops.get(iterationID); - if (loop == null) { - loop = new StreamLoop(iterationID, timeOut, feedbackType); - streamLoops.put(iterationID, loop); - } - - loop.addHeadOperator(getStreamNode(iterationHead)); - } - - public void addIterationTail(List> feedbackStreams, Integer iterationID, - boolean keepPartitioning) { + protected StreamNode addNode(Integer vertexID, Class vertexClass, + StreamOperator operatorObject, String operatorName) { - if (!streamLoops.containsKey(iterationID)) { - throw new RuntimeException("Cannot close iteration without head operator."); + if (streamNodes.containsKey(vertexID)) { + throw new RuntimeException("Duplicate vertexID " + vertexID); } - StreamLoop loop = streamLoops.get(iterationID); + StreamNode vertex = new StreamNode(environemnt, vertexID, operatorObject, operatorName, + new ArrayList>(), vertexClass); - for (DataStream stream : feedbackStreams) { - loop.addTailOperator(getStreamNode(stream.getId()), stream.getPartitioner(), - stream.getSelectedNames()); - } + streamNodes.put(vertexID, vertex); - if (keepPartitioning) { - loop.applyTailPartitioning(); - } + return vertex; } - @SuppressWarnings({ "rawtypes", "unchecked" }) - public void finalizeLoops() { - - // We create each loop separately, the order does not matter as sinks - // and sources don't interact - for (StreamLoop loop : streamLoops.values()) { - - // We make sure not to re-create the loops if the method is called - // multiple times - if (loop.getSourceSinkPairs().isEmpty()) { - - List headOps = loop.getHeads(); - List tailOps = loop.getTails(); + /** + * Adds a new virtual node that is used to connect a downstream vertex to only the outputs + * with the selected names. + * + * When adding an edge from the virtual node to a downstream node the connection will be made + * to the original node, only with the selected names given here. + * + * @param originalId ID of the node that should be connected to. + * @param virtualId ID of the virtual node. + * @param selectedNames The selected names. + */ + public void addVirtualSelectNode(Integer originalId, Integer virtualId, List selectedNames) { - // This means that the iteration was not closed. It should not - // be - // allowed. - if (tailOps.isEmpty()) { - throw new RuntimeException("Cannot execute job with empty iterations."); - } + if (virtualSelectNodes.containsKey(virtualId)) { + throw new IllegalStateException("Already has virtual select node with id " + virtualId); + } - // Check whether we keep the feedback partitioning - if (loop.keepsPartitioning()) { - // This is the complicated case as we need to enforce - // partitioning on the tail -> sink side, which - // requires strict forward connections at source -> head - - // We need one source/sink pair per different head - // parallelism - // as we depend on strict forwards connections - Map> parallelismToHeads = new HashMap>(); - - // Group head operators by parallelism - for (StreamNode head : headOps) { - int p = head.getParallelism(); - if (!parallelismToHeads.containsKey(p)) { - parallelismToHeads.put(p, new ArrayList()); - } - parallelismToHeads.get(p).add(head); - } - - // We create the sink/source pair for each parallelism - // group, - // tails will forward to all sinks but each head operator - // will - // only receive from one source (corresponding to its - // parallelism) - int c = 0; - for (Entry> headGroup : parallelismToHeads.entrySet()) { - List headOpsInGroup = headGroup.getValue(); - - Tuple2 sourceSinkPair = createItSourceAndSink(loop, - c); - StreamNode source = sourceSinkPair.f0; - StreamNode sink = sourceSinkPair.f1; - - // We connect the source to the heads in this group - // (forward), setting - // type to 2 in case we have a coIteration (this sets - // the - // input as the second input of the co-operator) - for (StreamNode head : headOpsInGroup) { - int inputType = loop.isCoIteration() ? 2 : 0; - addEdge(source.getId(), head.getId(), new RebalancePartitioner(true), - inputType, new ArrayList()); - } - - // We connect all the tails to the sink keeping the - // partitioner - for (int i = 0; i < tailOps.size(); i++) { - StreamNode tail = tailOps.get(i); - StreamPartitioner partitioner = loop.getTailPartitioners().get(i); - addEdge(tail.getId(), sink.getId(), partitioner.copy(), 0, loop - .getTailSelectedNames().get(i)); - } - - // We set the sink/source parallelism to the group - // parallelism - source.setParallelism(headGroup.getKey()); - sink.setParallelism(source.getParallelism()); - - // We set the proper serializers for the sink/source - setSerializersFrom(tailOps.get(0).getId(), sink.getId()); - if (loop.isCoIteration()) { - source.setSerializerOut(loop.getFeedbackType().createSerializer(executionConfig)); - } else { - setSerializersFrom(headOpsInGroup.get(0).getId(), source.getId()); - } - - c++; - } - - } else { - // This is the most simple case, we add one iteration - // sink/source pair with the parallelism of the first tail - // operator. Tail operators will forward the records and - // partitioning will be enforced from source -> head - - Tuple2 sourceSinkPair = createItSourceAndSink(loop, 0); - StreamNode source = sourceSinkPair.f0; - StreamNode sink = sourceSinkPair.f1; - - // We get the feedback partitioner from the first input of - // the - // first head. - StreamPartitioner partitioner = headOps.get(0).getInEdges().get(0) - .getPartitioner(); - - // Connect the sources to heads using this partitioner - for (StreamNode head : headOps) { - addEdge(source.getId(), head.getId(), partitioner.copy(), 0, - new ArrayList()); - } - - // The tails are connected to the sink with forward - // partitioning - for (int i = 0; i < tailOps.size(); i++) { - StreamNode tail = tailOps.get(i); - addEdge(tail.getId(), sink.getId(), new RebalancePartitioner(true), 0, loop - .getTailSelectedNames().get(i)); - } - - // We set the parallelism to match the first tail op to make - // the - // forward more efficient - sink.setParallelism(tailOps.get(0).getParallelism()); - source.setParallelism(sink.getParallelism()); - - // We set the proper serializers - setSerializersFrom(headOps.get(0).getId(), source.getId()); - setSerializersFrom(tailOps.get(0).getId(), sink.getId()); - } + virtualSelectNodes.put(virtualId, + new Tuple2>(originalId, selectedNames)); + } - } + /** + * Adds a new virtual node that is used to connect a downstream vertex to an input with a certain + * partitioning. + * + * When adding an edge from the virtual node to a downstream node the connection will be made + * to the original node, but with the partitioning given here. + * + * @param originalId ID of the node that should be connected to. + * @param virtualId ID of the virtual node. + * @param partitioner The partitioner + */ + public void addVirtualPartitionNode(Integer originalId, Integer virtualId, StreamPartitioner partitioner) { + if (virtuaPartitionNodes.containsKey(virtualId)) { + throw new IllegalStateException("Already has virtual partition node with id " + virtualId); } + virtuaPartitionNodes.put(virtualId, + new Tuple2>(originalId, partitioner)); } - private Tuple2 createItSourceAndSink(StreamLoop loop, int c) { - StreamNode source = addNode(-1 * streamNodes.size(), StreamIterationHead.class, null, null); - sources.add(source.getId()); - - StreamNode sink = addNode(-1 * streamNodes.size(), StreamIterationTail.class, null, null); - - source.setOperatorName("IterationSource-" + loop.getID() + "_" + c); - sink.setOperatorName("IterationSink-" + loop.getID() + "_" + c); - vertexIDtoBrokerID.put(source.getId(), loop.getID() + "_" + c); - vertexIDtoBrokerID.put(sink.getId(), loop.getID() + "_" + c); - vertexIDtoLoop.put(source.getId(), loop); - vertexIDtoLoop.put(sink.getId(), loop); - loop.addSourceSinkPair(source, sink); + public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID, int typeNumber) { + addEdgeInternal(upStreamVertexID, + downStreamVertexID, + typeNumber, + null, + Lists.newArrayList()); - return new Tuple2(source, sink); } - protected StreamNode addNode(Integer vertexID, Class vertexClass, - StreamOperator operatorObject, String operatorName) { + private void addEdgeInternal(Integer upStreamVertexID, + Integer downStreamVertexID, + int typeNumber, + StreamPartitioner partitioner, + List outputNames) { - StreamNode vertex = new StreamNode(environemnt, vertexID, operatorObject, operatorName, - new ArrayList>(), vertexClass); - streamNodes.put(vertexID, vertex); + if (virtualSelectNodes.containsKey(upStreamVertexID)) { + int virtualId = upStreamVertexID; + upStreamVertexID = virtualSelectNodes.get(virtualId).f0; + if (outputNames.isEmpty()) { + // selections that happen downstream override earlier selections + outputNames = virtualSelectNodes.get(virtualId).f1; + } + addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames); + } else if (virtuaPartitionNodes.containsKey(upStreamVertexID)) { + int virtualId = upStreamVertexID; + upStreamVertexID = virtuaPartitionNodes.get(virtualId).f0; + if (partitioner == null) { + partitioner = virtuaPartitionNodes.get(virtualId).f1; + } + addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames); + } else { + StreamNode upstreamNode = getStreamNode(upStreamVertexID); + StreamNode downstreamNode = getStreamNode(downStreamVertexID); + + // If no partitioner was specified and the parallelism of upstream and downstream + // operator matches use forward partitioning, use rebalance otherwise. + if (partitioner == null && upstreamNode.getParallelism() == downstreamNode.getParallelism()) { + partitioner = new ForwardPartitioner(); + } else if (partitioner == null) { + partitioner = new RebalancePartitioner(); + } - return vertex; - } + if (partitioner instanceof ForwardPartitioner) { + if (upstreamNode.getParallelism() != downstreamNode.getParallelism()) { + throw new UnsupportedOperationException("Forward partitioning does not allow " + + "change of parallelism. Upstream operation: " + upstreamNode + " parallelism: " + upstreamNode.getParallelism() + + ", downstream operation: " + downstreamNode + " parallelism: " + downstreamNode.getParallelism() + + " You must use another partitioning strategy, such as broadcast, rebalance, shuffle or global."); + } + } - public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID, - StreamPartitioner partitionerObject, int typeNumber, List outputNames) { + StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner); - StreamEdge edge = new StreamEdge(getStreamNode(upStreamVertexID), - getStreamNode(downStreamVertexID), typeNumber, outputNames, partitionerObject); - getStreamNode(edge.getSourceId()).addOutEdge(edge); - getStreamNode(edge.getTargetId()).addInEdge(edge); + getStreamNode(edge.getSourceId()).addOutEdge(edge); + getStreamNode(edge.getTargetId()).addInEdge(edge); + } } public void addOutputSelector(Integer vertexID, OutputSelector outputSelector) { - getStreamNode(vertexID).addOutputSelector(outputSelector); + if (virtuaPartitionNodes.containsKey(vertexID)) { + addOutputSelector(virtuaPartitionNodes.get(vertexID).f0, outputSelector); + } else if (virtualSelectNodes.containsKey(vertexID)) { + addOutputSelector(virtualSelectNodes.get(vertexID).f0, outputSelector); + } else { + getStreamNode(vertexID).addOutputSelector(outputSelector); - if (LOG.isDebugEnabled()) { - LOG.debug("Outputselector set for {}", vertexID); + if (LOG.isDebugEnabled()) { + LOG.debug("Outputselector set for {}", vertexID); + } } } public void setParallelism(Integer vertexID, int parallelism) { - getStreamNode(vertexID).setParallelism(parallelism); + if (getStreamNode(vertexID) != null) { + getStreamNode(vertexID).setParallelism(parallelism); + } } public void setKey(Integer vertexID, KeySelector key) { @@ -437,17 +368,19 @@ public void setKey(Integer vertexID, KeySelector key) { } public void setBufferTimeout(Integer vertexID, long bufferTimeout) { - getStreamNode(vertexID).setBufferTimeout(bufferTimeout); + if (getStreamNode(vertexID) != null) { + getStreamNode(vertexID).setBufferTimeout(bufferTimeout); + } } - private void setSerializers(Integer vertexID, TypeSerializer in1, TypeSerializer in2, TypeSerializer out) { + public void setSerializers(Integer vertexID, TypeSerializer in1, TypeSerializer in2, TypeSerializer out) { StreamNode vertex = getStreamNode(vertexID); vertex.setSerializerIn1(in1); vertex.setSerializerIn2(in2); vertex.setSerializerOut(out); } - private void setSerializersFrom(Integer from, Integer to) { + public void setSerializersFrom(Integer from, Integer to) { StreamNode fromVertex = getStreamNode(from); StreamNode toVertex = getStreamNode(to); @@ -469,6 +402,10 @@ public void setInputFormat(Integer vertexID, InputFormat inputFormat) { public void setResourceStrategy(Integer vertexID, ResourceStrategy strategy) { StreamNode node = getStreamNode(vertexID); + if (node == null) { + return; + } + switch (strategy) { case ISOLATE: node.isolateSlot(); @@ -506,6 +443,11 @@ public Collection getSourceIDs() { return sources; } + + public Collection getSinkIDs() { + return sinks; + } + public Collection getStreamNodes() { return streamNodes.values(); } @@ -519,20 +461,44 @@ public Set>> getOperators() { return operatorSet; } - public Collection getStreamLoops() { - return streamLoops.values(); + public String getBrokerID(Integer vertexID) { + return vertexIDtoBrokerID.get(vertexID); } - public Integer getLoopID(Integer vertexID) { - return vertexIDtoLoop.get(vertexID).getID(); + public long getLoopTimeout(Integer vertexID) { + return vertexIDtoLoopTimeout.get(vertexID); } - public String getBrokerID(Integer vertexID) { - return vertexIDtoBrokerID.get(vertexID); + public Tuple2 createIterationSourceAndSink(int loopId, int sourceId, int sinkId, long timeout, int parallelism) { + + StreamNode source = this.addNode(sourceId, + StreamIterationHead.class, + null, + null); + sources.add(source.getId()); + setParallelism(source.getId(), parallelism); + + StreamNode sink = this.addNode(sinkId, + StreamIterationTail.class, + null, + null); + sinks.add(sink.getId()); + setParallelism(sink.getId(), parallelism); + + iterationSourceSinkPairs.add(new Tuple2(source, sink)); + + source.setOperatorName("IterationSource-" + loopId); + sink.setOperatorName("IterationSink-" + loopId); + this.vertexIDtoBrokerID.put(source.getId(), "broker-" + loopId); + this.vertexIDtoBrokerID.put(sink.getId(), "broker-" + loopId); + this.vertexIDtoLoopTimeout.put(source.getId(), timeout); + this.vertexIDtoLoopTimeout.put(sink.getId(), timeout); + + return new Tuple2(source, sink); } - public long getLoopTimeout(Integer vertexID) { - return vertexIDtoLoop.get(vertexID).getTimeout(); + public Set> getIterationSourceSinkPairs() { + return iterationSourceSinkPairs; } protected void removeEdge(StreamEdge edge) { @@ -570,7 +536,6 @@ public JobGraph getJobGraph() { * name of the jobGraph */ public JobGraph getJobGraph(String jobGraphName) { - finalizeLoops(); // temporarily forbid checkpointing for iterative jobs if (isIterative() && isCheckpointingEnabled() && !forceCheckpoint) { throw new UnsupportedOperationException( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java new file mode 100644 index 0000000000000..6df8cb5c80426 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java @@ -0,0 +1,530 @@ +/** + * 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.streaming.api.graph; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.FileSourceFunction; +import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation; +import org.apache.flink.streaming.api.transformations.FeedbackTransformation; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.api.transformations.SelectTransformation; +import org.apache.flink.streaming.api.transformations.SinkTransformation; +import org.apache.flink.streaming.api.transformations.SourceTransformation; +import org.apache.flink.streaming.api.transformations.SplitTransformation; +import org.apache.flink.streaming.api.transformations.StreamTransformation; +import org.apache.flink.streaming.api.transformations.TwoInputTransformation; +import org.apache.flink.streaming.api.transformations.UnionTransformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * A generator that generates a {@link StreamGraph} from a graph of + * {@link StreamTransformation StreamTransformations}. + * + *

    + * This traverses the tree of {@code StreamTransformations} starting from the sinks. At each + * we transformation recursively transform the inputs, then create a node in the {@code StreamGraph} + * and add edges from the input Nodes to our newly created node. The transformation methods + * return the IDs of the nodes in the StreamGraph that represent the input transformation. Several + * IDs can be returned to be able to deal with feedback transformations and unions. + * + *

    + * Partitioning, split/select and union don't create actual nodes in the {@code StreamGraph}. For + * these, we create a virtual node in the {@code StreamGraph} that holds the specific property, i.e. + * partitioning, selector and so on. When an edge is created from a virtual node to a downstream + * node the {@code StreamGraph} resolved the id of the original node and creates an edge + * in the graph with the desired property. For example, if you have this graph: + * + *

    + *     Map-1 -> HashPartition-2 -> Map-3
    + * 
    + * + * where the numbers represent transformation IDs. We first recurse all the way down. {@code Map-1} + * is transformed, i.e. we create a {@code StreamNode} with ID 1. Then we transform the + * {@code HashPartition}, for this, we create virtual node of ID 4 that holds the property + * {@code HashPartition}. This transformation returns the ID 4. Then we transform the {@code Map-3}. + * We add the edge {@code 4 -> 3}. The {@code StreamGraph} resolved the actual node with ID 1 and + * creates and edge {@code 1 -> 3} with the property HashPartition. + */ +public class StreamGraphGenerator { + + private static final Logger LOG = LoggerFactory.getLogger(StreamGraphGenerator.class); + + // The StreamGraph that is being built, this is initialized at the beginning. + private StreamGraph streamGraph; + + private final StreamExecutionEnvironment env; + + // This is used to assign a unique ID to iteration source/sink + protected static Integer iterationIdCounter = 0; + public static int getNewIterationNodeId() { + iterationIdCounter--; + return iterationIdCounter; + } + + // Keep track of which Transforms we have already transformed, this is necessary because + // we have loops, i.e. feedback edges. + private Map, Collection> alreadyTransformed; + + + /** + * Private constructor. The generator should only be invoked using {@link #generate}. + */ + private StreamGraphGenerator(StreamExecutionEnvironment env) { + this.streamGraph = new StreamGraph(env); + this.streamGraph.setChaining(env.isChainingEnabled()); + if (env.getCheckpointInterval() > 0) { + this.streamGraph.setCheckpointingEnabled(true); + this.streamGraph.setCheckpointingInterval(env.getCheckpointInterval()); + this.streamGraph.setCheckpointingMode(env.getCheckpointingMode()); + } + this.streamGraph.setStateHandleProvider(env.getStateHandleProvider()); + if (env.isForceCheckpointing()) { + this.streamGraph.forceCheckpoint(); + } + this.env = env; + this.alreadyTransformed = Maps.newHashMap(); + } + + /** + * Generates a {@code StreamGraph} by traversing the graph of {@code StreamTransformations} + * starting from the given transformations. + * + * @param env The {@code StreamExecutionEnvironment} that is used to set some parameters of the + * job + * @param transformations The transformations starting from which to transform the graph + * + * @return The generated {@code StreamGraph} + */ + public static StreamGraph generate(StreamExecutionEnvironment env, List> transformations) { + return new StreamGraphGenerator(env).generateInternal(transformations); + } + + /** + * This starts the actual transformation, beginning from the sinks. + */ + private StreamGraph generateInternal(List> transformations) { + for (StreamTransformation transformation: transformations) { + transform(transformation); + } + return streamGraph; + } + + /** + * Transforms one {@code StreamTransformation}. + * + *

    + * This checks whether we already transformed it and exits early in that case. If not it + * delegates to one of the transformation specific methods. + */ + private Collection transform(StreamTransformation transform) { + + if (alreadyTransformed.containsKey(transform)) { + return alreadyTransformed.get(transform); + } + + LOG.debug("Transforming " + transform); + + // call at least once to trigger exceptions about MissingTypeInfo + transform.getOutputType(); + + Collection transformedIds; + if (transform instanceof OneInputTransformation) { + transformedIds = transformOnInputTransform((OneInputTransformation) transform); + } else if (transform instanceof TwoInputTransformation) { + transformedIds = transformTwoInputTransform((TwoInputTransformation) transform); + } else if (transform instanceof SourceTransformation) { + transformedIds = transformSource((SourceTransformation) transform); + } else if (transform instanceof SinkTransformation) { + transformedIds = transformSink((SinkTransformation) transform); + } else if (transform instanceof UnionTransformation) { + transformedIds = transformUnion((UnionTransformation) transform); + } else if (transform instanceof SplitTransformation) { + transformedIds = transformSplit((SplitTransformation) transform); + } else if (transform instanceof SelectTransformation) { + transformedIds = transformSelect((SelectTransformation) transform); + } else if (transform instanceof FeedbackTransformation) { + transformedIds = transformFeedback((FeedbackTransformation) transform); + } else if (transform instanceof CoFeedbackTransformation) { + transformedIds = transformCoFeedback((CoFeedbackTransformation) transform); + } else if (transform instanceof PartitionTransformation) { + transformedIds = transformPartition((PartitionTransformation) transform); + } else { + throw new IllegalStateException("Unknown transformation: " + transform); + } + + // need this check because the iterate transformation adds itself before + // transforming the feedback edges + if (!alreadyTransformed.containsKey(transform)) { + alreadyTransformed.put(transform, transformedIds); + } + + if (transform.getBufferTimeout() > 0) { + streamGraph.setBufferTimeout(transform.getId(), transform.getBufferTimeout()); + } + if (transform.getResourceStrategy() != StreamGraph.ResourceStrategy.DEFAULT) { + streamGraph.setResourceStrategy(transform.getId(), transform.getResourceStrategy()); + } + + return transformedIds; + } + + /** + * Transforms a {@code UnionTransformation}. + * + *

    + * This is easy, we only have to transform the inputs and return all the IDs in a list so + * that downstream operations can connect to all upstream nodes. + */ + private Collection transformUnion(UnionTransformation union) { + List> inputs = union.getInputs(); + List resultIds = Lists.newArrayList(); + + for (StreamTransformation input: inputs) { + resultIds.addAll(transform(input)); + } + + return resultIds; + } + + /** + * Transforms a {@code PartitionTransformation}. + * + *

    + * For this we create a virtual node in the {@code StreamGraph} that holds the partition + * property. @see StreamGraphGenerator + */ + private Collection transformPartition(PartitionTransformation partition) { + StreamTransformation input = partition.getInput(); + List resultIds = Lists.newArrayList(); + + Collection transformedIds = transform(input); + for (Integer transformedId: transformedIds) { + int virtualId = StreamTransformation.getNewNodeId(); + streamGraph.addVirtualPartitionNode(transformedId, virtualId, partition.getPartitioner()); + resultIds.add(virtualId); + } + + return resultIds; + } + + /** + * Transforms a {@code SplitTransformation}. + * + *

    + * We add the output selector to previously transformed nodes. + */ + private Collection transformSplit(SplitTransformation split) { + + StreamTransformation input = split.getInput(); + Collection resultIds = transform(input); + + // the recursive transform call might have transformed this already + if (alreadyTransformed.containsKey(split)) { + return alreadyTransformed.get(split); + } + + for (int inputId : resultIds) { + streamGraph.addOutputSelector(inputId, split.getOutputSelector()); + } + + + return resultIds; + } + + /** + * Transforms a {@code SelectTransformation}. + * + *

    + * For this we create a virtual node in the {@code StreamGraph} holds the selected names. + * @see org.apache.flink.streaming.api.graph.StreamGraphGenerator + */ + private Collection transformSelect(SelectTransformation select) { + StreamTransformation input = select.getInput(); + Collection resultIds = transform(input); + + + // the recursive transform might have already transformed this + if (alreadyTransformed.containsKey(select)) { + return alreadyTransformed.get(select); + } + + List virtualResultIds = Lists.newArrayList(); + + for (int inputId : resultIds) { + int virtualId = StreamTransformation.getNewNodeId(); + streamGraph.addVirtualSelectNode(inputId, virtualId, select.getSelectedNames()); + virtualResultIds.add(virtualId); + } + return virtualResultIds; + } + + /** + * Transforms a {@code FeedbackTransformation}. + * + *

    + * This will recursively transform the input and the feedback edges. We return the concatenation + * of the input IDs and the feedback IDs so that downstream operations can be wired to both. + * + *

    + * This is responsible for creating the IterationSource and IterationSink which + * are used to feed back the elements. + */ + private Collection transformFeedback(FeedbackTransformation iterate) { + + if (iterate.getFeedbackEdges().size() <= 0) { + throw new IllegalStateException("Iteration " + iterate + " does not have any feedback edges."); + } + + StreamTransformation input = iterate.getInput(); + List resultIds = Lists.newArrayList(); + + // first transform the input stream(s) and store the result IDs + resultIds.addAll(transform(input)); + + // the recursive transform might have already transformed this + if (alreadyTransformed.containsKey(iterate)) { + return alreadyTransformed.get(iterate); + } + + + // create the fake iteration source/sink pair + Tuple2 itSourceAndSink = streamGraph.createIterationSourceAndSink( + iterate.getId(), + getNewIterationNodeId(), + getNewIterationNodeId(), + iterate.getWaitTime(), + iterate.getParallelism()); + + StreamNode itSource = itSourceAndSink.f0; + StreamNode itSink = itSourceAndSink.f1; + + // We set the proper serializers for the sink/source + streamGraph.setSerializers(itSource.getId(), null, null, iterate.getOutputType().createSerializer(env.getConfig())); + streamGraph.setSerializers(itSink.getId(), iterate.getOutputType().createSerializer(env.getConfig()), null, null); + + // also add the feedback source ID to the result IDs, so that downstream operators will + // add both as input + resultIds.add(itSource.getId()); + + // at the iterate to the already-seen-set with the result IDs, so that we can transform + // the feedback edges and let them stop when encountering the iterate node + alreadyTransformed.put(iterate, resultIds); + + for (StreamTransformation feedbackEdge : iterate.getFeedbackEdges()) { + Collection feedbackIds = transform(feedbackEdge); + for (Integer feedbackId: feedbackIds) { + streamGraph.addEdge(feedbackId, + itSink.getId(), + 0 + ); + } + } + + return resultIds; + } + + /** + * Transforms a {@code CoFeedbackTransformation}. + * + *

    + * This will only transform feedback edges, the result of this transform will be wired + * to the second input of a Co-Transform. The original input is wired directly to the first + * input of the downstream Co-Transform. + * + *

    + * This is responsible for creating the IterationSource and IterationSink which + * are used to feed back the elements. + */ + private Collection transformCoFeedback(CoFeedbackTransformation coIterate) { + + // For Co-Iteration we don't need to transform the input and wire the input to the + // head operator by returning the input IDs, the input is directly wired to the left + // input of the co-operation. This transform only needs to return the ids of the feedback + // edges, since they need to be wired to the second input of the co-operation. + + // create the fake iteration source/sink pair + Tuple2 itSourceAndSink = streamGraph.createIterationSourceAndSink( + coIterate.getId(), + getNewIterationNodeId(), + getNewIterationNodeId(), + coIterate.getWaitTime(), + coIterate.getParallelism()); + + StreamNode itSource = itSourceAndSink.f0; + StreamNode itSink = itSourceAndSink.f1; + + // We set the proper serializers for the sink/source + streamGraph.setSerializers(itSource.getId(), null, null, coIterate.getOutputType().createSerializer(env.getConfig())); + streamGraph.setSerializers(itSink.getId(), coIterate.getOutputType().createSerializer(env.getConfig()), null, null); + + Collection resultIds = Collections.singleton(itSource.getId()); + + // at the iterate to the already-seen-set with the result IDs, so that we can transform + // the feedback edges and let them stop when encountering the iterate node + alreadyTransformed.put(coIterate, resultIds); + + for (StreamTransformation feedbackEdge : coIterate.getFeedbackEdges()) { + Collection feedbackIds = transform(feedbackEdge); + for (Integer feedbackId: feedbackIds) { + streamGraph.addEdge(feedbackId, + itSink.getId(), + 0 + ); + } + } + + return Collections.singleton(itSource.getId()); + } + + /** + * Transforms a {@code SourceTransformation}. + */ + private Collection transformSource(SourceTransformation source) { + streamGraph.addSource(source.getId(), + source.getOperator(), + null, + source.getOutputType(), + "Source: " + source.getName()); + if (source.getOperator().getUserFunction() instanceof FileSourceFunction) { + FileSourceFunction fs = (FileSourceFunction) source.getOperator().getUserFunction(); + streamGraph.setInputFormat(source.getId(), fs.getFormat()); + } + streamGraph.setParallelism(source.getId(), source.getParallelism()); + return Collections.singleton(source.getId()); + } + + /** + * Transforms a {@code SourceTransformation}. + */ + private Collection transformSink(SinkTransformation sink) { + + Collection inputIds = transform(sink.getInput()); + + streamGraph.addSink(sink.getId(), + sink.getOperator(), + sink.getInput().getOutputType(), + null, + "Sink: " + sink.getName()); + + streamGraph.setParallelism(sink.getId(), sink.getParallelism()); + + for (Integer inputId: inputIds) { + streamGraph.addEdge(inputId, + sink.getId(), + 0 + ); + } + + + if (sink.getStateKeySelector() != null) { + streamGraph.setKey(sink.getId(), sink.getStateKeySelector()); + } + + return Collections.emptyList(); + } + + /** + * Transforms a {@code OneInputTransformation}. + * + *

    + * This recusively transforms the inputs, creates a new {@code StreamNode} in the graph and + * wired the inputs to this new node. + */ + private Collection transformOnInputTransform(OneInputTransformation transform) { + + Collection inputIds = transform(transform.getInput()); + + // the recursive call might have already transformed this + if (alreadyTransformed.containsKey(transform)) { + return alreadyTransformed.get(transform); + } + + streamGraph.addOperator(transform.getId(), + transform.getOperator(), + transform.getInputType(), + transform.getOutputType(), + transform.getName()); + + if (transform.getStateKeySelector() != null) { + streamGraph.setKey(transform.getId(), transform.getStateKeySelector()); + } + + streamGraph.setParallelism(transform.getId(), transform.getParallelism()); + + for (Integer inputId: inputIds) { + streamGraph.addEdge(inputId, transform.getId(), 0); + } + + return Collections.singleton(transform.getId()); + } + + /** + * Transforms a {@code TwoInputTransformation}. + * + *

    + * This recusively transforms the inputs, creates a new {@code StreamNode} in the graph and + * wired the inputs to this new node. + */ + private Collection transformTwoInputTransform(TwoInputTransformation transform) { + + Collection inputIds1 = transform(transform.getInput1()); + Collection inputIds2 = transform(transform.getInput2()); + + // the recursive call might have already transformed this + if (alreadyTransformed.containsKey(transform)) { + return alreadyTransformed.get(transform); + } + + streamGraph.addCoOperator( + transform.getId(), + transform.getOperator(), + transform.getInputType1(), + transform.getInputType2(), + transform.getOutputType(), + transform.getName()); + + streamGraph.setParallelism(transform.getId(), transform.getParallelism()); + + for (Integer inputId: inputIds1) { + streamGraph.addEdge(inputId, + transform.getId(), + 1 + ); + } + + for (Integer inputId: inputIds2) { + streamGraph.addEdge(inputId, + transform.getId(), + 2 + ); + } + + return Collections.singleton(transform.getId()); + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java deleted file mode 100644 index ba987ef52acf1..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.streaming.api.graph; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; - -/** - * Object for representing loops in streaming programs. - * - */ -public class StreamLoop { - - private Integer loopID; - - private List headOperators = new ArrayList(); - private List tailOperators = new ArrayList(); - private List> tailPartitioners = new ArrayList>(); - private List> tailSelectedNames = new ArrayList>(); - - private boolean coIteration = false; - private TypeInformation feedbackType = null; - - private long timeout; - private boolean tailPartitioning = false; - - private List> sourcesAndSinks = new ArrayList>(); - - public StreamLoop(Integer loopID, long timeout, TypeInformation feedbackType) { - this.loopID = loopID; - this.timeout = timeout; - if (feedbackType != null) { - this.feedbackType = feedbackType; - coIteration = true; - tailPartitioning = true; - } - } - - public Integer getID() { - return loopID; - } - - public long getTimeout() { - return timeout; - } - - public boolean isCoIteration() { - return coIteration; - } - - public TypeInformation getFeedbackType() { - return feedbackType; - } - - public void addSourceSinkPair(StreamNode source, StreamNode sink) { - this.sourcesAndSinks.add(new Tuple2(source, sink)); - } - - public List> getSourceSinkPairs() { - return this.sourcesAndSinks; - } - - public void addHeadOperator(StreamNode head) { - this.headOperators.add(head); - } - - public void addTailOperator(StreamNode tail, StreamPartitioner partitioner, - List selectedNames) { - this.tailOperators.add(tail); - this.tailPartitioners.add(partitioner); - this.tailSelectedNames.add(selectedNames); - } - - public void applyTailPartitioning() { - this.tailPartitioning = true; - } - - public boolean keepsPartitioning() { - return tailPartitioning; - } - - public List getHeads() { - return headOperators; - } - - public List getTails() { - return tailOperators; - } - - public List> getTailPartitioners() { - return tailPartitioners; - } - - public List> getTailSelectedNames() { - return tailSelectedNames; - } - - @Override - public String toString() { - return "ID: " + loopID + "\n" + "Head: " + headOperators + "\n" + "Tail: " + tailOperators - + "\n" + "TP: " + tailPartitioners + "\n" + "TSN: " + tailSelectedNames; - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java index 62e2d839f52b8..9110cd334a53b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java @@ -125,7 +125,11 @@ public Integer getId() { } public int getParallelism() { - return parallelism != null ? parallelism : env.getParallelism(); + if (parallelism == -1) { + return env.getParallelism(); + } else { + return parallelism; + } } public void setParallelism(Integer parallelism) { @@ -218,7 +222,7 @@ public void isolateSlot() { @Override public String toString() { - return operatorName + id; + return operatorName + "-" + id; } public KeySelector getStatePartitioner() { @@ -228,4 +232,23 @@ public String toString() { public void setStatePartitioner(KeySelector statePartitioner) { this.statePartitioner = statePartitioner; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StreamNode that = (StreamNode) o; + + return id.equals(that.id); + } + + @Override + public int hashCode() { + return id.hashCode(); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 5280fb27c638d..314d1b3596673 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -44,8 +44,8 @@ import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; -import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner.PartitioningStrategy; import org.apache.flink.streaming.runtime.tasks.StreamIterationHead; import org.apache.flink.streaming.runtime.tasks.StreamIterationTail; import org.apache.flink.util.InstantiationUtil; @@ -294,11 +294,6 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, List allOutputs = new ArrayList(chainableOutputs); allOutputs.addAll(nonChainableOutputs); - for (StreamEdge output : allOutputs) { - config.setSelectedNames(output.getTargetId(), - streamGraph.getStreamEdge(vertexID, output.getTargetId()).getSelectedNames()); - } - vertexConfigs.put(vertexID, config); } @@ -316,7 +311,7 @@ private void connect(Integer headOfChain, StreamEdge edge) { downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1); StreamPartitioner partitioner = edge.getPartitioner(); - if (partitioner.getStrategy() == PartitioningStrategy.FORWARD) { + if (partitioner instanceof ForwardPartitioner) { downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE); } else { downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.ALL_TO_ALL); @@ -345,7 +340,7 @@ private boolean isChainable(StreamEdge edge) { && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS || headOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS) - && (edge.getPartitioner().getStrategy() == PartitioningStrategy.FORWARD || downStreamVertex + && (edge.getPartitioner() instanceof ForwardPartitioner || downStreamVertex .getParallelism() == 1) && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() && (streamGraph.isChainingEnabled() || @@ -370,21 +365,19 @@ private void setSlotSharing() { } } - for (StreamLoop loop : streamGraph.getStreamLoops()) { - for (Tuple2 pair : loop.getSourceSinkPairs()) { - - CoLocationGroup ccg = new CoLocationGroup(); - - JobVertex source = jobVertices.get(pair.f0.getId()); - JobVertex sink = jobVertices.get(pair.f1.getId()); - - ccg.addVertex(source); - ccg.addVertex(sink); - source.updateCoLocationGroup(ccg); - sink.updateCoLocationGroup(ccg); - } + for (Tuple2 pair : streamGraph.getIterationSourceSinkPairs()) { + CoLocationGroup ccg = new CoLocationGroup(); + + JobVertex source = jobVertices.get(pair.f0.getId()); + JobVertex sink = jobVertices.get(pair.f1.getId()); + + ccg.addVertex(source); + ccg.addVertex(sink); + source.updateCoLocationGroup(ccg); + sink.updateCoLocationGroup(ccg); } + } private void configureCheckpointing() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java index dce7684c8eda3..cbd2a40df129e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java @@ -28,7 +28,6 @@ import org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer; import org.apache.flink.streaming.api.operators.windowing.WindowFlattener; import org.apache.flink.streaming.api.operators.windowing.WindowMerger; -import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; public class WindowingOptimizer { @@ -64,8 +63,7 @@ private static void removeMergeBeforeFlatten(StreamGraph streamGraph) { StreamNode mergeInput = input.getInEdges().get(0).getSourceVertex(); // We connect the merge input to the flattener directly - streamGraph.addEdge(mergeInput.getId(), flattenerId, - new RebalancePartitioner(true), 0, new ArrayList()); + streamGraph.addEdge(mergeInput.getId(), flattenerId, 0); // If the merger is only connected to the flattener we delete it // completely, otherwise we only remove the edge @@ -107,8 +105,7 @@ private static void setDiscretizerReuse(StreamGraph streamGraph) { for (StreamEdge edge1 : discretizer.getInEdges()) { for (StreamEdge edge2 : candidate.f1.get(0).getInEdges()) { - if (edge1.getPartitioner().getStrategy() != edge2.getPartitioner() - .getStrategy()) { + if (edge1.getPartitioner().getClass() != edge2.getPartitioner().getClass()) { partitionersMatch = false; } } @@ -155,8 +152,7 @@ private static void replaceDiscretizer(StreamGraph streamGraph, Integer toReplac for (int i = 0; i < numOutputs; i++) { StreamEdge outEdge = outEdges.get(i); - streamGraph.addEdge(replaceWithId, outEdge.getTargetId(), outEdge.getPartitioner(), 0, - new ArrayList()); + streamGraph.addEdge(replaceWithId, outEdge.getTargetId(), 0); } // Remove the other discretizer diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java new file mode 100644 index 0000000000000..67ccbd6ce9554 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java @@ -0,0 +1,122 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * This represents a feedback point in a topology. The type of the feedback elements must not match + * the type of the upstream {@code StreamTransformation} because the only allowed operations + * after a {@code CoFeedbackTransformation} are + * {@link org.apache.flink.streaming.api.transformations.TwoInputTransformation TwoInputTransformations}. + * The upstream {@code StreamTransformation} will be connected to the first input of the Co-Transform + * while the feedback edges will be connected to the second input. + * + *

    + * Both the partitioning of the input and the feedback edges is preserved. They can also have + * differing partitioning strategies. This requires, however, that the parallelism of the feedback + * {@code StreamTransformations} must match the parallelism of the input + * {@code StreamTransformation}. + * + *

    + * The upstream {@code StreamTransformation} is not wired to this {@code CoFeedbackTransformation}. + * It is instead directly wired to the {@code TwoInputTransformation} after this + * {@code CoFeedbackTransformation}. + * + *

    + * This is different from Iterations in batch processing. + * @see org.apache.flink.streaming.api.transformations.FeedbackTransformation + * + * @param The type of the feedback elements. + * + */ +public class CoFeedbackTransformation extends StreamTransformation { + + private final List> feedbackEdges; + + private final Long waitTime; + + /** + * Creates a new {@code CoFeedbackTransformation} from the given input. + * + * @param parallelism The parallelism of the upstream {@code StreamTransformatino} and the + * feedback edges. + * @param feedbackType The type of the feedback edges + * @param waitTime The wait time of the feedback operator. After the time expires + * the operation will close and not receive any more feedback elements. + */ + public CoFeedbackTransformation(int parallelism, + TypeInformation feedbackType, + Long waitTime) { + super("CoFeedback", feedbackType, parallelism); + this.waitTime = waitTime; + this.feedbackEdges = Lists.newArrayList(); + } + + /** + * Adds a feedback edge. The parallelism of the {@code StreamTransformation} must match + * the parallelism of the input {@code StreamTransformation} of the upstream + * {@code StreamTransformation}. + * + * @param transform The new feedback {@code StreamTransformation}. + */ + public void addFeedbackEdge(StreamTransformation transform) { + + if (transform.getParallelism() != this.getParallelism()) { + throw new UnsupportedOperationException( + "Parallelism of the feedback stream must match the parallelism of the original" + + " stream. Parallelism of original stream: " + this.getParallelism() + + "; parallelism of feedback stream: " + transform.getParallelism()); + } + + feedbackEdges.add(transform); + } + + /** + * Returns the list of feedback {@code StreamTransformations}. + */ + public List> getFeedbackEdges() { + return feedbackEdges; + } + + /** + * Returns the wait time. This is the amount of time that the feedback operator keeps listening + * for feedback elements. Once the time expires the operation will close and will not receive + * further elements. + */ + public Long getWaitTime() { + return waitTime; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation."); + } + + @Override + public Collection> getTransitivePredecessors() { + return Collections.>singleton(this); + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java new file mode 100644 index 0000000000000..11a2f33dd1e6f --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java @@ -0,0 +1,124 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; +import java.util.List; + +/** + * This represents a feedback point in a topology. + * + *

    + * This is different from how iterations work in batch processing. Once a feedback point is defined + * you can connect one or several {@code StreamTransformations} as a feedback edges. Operations + * downstream from the feedback point will receive elements from the input of this feedback point + * and from the feedback edges. + * + *

    + * Both the partitioning of the input and the feedback edges is preserved. They can also have + * differing partitioning strategies. This requires, however, that the parallelism of the feedback + * {@code StreamTransformations} must match the parallelism of the input + * {@code StreamTransformation}. + * + *

    + * The type of the input {@code StreamTransformation} and the feedback {@code StreamTransformation} + * must match. + * + * @param The type of the input elements and the feedback elements. + */ +public class FeedbackTransformation extends StreamTransformation { + + private final StreamTransformation input; + + private final List> feedbackEdges; + + private final Long waitTime; + + /** + * Creates a new {@code FeedbackTransformation} from the given input. + * + * @param input The input {@code StreamTransformation} + * @param waitTime The wait time of the feedback operator. After the time expires + * the operation will close and not receive any more feedback elements. + */ + public FeedbackTransformation(StreamTransformation input, Long waitTime) { + super("Feedback", input.getOutputType(), input.getParallelism()); + this.input = input; + this.waitTime = waitTime; + this.feedbackEdges = Lists.newArrayList(); + } + + /** + * Returns the input {@code StreamTransformation} of this {@code FeedbackTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + /** + * Adds a feedback edge. The parallelism of the {@code StreamTransformation} must match + * the parallelism of the input {@code StreamTransformation} of this + * {@code FeedbackTransformation} + * + * @param transform The new feedback {@code StreamTransformation}. + */ + public void addFeedbackEdge(StreamTransformation transform) { + + if (transform.getParallelism() != this.getParallelism()) { + throw new UnsupportedOperationException( + "Parallelism of the feedback stream must match the parallelism of the original" + + " stream. Parallelism of original stream: " + this.getParallelism() + + "; parallelism of feedback stream: " + transform.getParallelism()); + } + + feedbackEdges.add(transform); + } + + /** + * Returns the list of feedback {@code StreamTransformations}. + */ + public List> getFeedbackEdges() { + return feedbackEdges; + } + + /** + * Returns the wait time. This is the amount of time that the feedback operator keeps listening + * for feedback elements. Once the time expires the operation will close and will not receive + * further elements. + */ + public Long getWaitTime() { + return waitTime; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation."); + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java new file mode 100644 index 0000000000000..945d8eb852ffc --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java @@ -0,0 +1,117 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; +import java.util.List; + +/** + * This Transformation represents the application of a + * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} to one input + * {@link org.apache.flink.streaming.api.transformations.StreamTransformation}. + * + * @param The type of the elements in the nput {@code StreamTransformation} + * @param The type of the elements that result from this {@code OneInputTransformation} + */ +public class OneInputTransformation extends StreamTransformation { + + private final StreamTransformation input; + + private KeySelector stateKeySelector; + + private final OneInputStreamOperator operator; + + /** + * Creates a new {@code OneInputTransformation} from the given input and operator. + * + * @param input The input {@code StreamTransformation} + * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log + * @param operator The {@code TwoInputStreamOperator} + * @param outputType The type of the elements produced by this {@code OneInputTransformation} + * @param parallelism The parallelism of this {@code OneInputTransformation} + */ + public OneInputTransformation( + StreamTransformation input, + String name, + OneInputStreamOperator operator, + TypeInformation outputType, + int parallelism) { + super(name, outputType, parallelism); + this.input = input; + this.operator = operator; + } + + /** + * Returns the input {@code StreamTransformation} of this {@code OneInputTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + /** + * Returns the {@code TypeInformation} for the elements of the input. + */ + public TypeInformation getInputType() { + return input.getOutputType(); + } + + /** + * Returns the {@code TwoInputStreamOperator} of this Transformation. + */ + public OneInputStreamOperator getOperator() { + return operator; + } + + /** + * Sets the {@link KeySelector} that must be used for partitioning keyed state of this operation. + * + * @param stateKeySelector The {@code KeySelector} to set + */ + public void setStateKeySelector(KeySelector stateKeySelector) { + this.stateKeySelector = stateKeySelector; + } + + /** + * Returns the {@code KeySelector} that must be used for partitioning keyed state in this + * Operation. + * + * @see #setStateKeySelector + */ + public KeySelector getStateKeySelector() { + return stateKeySelector; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + operator.setChainingStrategy(strategy); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java new file mode 100644 index 0000000000000..1165d5d6563f1 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java @@ -0,0 +1,80 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; + +import java.util.Collection; +import java.util.List; + +/** + * This transformation represents a change of partitioning of the input elements. + * + *

    + * This does not create a physical operation, it only affects how upstream operations are + * connected to downstream operations. + * + * @param The type of the elements that result from this {@code PartitionTransformation} + */ +public class PartitionTransformation extends StreamTransformation { + private final StreamTransformation input; + private final StreamPartitioner partitioner; + + /** + * Creates a new {@code PartitionTransformation} from the given input and + * {@link StreamPartitioner}. + * + * @param input The input {@code StreamTransformation} + * @param partitioner The {@code StreamPartitioner} + */ + public PartitionTransformation(StreamTransformation input, StreamPartitioner partitioner) { + super("Partition", input.getOutputType(), input.getParallelism()); + this.input = input; + this.partitioner = partitioner; + } + + /** + * Returns the input {@code StreamTransformation} of this {@code SinkTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + /** + * Returns the {@code StreamPartitioner} that must be used for partitioning the elements + * of the input {@code StreamTransformation}. + */ + public StreamPartitioner getPartitioner() { + return partitioner; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation."); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java new file mode 100644 index 0000000000000..92033bdfd8ecd --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java @@ -0,0 +1,83 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; +import java.util.List; + +/** + * This transformation represents a selection of only certain upstream elements. This must + * follow a {@link org.apache.flink.streaming.api.transformations.SplitTransformation} that + * splits elements into several logical streams with assigned names. + * + *

    + * This does not create a physical operation, it only affects how upstream operations are + * connected to downstream operations. + * + * @param The type of the elements that result from this {@code SelectTransformation} + */ +public class SelectTransformation extends StreamTransformation { + private final StreamTransformation input; + private List selectedNames; + + /** + * Creates a new {@coe SelectionTransformation} from the given input that only selects + * the streams with the selected names. + * + * @param input The input {@code StreamTransformation} + * @param selectedNames The names from the upstream {@code SplitTransformation} that this + * {@code SelectTransformation} selects. + */ + public SelectTransformation(StreamTransformation input, + List selectedNames) { + super("Select", input.getOutputType(), input.getParallelism()); + this.input = input; + this.selectedNames = selectedNames; + } + + /** + * Returns the input {@code StreamTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + /** + * Returns the names of the split streams that this {@code SelectTransformation} selects. + */ + public List getSelectedNames() { + return selectedNames; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Select Transformation."); + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java new file mode 100644 index 0000000000000..2a4e2d0f8e9f3 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java @@ -0,0 +1,106 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamSink; + +import java.util.Collection; +import java.util.List; + +/** + * This Transformation represents a Sink. + * + * @param The type of the elements in the input {@code SinkTransformation} + */ +public class SinkTransformation extends StreamTransformation { + + private final StreamTransformation input; + + private final StreamSink operator; + + // We need this because sinks can also have state that is partitioned by key + private KeySelector stateKeySelector; + + /** + * Creates a new {@code SinkTransformation} from the given input {@code StreamTransformation}. + * + * @param input The input {@code StreamTransformation} + * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log + * @param operator The sink operator + * @param parallelism The parallelism of this {@code SinkTransformation} + */ + public SinkTransformation( + StreamTransformation input, + String name, + StreamSink operator, + int parallelism) { + super(name, TypeExtractor.getForClass(Object.class), parallelism); + this.input = input; + this.operator = operator; + } + + /** + * Returns the input {@code StreamTransformation} of this {@code SinkTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + /** + * Returns the {@link StreamSink} that is the operator of this {@code SinkTransformation}. + */ + public StreamSink getOperator() { + return operator; + } + + /** + * Sets the {@link KeySelector} that must be used for partitioning keyed state of this Sink. + * + * @param stateKeySelector The {@code KeySelector} to set + */ + public void setStateKeySelector(KeySelector stateKeySelector) { + this.stateKeySelector = stateKeySelector; + } + + /** + * Returns the {@code KeySelector} that must be used for partitioning keyed state in this + * Sink. + * + * @see #setStateKeySelector + */ + public KeySelector getStateKeySelector() { + return stateKeySelector; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + operator.setChainingStrategy(strategy); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java new file mode 100644 index 0000000000000..c14c58cb17bb3 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java @@ -0,0 +1,70 @@ +/** + * 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.streaming.api.transformations; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamSource; + +import java.util.Collection; +import java.util.Collections; + +/** + * This represents a Source. This does not actually transform anything since it has no inputs but + * it is the root {@code StreamTransformation} of any topology. + * + * @param The type of the elements that this source produces + */ +public class SourceTransformation extends StreamTransformation { + + private final StreamSource operator; + + /** + * Creates a new {@code SourceTransformation} from the given operator. + * + * @param name The name of the {@code SourceTransformation}, this will be shown in Visualizations and the Log + * @param operator The {@code StreamSource} that is the operator of this Transformation + * @param outputType The type of the elements produced by this {@code SourceTransformation} + * @param parallelism The parallelism of this {@code SourceTransformation} + */ + public SourceTransformation( + String name, + StreamSource operator, + TypeInformation outputType, + int parallelism) { + super(name, outputType, parallelism); + this.operator = operator; + } + + /** + * Returns the {@code StreamSource}, the operator of this {@code SourceTransformation}. + */ + public StreamSource getOperator() { + return operator; + } + + @Override + public Collection> getTransitivePredecessors() { + return Collections.>singleton(this); + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + operator.setChainingStrategy(strategy); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java new file mode 100644 index 0000000000000..d392fd5bef22b --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java @@ -0,0 +1,84 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; +import java.util.List; + +/** + * This transformation represents a split of one + * {@link org.apache.flink.streaming.api.datastream.DataStream} into several {@code DataStreams} + * using an {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}. + * + *

    + * This does not create a physical operation, it only affects how upstream operations are + * connected to downstream operations. + * + * @param The type of the elements that result from this {@code SplitTransformation} + */ +public class SplitTransformation extends StreamTransformation { + + private final StreamTransformation input; + + private final OutputSelector outputSelector; + + /** + * Creates a new {@code SplitTransformation} from the given input and {@code OutputSelector}. + * + * @param input The input {@code StreamTransformation} + * @param outputSelector The output selector + */ + public SplitTransformation(StreamTransformation input, + OutputSelector outputSelector) { + super("Split", input.getOutputType(), input.getParallelism()); + this.input = input; + this.outputSelector = outputSelector; + } + + /** + * Returns the input {@code StreamTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + /** + * Returns the {@code OutputSelector} + */ + public OutputSelector getOutputSelector() { + return outputSelector; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation."); + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java new file mode 100644 index 0000000000000..dadcfa28907c4 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java @@ -0,0 +1,320 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.base.Preconditions; +import org.apache.flink.api.common.functions.InvalidTypesException; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.MissingTypeInfo; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; + +/** + * A {@code StreamTransformation} represents the operation that creates a + * {@link org.apache.flink.streaming.api.datastream.DataStream}. Every + * {@link org.apache.flink.streaming.api.datastream.DataStream} has an underlying + * {@code StreamTransformation} that is the origin of said DataStream. + * + *

    + * API operations such as {@link org.apache.flink.streaming.api.datastream.DataStream#map} create + * a tree of {@code StreamTransformation}s underneath. When the stream program is to be executed this + * graph is translated to a {@link StreamGraph} using + * {@link org.apache.flink.streaming.api.graph.StreamGraphGenerator}. + * + *

    + * A {@code StreamTransformation} does not necessarily correspond to a physical operation + * at runtime. Some operations are only logical concepts. Examples of this are union, + * split/select data stream, partitioning. + * + *

    + * The following graph of {@code StreamTransformations}: + * + *

    + *   Source              Source        
    + *      +                   +           
    + *      |                   |           
    + *      v                   v           
    + *  Rebalance          HashPartition    
    + *      +                   +           
    + *      |                   |           
    + *      |                   |           
    + *      +------>Union<------+           
    + *                +                     
    + *                |                     
    + *                v                     
    + *              Split                   
    + *                +                     
    + *                |                     
    + *                v                     
    + *              Select                  
    + *                +                     
    + *                v                     
    + *               Map                    
    + *                +                     
    + *                |                     
    + *                v                     
    + *              Sink 
    + * 
    + * + * Would result in this graph of operations at runtime: + * + *
    + *  Source              Source
    + *    +                   +
    + *    |                   |
    + *    |                   |
    + *    +------->Map<-------+
    + *              +
    + *              |
    + *              v
    + *             Sink
    + * 
    + * + * The information about partitioning, union, split/select end up being encoded in the edges + * that connect the sources to the map operation. + * + * @param The type of the elements that result from this {@code StreamTransformation} + */ +public abstract class StreamTransformation { + + // This is used to assign a unique ID to every StreamTransformation + protected static Integer idCounter = 0; + public static int getNewNodeId() { + idCounter++; + return idCounter; + } + + protected final int id; + + protected String name; + + protected TypeInformation outputType; + // This is used to handle MissingTypeInfo. As long as the outputType has not been queried + // it can still be changed using setOutputType(). Afterwards an exception is thrown when + // trying to change the output type. + protected boolean typeUsed; + + private int parallelism; + + protected long bufferTimeout = -1; + + protected StreamGraph.ResourceStrategy resourceStrategy = StreamGraph.ResourceStrategy.DEFAULT; + + /** + * Creates a new {@code StreamTransformation} with the given name, output type and parallelism. + * + * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log + * @param outputType The output type of this {@code StreamTransformation} + * @param parallelism The parallelism of this {@code StreamTransformation} + */ + public StreamTransformation(String name, TypeInformation outputType, int parallelism) { + this.id = getNewNodeId(); + this.name = Preconditions.checkNotNull(name); + this.outputType = outputType; + this.parallelism = parallelism; + } + + /** + * Returns the unique ID of this {@code StreamTransformation}. + */ + public int getId() { + return id; + } + + /** + * Changes the name of this {@code StreamTransformation}. + */ + public void setName(String name) { + this.name = name; + } + + /** + * Returns the name of this {@code StreamTransformation}. + */ + public String getName() { + return name; + } + + /** + * Returns the parallelism of this {@code StreamTransformation} + */ + public int getParallelism() { + return parallelism; + } + + /** + * Sets the parallelism of this {@code StreamTransformation} + * @param parallelism The new parallelism to set on this {@code StreamTransformation} + */ + public void setParallelism(int parallelism) { + Preconditions.checkArgument(parallelism > 0, "Parallelism must be bigger than zero."); + this.parallelism = parallelism; + } + + /** + * Tries to fill in the type information. Type information can be filled in + * later when the program uses a type hint. This method checks whether the + * type information has ever been accessed before and does not allow + * modifications if the type was accessed already. This ensures consistency + * by making sure different parts of the operation do not assume different + * type information. + * + * @param outputType The type information to fill in. + * + * @throws IllegalStateException Thrown, if the type information has been accessed before. + */ + public void setOutputType(TypeInformation outputType) { + if (typeUsed) { + throw new IllegalStateException( + "TypeInformation cannot be filled in for the type after it has been used. " + + "Please make sure that the type info hints are the first call after" + + " the transformation function, " + + "before any access to types or semantic properties, etc."); + } + this.outputType = outputType; + } + + /** + * Returns the output type of this {@code StreamTransformation} as a {@link TypeInformation}. Once + * this is used once the output type cannot be changed anymore using {@link #setOutputType}. + * + * @return The output type of this {@code StreamTransformation} + */ + public TypeInformation getOutputType() { + if (outputType instanceof MissingTypeInfo) { + MissingTypeInfo typeInfo = (MissingTypeInfo) this.outputType; + throw new InvalidTypesException( + "The return type of function '" + + typeInfo.getFunctionName() + + "' could not be determined automatically, due to type erasure. " + + "You can give type information hints by using the returns(...) " + + "method on the result of the transformation call, or by letting " + + "your function implement the 'ResultTypeQueryable' " + + "interface.", typeInfo.getTypeException()); + } + typeUsed = true; + return this.outputType; + } + + /** + * Sets the chaining strategy of this {@code StreamTransformation}. + */ + public abstract void setChainingStrategy(StreamOperator.ChainingStrategy strategy); + + /** + * Set the buffer timeout of this {@code StreamTransformation}. The timeout is used when + * sending elements over the network. The timeout specifies how long a network buffer + * should be kept waiting before sending. A higher timeout means that more elements will + * be sent in one buffer, this increases throughput. The latency, however, is negatively + * affected by a higher timeout. + */ + public void setBufferTimeout(long bufferTimeout) { + this.bufferTimeout = bufferTimeout; + } + + /** + * Returns the buffer timeout of this {@code StreamTransformation}. + * + *

    + * {@see #setBufferTimeout} + */ + public long getBufferTimeout() { + return bufferTimeout; + } + + /** + * Sets the {@link org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy} of this + * {@code StreamTransformation}. The resource strategy is used when scheduling operations on actual + * workers when transforming the StreamTopology to an + * {@link org.apache.flink.runtime.executiongraph.ExecutionGraph}. + */ + public void setResourceStrategy(StreamGraph.ResourceStrategy resourceStrategy) { + this.resourceStrategy = resourceStrategy; + } + + /** + * Returns the {@code ResourceStrategy} of this {@code StreamTransformation}. + * + *

    + * {@see #setResourceStrategy} + */ + public StreamGraph.ResourceStrategy getResourceStrategy() { + return resourceStrategy; + } + + /** + * Returns all transitive predecessor {@code StreamTransformation}s of this {@code StreamTransformation}. This + * is, for example, used when determining whether a feedback edge of an iteration + * actually has the iteration head as a predecessor. + * + * @return The list of transitive predecessors. + */ + public abstract Collection> getTransitivePredecessors(); + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + "id=" + id + + ", name='" + name + '\'' + + ", outputType=" + outputType + + ", parallelism=" + parallelism + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof StreamTransformation)) { + return false; + } + + StreamTransformation that = (StreamTransformation) o; + + if (bufferTimeout != that.bufferTimeout) { + return false; + } + if (id != that.id) { + return false; + } + if (parallelism != that.parallelism) { + return false; + } + if (!name.equals(that.name)) { + return false; + } + if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) { + return false; + } + return resourceStrategy == that.resourceStrategy; + } + + @Override + public int hashCode() { + int result = id; + result = 31 * result + name.hashCode(); + result = 31 * result + (outputType != null ? outputType.hashCode() : 0); + result = 31 * result + parallelism; + result = 31 * result + (int) (bufferTimeout ^ (bufferTimeout >>> 32)); + result = 31 * result + resourceStrategy.hashCode(); + return result; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java new file mode 100644 index 0000000000000..e7273c56efe54 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java @@ -0,0 +1,116 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; + +import java.util.Collection; +import java.util.List; + +/** + * This Transformation represents the application of a + * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator} to two input + * {@code StreamTransformations}. The result is again only one stream. + * + * @param The type of the elements in the first input {@code StreamTransformation} + * @param The type of the elements in the second input {@code StreamTransformation} + * @param The type of the elements that result from this {@code TwoInputTransformation} + */ +public class TwoInputTransformation extends StreamTransformation { + + private final StreamTransformation input1; + private final StreamTransformation input2; + + private final TwoInputStreamOperator operator; + + /** + * Creates a new {@code TwoInputTransformation} from the given inputs and operator. + * + * @param input1 The first input {@code StreamTransformation} + * @param input2 The second input {@code StreamTransformation} + * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log + * @param operator The {@code TwoInputStreamOperator} + * @param outputType The type of the elements produced by this Transformation + * @param parallelism The parallelism of this Transformation + */ + public TwoInputTransformation( + StreamTransformation input1, + StreamTransformation input2, + String name, + TwoInputStreamOperator operator, + TypeInformation outputType, + int parallelism) { + super(name, outputType, parallelism); + this.input1 = input1; + this.input2 = input2; + this.operator = operator; + } + + /** + * Returns the first input {@code StreamTransformation} of this {@code TwoInputTransformation}. + */ + public StreamTransformation getInput1() { + return input1; + } + + /** + * Returns the first input {@code StreamTransformation} of this {@code TwoInputTransformation}. + */ + public StreamTransformation getInput2() { + return input2; + } + + /** + * Returns the {@code TypeInformation} for the elements from the first input. + */ + public TypeInformation getInputType1() { + return input1.getOutputType(); + } + + /** + * Returns the {@code TypeInformation} for the elements from the first input. + */ + public TypeInformation getInputType2() { + return input2.getOutputType(); + } + + /** + * Returns the {@code TwoInputStreamOperator} of this Transformation. + */ + public TwoInputStreamOperator getOperator() { + return operator; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input1.getTransitivePredecessors()); + result.addAll(input2.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + operator.setChainingStrategy(strategy); + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java new file mode 100644 index 0000000000000..4fa3c0a52f998 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java @@ -0,0 +1,81 @@ +/** + * 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.streaming.api.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.streaming.api.operators.StreamOperator; + +import java.util.Collection; +import java.util.List; + +/** + * This transformation represents a union of several input + * {@link StreamTransformation StreamTransformations}. + * + *

    + * This does not create a physical operation, it only affects how upstream operations are + * connected to downstream operations. + * + * @param The type of the elements that result from this {@code UnionTransformation} + */ +public class UnionTransformation extends StreamTransformation { + private final List> inputs; + + /** + * Creates a new {@code UnionTransformation} from the given input {@code StreamTransformations}. + * + *

    + * The input {@code StreamTransformations} must all have the same type. + * + * @param inputs The list of input {@code StreamTransformations} + */ + public UnionTransformation(List> inputs) { + super("Union", inputs.get(0).getOutputType(), inputs.get(0).getParallelism()); + + for (StreamTransformation input: inputs) { + if (!input.getOutputType().equals(getOutputType())) { + throw new UnsupportedOperationException("Type mismatch in input " + input); + } + } + + this.inputs = Lists.newArrayList(inputs); + } + + /** + * Returns the list of input {@code StreamTransformations}. + */ + public List> getInputs() { + return inputs; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + for (StreamTransformation input: inputs) { + result.addAll(input.getTransitivePredecessors()); + } + return result; + } + + @Override + public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation."); + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java index f51a04fcd1075..f3d851c9e657d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java @@ -23,8 +23,7 @@ /** * Partitioner that selects all the output channels. * - * @param - * Type of the Tuple + * @param Type of the elements in the Stream being broadcast */ public class BroadcastPartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; @@ -33,10 +32,6 @@ public class BroadcastPartitioner extends StreamPartitioner { boolean set; int setNumber; - public BroadcastPartitioner() { - super(PartitioningStrategy.BROADCAST); - } - @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { @@ -52,4 +47,14 @@ public int[] selectChannels(SerializationDelegate> record, return returnArray; } } + + @Override + public StreamPartitioner copy() { + return this; + } + + @Override + public String toString() { + return "BROADCAST"; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java index 6c40c03521c90..7bb948068ea9b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java @@ -38,7 +38,6 @@ public class CustomPartitionerWrapper extends StreamPartitioner { KeySelector keySelector; public CustomPartitionerWrapper(Partitioner partitioner, KeySelector keySelector) { - super(PartitioningStrategy.CUSTOM); this.partitioner = partitioner; this.keySelector = keySelector; } @@ -58,4 +57,14 @@ public int[] selectChannels(SerializationDelegate> record, int n return returnArray; } + + @Override + public StreamPartitioner copy() { + return this; + } + + @Override + public String toString() { + return "CUSTOM"; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java new file mode 100644 index 0000000000000..4fb460cd7c971 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java @@ -0,0 +1,46 @@ +/* + * 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.streaming.runtime.partitioner; + +import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +/** + * Partitioner that forwards elements only to the locally running downstream operation. + * + * @param Type of the elements in the Stream + */ +public class ForwardPartitioner extends StreamPartitioner { + private static final long serialVersionUID = 1L; + + private int[] returnArray = new int[] {0}; + + @Override + public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { + return returnArray; + } + + public StreamPartitioner copy() { + return this; + } + + @Override + public String toString() { + return "FORWARD"; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java index 46b290b47c3ad..b19fb41921bac 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java @@ -20,19 +20,29 @@ import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -//Group to the partitioner with the lowest id +/** + * Partitioner that sends all elements to the downstream operator with subtask ID=0; + * + * @param Type of the elements in the Stream being partitioned + */ public class GlobalPartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; private int[] returnArray = new int[] { 0 }; - public GlobalPartitioner() { - super(PartitioningStrategy.GLOBAL); - } - @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { return returnArray; } + + @Override + public StreamPartitioner copy() { + return this; + } + + @Override + public String toString() { + return "GLOBAL"; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java similarity index 79% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java index 7026d45203633..a3f5158fd8016 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java @@ -22,20 +22,18 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** - * Partitioner that selects the same (one) channel for two Tuples having a - * specified fields equal. - * - * @param - * Type of the Tuple + * Partitioner selects the target channel based on the hash value of a key from a + * {@link KeySelector}. + * + * @param Type of the elements in the Stream being partitioned */ -public class FieldsPartitioner extends StreamPartitioner { +public class HashPartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; private int[] returnArray = new int[1]; KeySelector keySelector; - public FieldsPartitioner(KeySelector keySelector) { - super(PartitioningStrategy.GROUPBY); + public HashPartitioner(KeySelector keySelector) { this.keySelector = keySelector; } @@ -52,4 +50,14 @@ public int[] selectChannels(SerializationDelegate> record, return returnArray; } + + @Override + public StreamPartitioner copy() { + return this; + } + + @Override + public String toString() { + return "HASH"; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java index e6ad821972d01..2dfff0e333a97 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java @@ -24,34 +24,26 @@ * Partitioner that distributes the data equally by cycling through the output * channels. * - * @param - * Type of the Tuple + * @param Type of the elements in the Stream being rebalanced */ public class RebalancePartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; private int[] returnArray = new int[] {-1}; - private boolean forward; - - public RebalancePartitioner(boolean forward) { - super(forward ? PartitioningStrategy.FORWARD : PartitioningStrategy.DISTRIBUTE); - this.forward = forward; - } @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { this.returnArray[0] = (this.returnArray[0] + 1) % numberOfOutputChannels; - return this.returnArray; } public StreamPartitioner copy() { - return new RebalancePartitioner(forward); + return this; } @Override public String toString() { - return forward ? "ForwardPartitioner" : "RebalancePartitioner"; + return "REBALANCE"; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java index ba50113cecc1b..93c6f9cb61d1d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java @@ -36,14 +36,20 @@ public class ShufflePartitioner extends StreamPartitioner { private int[] returnArray = new int[1]; - public ShufflePartitioner() { - super(PartitioningStrategy.SHUFFLE); - } - @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { returnArray[0] = random.nextInt(numberOfOutputChannels); return returnArray; } + + @Override + public StreamPartitioner copy() { + return new ShufflePartitioner(); + } + + @Override + public String toString() { + return "SHUFFLE"; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java index b37655b0c5e4c..4ef360d910071 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java @@ -24,30 +24,7 @@ public abstract class StreamPartitioner implements ChannelSelector>>, Serializable { - - public enum PartitioningStrategy { - - FORWARD, DISTRIBUTE, SHUFFLE, BROADCAST, GLOBAL, GROUPBY, CUSTOM - - } - private static final long serialVersionUID = 1L; - private PartitioningStrategy strategy; - - public StreamPartitioner(PartitioningStrategy strategy) { - this.strategy = strategy; - } - - public PartitioningStrategy getStrategy() { - return strategy; - } - public StreamPartitioner copy() { - return this; - } - - @Override - public String toString() { - return this.getClass().getSimpleName(); - } + public abstract StreamPartitioner copy(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java index 3ad6b8e198a8f..fdf7697593ab7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java @@ -24,19 +24,23 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.util.NoOpSink; +import org.apache.flink.streaming.util.ReceiveCheckNoOpSink; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; @SuppressWarnings("serial") -public class ChainedRuntimeContextTest { - private static final long MEMORYSIZE = 32; +public class ChainedRuntimeContextTest extends StreamingMultipleProgramsTestBase { private static RuntimeContext srcContext; private static RuntimeContext mapContext; @Test public void test() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); - env.addSource(new TestSource()).map(new TestMap()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + env.addSource(new TestSource()).map(new TestMap()).addSink(new NoOpSink()); env.execute(); assertNotEquals(srcContext, mapContext); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java index 369b3845e8458..7ea1309aee446 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java @@ -31,21 +31,21 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.Collector; import org.junit.Test; -public class CoStreamTest { - - private static final long MEMORY_SIZE = 32; +public class CoStreamTest extends StreamingMultipleProgramsTestBase { private static ArrayList expected = new ArrayList(); @Test public void test() { - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); TestListResultSink resultSink = new TestListResultSink(); @@ -129,4 +129,4 @@ public void flatMap2(Tuple2 value, Collector out) thro assertEquals(expected, result); } -} \ No newline at end of file +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index 324143f04eb3b..9775392908361 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -54,19 +54,20 @@ import org.apache.flink.streaming.api.windowing.helper.Count; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper; -import org.apache.flink.streaming.runtime.partitioner.FieldsPartitioner; +import org.apache.flink.streaming.runtime.partitioner.HashPartitioner; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; +import org.apache.flink.streaming.util.NoOpSink; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.Collector; import org.junit.Test; -public class DataStreamTest { +public class DataStreamTest extends StreamingMultipleProgramsTestBase { - private static final long MEMORYSIZE = 32; - private static int PARALLELISM = 2; /** * Tests {@link SingleOutputStreamOperator#name(String)} functionality. @@ -75,7 +76,7 @@ public class DataStreamTest { */ @Test public void testNaming() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream dataStream1 = env.generateSequence(0, 0).name("testSource1") .map(new MapFunction() { @@ -93,7 +94,7 @@ public Long map(Long value) throws Exception { } }).name("testMap"); - DataStream connected = dataStream1.connect(dataStream2) + DataStreamSink connected = dataStream1.connect(dataStream2) .flatMap(new CoFlatMapFunction() { @Override public void flatMap1(Long value, Collector out) throws Exception { @@ -110,7 +111,8 @@ public Long fold(Long accumulator, Long value) throws Exception { return null; } }).name("testWindowFold") - .flatten(); + .flatten() + .print(); //test functionality through the operator names in the execution plan String plan = env.getExecutionPlan(); @@ -130,8 +132,7 @@ public Long fold(Long accumulator, Long value) throws Exception { @Test @SuppressWarnings("unchecked") public void testPartitioning() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); - StreamGraph graph = env.getStreamGraph(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream src1 = env.fromElements(new Tuple2(0L, 0L)); DataStream src2 = env.fromElements(new Tuple2(0L, 0L)); @@ -143,10 +144,15 @@ public void testPartitioning() { DataStream group3 = src1.groupBy("f0"); DataStream group4 = src1.groupBy(new FirstSelector()); - assertTrue(isPartitioned(graph.getStreamEdge(group1.getId(), createDownStreamId(group1)))); - assertTrue(isPartitioned(graph.getStreamEdge(group2.getId(), createDownStreamId(group2)))); - assertTrue(isPartitioned(graph.getStreamEdge(group3.getId(), createDownStreamId(group3)))); - assertTrue(isPartitioned(graph.getStreamEdge(group4.getId(), createDownStreamId(group4)))); + int id1 = createDownStreamId(group1); + int id2 = createDownStreamId(group2); + int id3 = createDownStreamId(group3); + int id4 = createDownStreamId(group4); + + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id1))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id2))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id3))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id4))); assertTrue(isGrouped(group1)); assertTrue(isGrouped(group2)); @@ -159,10 +165,15 @@ public void testPartitioning() { DataStream partition3 = src1.partitionByHash("f0"); DataStream partition4 = src1.partitionByHash(new FirstSelector()); - assertTrue(isPartitioned(graph.getStreamEdge(partition1.getId(), createDownStreamId(partition1)))); - assertTrue(isPartitioned(graph.getStreamEdge(partition2.getId(), createDownStreamId(partition2)))); - assertTrue(isPartitioned(graph.getStreamEdge(partition3.getId(), createDownStreamId(partition3)))); - assertTrue(isPartitioned(graph.getStreamEdge(partition4.getId(), createDownStreamId(partition4)))); + int pid1 = createDownStreamId(partition1); + int pid2 = createDownStreamId(partition2); + int pid3 = createDownStreamId(partition3); + int pid4 = createDownStreamId(partition4); + + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid1))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid2))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid3))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid4))); assertFalse(isGrouped(partition1)); assertFalse(isGrouped(partition3)); @@ -181,9 +192,13 @@ public int partition(Long key, int numPartitions) { DataStream customPartition3 = src1.partitionCustom(longPartitioner, "f0"); DataStream customPartition4 = src1.partitionCustom(longPartitioner, new FirstSelector()); - assertTrue(isCustomPartitioned(graph.getStreamEdge(customPartition1.getId(), createDownStreamId(customPartition1)))); - assertTrue(isCustomPartitioned(graph.getStreamEdge(customPartition3.getId(), createDownStreamId(customPartition3)))); - assertTrue(isCustomPartitioned(graph.getStreamEdge(customPartition4.getId(), createDownStreamId(customPartition4)))); + int cid1 = createDownStreamId(customPartition1); + int cid2 = createDownStreamId(customPartition3); + int cid3 = createDownStreamId(customPartition4); + + assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid1))); + assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid2))); + assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid3))); assertFalse(isGrouped(customPartition1)); assertFalse(isGrouped(customPartition3)); @@ -205,20 +220,20 @@ public int partition(Long key, int numPartitions) { ConnectedDataStream connectedGroup5 = connected.groupBy(new FirstSelector(), new FirstSelector()); Integer downStreamId5 = createDownStreamId(connectedGroup5); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup1.getFirst().getId(), downStreamId1))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup1.getSecond().getId(), downStreamId1))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId1))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup2.getFirst().getId(), downStreamId2))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup2.getSecond().getId(), downStreamId2))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId2))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId2))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup3.getFirst().getId(), downStreamId3))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup3.getSecond().getId(), downStreamId3))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId3))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId3))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup4.getFirst().getId(), downStreamId4))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup4.getSecond().getId(), downStreamId4))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId4))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId4))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup5.getFirst().getId(), downStreamId5))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup5.getSecond().getId(), downStreamId5))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId5))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId5))); assertTrue(isGrouped(connectedGroup1)); assertTrue(isGrouped(connectedGroup2)); @@ -242,20 +257,30 @@ public int partition(Long key, int numPartitions) { ConnectedDataStream connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector()); Integer connectDownStreamId5 = createDownStreamId(connectedPartition5); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition1.getFirst().getId(), connectDownStreamId1))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition1.getSecond().getId(), connectDownStreamId1))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), + connectDownStreamId1))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), + connectDownStreamId1))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition2.getFirst().getId(), connectDownStreamId2))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition2.getSecond().getId(), connectDownStreamId2))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), + connectDownStreamId2))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), + connectDownStreamId2))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition3.getFirst().getId(), connectDownStreamId3))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition3.getSecond().getId(), connectDownStreamId3))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), + connectDownStreamId3))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), + connectDownStreamId3))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition4.getFirst().getId(), connectDownStreamId4))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition4.getSecond().getId(), connectDownStreamId4))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), + connectDownStreamId4))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), + connectDownStreamId4))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition5.getFirst().getId(), connectDownStreamId5))); - assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition5.getSecond().getId(), connectDownStreamId5))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), + connectDownStreamId5))); + assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), + connectDownStreamId5))); assertFalse(isGrouped(connectedPartition1)); assertFalse(isGrouped(connectedPartition2)); @@ -269,17 +294,17 @@ public int partition(Long key, int numPartitions) { */ @Test public void testParallelism() { - StreamExecutionEnvironment env = new TestStreamEnvironment(10, MEMORYSIZE); - StreamGraph graph = env.getStreamGraph(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource> src = env.fromElements(new Tuple2(0L, 0L)); + env.setParallelism(10); SingleOutputStreamOperator map = src.map(new MapFunction, Long>() { @Override public Long map(Tuple2 value) throws Exception { return null; } - }); + }).name("MyMap"); DataStream windowed = map .window(Count.of(10)) @@ -288,7 +313,10 @@ public Long map(Tuple2 value) throws Exception { public Long fold(Long accumulator, Long value) throws Exception { return null; } - }).flatten(); + }) + .flatten(); + + windowed.addSink(new NoOpSink()); DataStreamSink sink = map.addSink(new SinkFunction() { @Override @@ -296,16 +324,21 @@ public void invoke(Long value) throws Exception { } }); - assertEquals(1, graph.getStreamNode(src.getId()).getParallelism()); - assertEquals(10, graph.getStreamNode(map.getId()).getParallelism()); - assertEquals(10, graph.getStreamNode(windowed.getId()).getParallelism()); - assertEquals(10, graph.getStreamNode(sink.getId()).getParallelism()); + assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism()); + assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism()); + assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism()); + assertEquals(10, + env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism()); env.setParallelism(7); - assertEquals(1, graph.getStreamNode(src.getId()).getParallelism()); - assertEquals(7, graph.getStreamNode(map.getId()).getParallelism()); - assertEquals(7, graph.getStreamNode(windowed.getId()).getParallelism()); - assertEquals(7, graph.getStreamNode(sink.getId()).getParallelism()); + + // Some parts, such as windowing rely on the fact that previous operators have a parallelism + // set when instantiating the Discretizer. This would break if we dynamically changed + // the parallelism of operations when changing the setting on the Execution Environment. + assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism()); + assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism()); + assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism()); + assertEquals(10, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism()); try { src.setParallelism(3); @@ -314,21 +347,22 @@ public void invoke(Long value) throws Exception { } DataStreamSource parallelSource = env.generateSequence(0, 0); - assertEquals(7, graph.getStreamNode(parallelSource.getId()).getParallelism()); + parallelSource.addSink(new NoOpSink()); + assertEquals(7, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism()); parallelSource.setParallelism(3); - assertEquals(3, graph.getStreamNode(parallelSource.getId()).getParallelism()); + assertEquals(3, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism()); map.setParallelism(2); - assertEquals(2, graph.getStreamNode(map.getId()).getParallelism()); + assertEquals(2, env.getStreamGraph().getStreamNode(map.getId()).getParallelism()); sink.setParallelism(4); - assertEquals(4, graph.getStreamNode(sink.getId()).getParallelism()); + assertEquals(4, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism()); } @Test public void testTypeInfo() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream src1 = env.generateSequence(0, 0); assertEquals(TypeExtractor.getForClass(Long.class), src1.getType()); @@ -366,9 +400,7 @@ public CustomPOJO fold(CustomPOJO accumulator, String value) throws Exception { @Test public void operatorTest() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); - - StreamGraph streamGraph = env.getStreamGraph(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource src = env.generateSequence(0, 0); @@ -379,6 +411,7 @@ public Integer map(Long value) throws Exception { } }; DataStream map = src.map(mapFunction); + map.addSink(new NoOpSink()); assertEquals(mapFunction, getFunctionForDataStream(map)); @@ -388,6 +421,7 @@ public void flatMap(Long value, Collector out) throws Exception { } }; DataStream flatMap = src.flatMap(flatMapFunction); + flatMap.addSink(new NoOpSink()); assertEquals(flatMapFunction, getFunctionForDataStream(flatMap)); FilterFunction filterFunction = new FilterFunction() { @@ -401,16 +435,18 @@ public boolean filter(Integer value) throws Exception { .union(flatMap) .filter(filterFunction); + unionFilter.addSink(new NoOpSink()); + assertEquals(filterFunction, getFunctionForDataStream(unionFilter)); try { - streamGraph.getStreamEdge(map.getId(), unionFilter.getId()); + env.getStreamGraph().getStreamEdge(map.getId(), unionFilter.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } try { - streamGraph.getStreamEdge(flatMap.getId(), unionFilter.getId()); + env.getStreamGraph().getStreamEdge(flatMap.getId(), unionFilter.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } @@ -423,14 +459,15 @@ public Iterable select(Integer value) { }; SplitDataStream split = unionFilter.split(outputSelector); - List> outputSelectors = streamGraph.getStreamNode(split.getId()).getOutputSelectors(); + split.select("dummy").addSink(new NoOpSink()); + List> outputSelectors = env.getStreamGraph().getStreamNode(unionFilter.getId()).getOutputSelectors(); assertEquals(1, outputSelectors.size()); assertEquals(outputSelector, outputSelectors.get(0)); DataStream select = split.select("a"); DataStreamSink sink = select.print(); - StreamEdge splitEdge = streamGraph.getStreamEdge(select.getId(), sink.getId()); + StreamEdge splitEdge = env.getStreamGraph().getStreamEdge(unionFilter.getId(), sink.getTransformation().getId()); assertEquals("a", splitEdge.getSelectedNames().get(0)); ConnectedDataStream connect = map.connect(flatMap); @@ -446,16 +483,17 @@ public String map2(Integer value) { } }; DataStream coMap = connect.map(coMapper); + coMap.addSink(new NoOpSink()); assertEquals(coMapper, getFunctionForDataStream(coMap)); try { - streamGraph.getStreamEdge(map.getId(), coMap.getId()); + env.getStreamGraph().getStreamEdge(map.getId(), coMap.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } try { - streamGraph.getStreamEdge(flatMap.getId(), coMap.getId()); + env.getStreamGraph().getStreamEdge(flatMap.getId(), coMap.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } @@ -463,12 +501,11 @@ public String map2(Integer value) { @Test public void sinkKeyTest() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); - StreamGraph streamGraph = env.getStreamGraph(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataStream sink = env.generateSequence(1, 100).print(); - assertTrue(streamGraph.getStreamNode(sink.getId()).getStatePartitioner() == null); - assertTrue(streamGraph.getStreamNode(sink.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner); + DataStreamSink sink = env.generateSequence(1, 100).print(); + assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getStatePartitioner() == null); + assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof ForwardPartitioner); KeySelector key1 = new KeySelector() { @@ -480,11 +517,11 @@ public Long getKey(Long value) throws Exception { } }; - DataStream sink2 = env.generateSequence(1, 100).keyBy(key1).print(); + DataStreamSink sink2 = env.generateSequence(1, 100).keyBy(key1).print(); - assertTrue(streamGraph.getStreamNode(sink2.getId()).getStatePartitioner() != null); - assertEquals(key1, streamGraph.getStreamNode(sink2.getId()).getStatePartitioner()); - assertTrue(streamGraph.getStreamNode(sink2.getId()).getInEdges().get(0).getPartitioner() instanceof FieldsPartitioner); + assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner() != null); + assertEquals(key1, env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner()); + assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner); KeySelector key2 = new KeySelector() { @@ -496,49 +533,52 @@ public Long getKey(Long value) throws Exception { } }; - DataStream sink3 = env.generateSequence(1, 100).keyBy(key2).print(); + DataStreamSink sink3 = env.generateSequence(1, 100).keyBy(key2).print(); - assertTrue(streamGraph.getStreamNode(sink3.getId()).getStatePartitioner() != null); - assertEquals(key2, streamGraph.getStreamNode(sink3.getId()).getStatePartitioner()); - assertTrue(streamGraph.getStreamNode(sink3.getId()).getInEdges().get(0).getPartitioner() instanceof FieldsPartitioner); + assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner() != null); + assertEquals(key2, env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner()); + assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner); } @Test public void testChannelSelectors() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); - - StreamGraph streamGraph = env.getStreamGraph(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource src = env.generateSequence(0, 0); DataStream broadcast = src.broadcast(); DataStreamSink broadcastSink = broadcast.print(); StreamPartitioner broadcastPartitioner = - streamGraph.getStreamEdge(broadcast.getId(), broadcastSink.getId()).getPartitioner(); + env.getStreamGraph().getStreamEdge(src.getId(), + broadcastSink.getTransformation().getId()).getPartitioner(); assertTrue(broadcastPartitioner instanceof BroadcastPartitioner); DataStream shuffle = src.shuffle(); DataStreamSink shuffleSink = shuffle.print(); StreamPartitioner shufflePartitioner = - streamGraph.getStreamEdge(shuffle.getId(), shuffleSink.getId()).getPartitioner(); + env.getStreamGraph().getStreamEdge(src.getId(), + shuffleSink.getTransformation().getId()).getPartitioner(); assertTrue(shufflePartitioner instanceof ShufflePartitioner); DataStream forward = src.forward(); DataStreamSink forwardSink = forward.print(); StreamPartitioner forwardPartitioner = - streamGraph.getStreamEdge(forward.getId(), forwardSink.getId()).getPartitioner(); - assertTrue(forwardPartitioner instanceof RebalancePartitioner); + env.getStreamGraph().getStreamEdge(src.getId(), + forwardSink.getTransformation().getId()).getPartitioner(); + assertTrue(forwardPartitioner instanceof ForwardPartitioner); DataStream rebalance = src.rebalance(); DataStreamSink rebalanceSink = rebalance.print(); StreamPartitioner rebalancePartitioner = - streamGraph.getStreamEdge(rebalance.getId(), rebalanceSink.getId()).getPartitioner(); + env.getStreamGraph().getStreamEdge(src.getId(), + rebalanceSink.getTransformation().getId()).getPartitioner(); assertTrue(rebalancePartitioner instanceof RebalancePartitioner); DataStream global = src.global(); DataStreamSink globalSink = global.print(); StreamPartitioner globalPartitioner = - streamGraph.getStreamEdge(global.getId(), globalSink.getId()).getPartitioner(); + env.getStreamGraph().getStreamEdge(src.getId(), + globalSink.getTransformation().getId()).getPartitioner(); assertTrue(globalPartitioner instanceof GlobalPartitioner); } @@ -559,7 +599,7 @@ private static Function getFunctionForDataStream(DataStream dataStream) { } private static Integer createDownStreamId(DataStream dataStream) { - return dataStream.print().getId(); + return dataStream.print().getTransformation().getId(); } private static boolean isGrouped(DataStream dataStream) { @@ -567,7 +607,7 @@ private static boolean isGrouped(DataStream dataStream) { } private static Integer createDownStreamId(ConnectedDataStream dataStream) { - return dataStream.map(new CoMapFunction, Tuple2, Object>() { + SingleOutputStreamOperator coMap = dataStream.map(new CoMapFunction, Tuple2, Object>() { @Override public Object map1(Tuple2 value) { return null; @@ -577,7 +617,9 @@ public Object map1(Tuple2 value) { public Object map2(Tuple2 value) { return null; } - }).getId(); + }); + coMap.addSink(new NoOpSink()); + return coMap.getId(); } private static boolean isGrouped(ConnectedDataStream dataStream) { @@ -585,7 +627,7 @@ private static boolean isGrouped(ConnectedDataStream dataStream) { } private static boolean isPartitioned(StreamEdge edge) { - return edge.getPartitioner() instanceof FieldsPartitioner; + return edge.getPartitioner() instanceof HashPartitioner; } private static boolean isCustomPartitioned(StreamEdge edge) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java index 0fad3dd14895b..285ee57c7c3be 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java @@ -1,27 +1,22 @@ /* - * 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. - */ +* 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.streaming.api; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -31,146 +26,235 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.IterativeDataStream; import org.apache.flink.streaming.api.datastream.IterativeDataStream.ConnectedIterativeDataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.datastream.SplitDataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.api.graph.StreamLoop; import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; +import org.apache.flink.streaming.util.EvenOddOutputSelector; +import org.apache.flink.streaming.util.NoOpIntMap; +import org.apache.flink.streaming.util.ReceiveCheckNoOpSink; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.Collector; import org.junit.Test; +import static org.junit.Assert.*; + @SuppressWarnings({ "unchecked", "unused", "serial" }) public class IterateTest extends StreamingMultipleProgramsTestBase { - private static final long MEMORYSIZE = 32; private static boolean iterated[]; - private static int PARALLELISM = 2; - @Test - public void testException() throws Exception { + @Test(expected = UnsupportedOperationException.class) + public void testIncorrectParallelism() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); DataStream source = env.fromElements(1, 10); + + IterativeDataStream iter1 = source.iterate(); + SingleOutputStreamOperator map1 = iter1.map(NoOpIntMap); + iter1.closeWith(map1).print(); + } + + @Test + public void testDoubleClosing() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // introduce dummy mapper to get to correct parallelism + DataStream source = env.fromElements(1, 10).map(NoOpIntMap); + IterativeDataStream iter1 = source.iterate(); - IterativeDataStream iter2 = source.iterate(); iter1.closeWith(iter1.map(NoOpIntMap)); - // Check for double closing - try { - iter1.closeWith(iter1.map(NoOpIntMap)); - fail(); - } catch (Exception e) { - } + iter1.closeWith(iter1.map(NoOpIntMap)); + } - // Check for closing iteration without head - try { - iter2.closeWith(iter1.map(NoOpIntMap)); - fail(); - } catch (Exception e) { - } - iter2.map(NoOpIntMap); + @Test(expected = UnsupportedOperationException.class) + public void testDifferingParallelism() throws Exception { - // Check for executing with empty iteration - try { - env.execute(); - fail(); - } catch (Exception e) { - } + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // introduce dummy mapper to get to correct parallelism + DataStream source = env.fromElements(1, 10) + .map(NoOpIntMap); + + IterativeDataStream iter1 = source.iterate(); + + + iter1.closeWith(iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2)); + + } + + + @Test(expected = UnsupportedOperationException.class) + public void testCoDifferingParallelism() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // introduce dummy mapper to get to correct parallelism + DataStream source = env.fromElements(1, 10).map(NoOpIntMap); + + ConnectedIterativeDataStream coIter = source.iterate().withFeedbackType( + Integer.class); + + + coIter.closeWith(coIter.map(NoOpIntCoMap).setParallelism(DEFAULT_PARALLELISM / 2)); + + } + + @Test(expected = UnsupportedOperationException.class) + public void testClosingFromOutOfLoop() throws Exception { + + // this test verifies that we cannot close an iteration with a DataStream that does not + // have the iteration in its predecessors + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // introduce dummy mapper to get to correct parallelism + DataStream source = env.fromElements(1, 10).map(NoOpIntMap); + + IterativeDataStream iter1 = source.iterate(); + IterativeDataStream iter2 = source.iterate(); + + + iter2.closeWith(iter1.map(NoOpIntMap)); + + } + + @Test(expected = UnsupportedOperationException.class) + public void testCoIterClosingFromOutOfLoop() throws Exception { + + // this test verifies that we cannot close an iteration with a DataStream that does not + // have the iteration in its predecessors + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // introduce dummy mapper to get to correct parallelism + DataStream source = env.fromElements(1, 10).map(NoOpIntMap); + + IterativeDataStream iter1 = source.iterate(); + ConnectedIterativeDataStream coIter = source.iterate().withFeedbackType( + Integer.class); + + + coIter.closeWith(iter1.map(NoOpIntMap)); + + } + + @Test(expected = IllegalStateException.class) + public void testExecutionWithEmptyIteration() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream source = env.fromElements(1, 10).map(NoOpIntMap); + + IterativeDataStream iter1 = source.iterate(); + + iter1.map(NoOpIntMap).print(); + + env.execute(); } @Test public void testImmutabilityWithCoiteration() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); - DataStream source = env.fromElements(1, 10); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream source = env.fromElements(1, 10).map(NoOpIntMap); // for rebalance IterativeDataStream iter1 = source.iterate(); // Calling withFeedbackType should create a new iteration ConnectedIterativeDataStream iter2 = iter1.withFeedbackType(String.class); - iter1.closeWith(iter1.map(NoOpIntMap)); - iter2.closeWith(iter2.map(NoOpCoMap)); + iter1.closeWith(iter1.map(NoOpIntMap)).print(); + iter2.closeWith(iter2.map(NoOpCoMap)).print(); StreamGraph graph = env.getStreamGraph(); - graph.getJobGraph(); + assertEquals(2, graph.getIterationSourceSinkPairs().size()); - assertEquals(2, graph.getStreamLoops().size()); - for (StreamLoop loop : graph.getStreamLoops()) { - assertEquals(loop.getHeads(), loop.getTails()); - List> sourceSinkPairs = loop.getSourceSinkPairs(); - assertEquals(1, sourceSinkPairs.size()); + for (Tuple2 sourceSinkPair: graph.getIterationSourceSinkPairs()) { + assertEquals(sourceSinkPair.f0.getOutEdges().get(0).getTargetVertex(), sourceSinkPair.f1.getInEdges().get(0).getSourceVertex()); } } @Test public void testmultipleHeadsTailsSimple() { - StreamExecutionEnvironment env = new TestStreamEnvironment(4, MEMORYSIZE); - DataStream source1 = env.fromElements(1, 2, 3, 4, 5).shuffle(); - DataStream source2 = env.fromElements(1, 2, 3, 4, 5); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - IterativeDataStream iter1 = source1.union(source2).iterate(); + DataStream source1 = env.fromElements(1, 2, 3, 4, 5) + .shuffle() + .map(NoOpIntMap).name("ParallelizeMapShuffle"); + DataStream source2 = env.fromElements(1, 2, 3, 4, 5) + .map(NoOpIntMap).name("ParallelizeMapRebalance"); - DataStream head1 = iter1.map(NoOpIntMap); - DataStream head2 = iter1.map(NoOpIntMap).setParallelism(2); - DataStream head3 = iter1.map(NoOpIntMap).setParallelism(2) - .addSink(new NoOpSink()); - DataStream head4 = iter1.map(NoOpIntMap).addSink(new NoOpSink()); + IterativeDataStream iter1 = source1.union(source2).iterate(); - SplitDataStream source3 = env.fromElements(1, 2, 3, 4, 5).split( - new OutputSelector() { + DataStream head1 = iter1.map(NoOpIntMap).name("IterRebalanceMap").setParallelism(DEFAULT_PARALLELISM / 2); + DataStream head2 = iter1.map(NoOpIntMap).name("IterForwardMap"); + DataStreamSink head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).addSink(new ReceiveCheckNoOpSink()); + DataStreamSink head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink()); - @Override - public Iterable select(Integer value) { - return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd"); - } - }); + SplitDataStream source3 = env.fromElements(1, 2, 3, 4, 5) + .map(NoOpIntMap).name("EvenOddSourceMap") + .split(new EvenOddOutputSelector()); iter1.closeWith(source3.select("even").union( - head1.map(NoOpIntMap).broadcast().setParallelism(1), head2.shuffle())); + head1.rebalance().map(NoOpIntMap).broadcast(), head2.shuffle())); StreamGraph graph = env.getStreamGraph(); JobGraph jg = graph.getJobGraph(); - assertEquals(1, graph.getStreamLoops().size()); - StreamLoop loop = new ArrayList(graph.getStreamLoops()).get(0); + assertEquals(1, graph.getIterationSourceSinkPairs().size()); - assertEquals(4, loop.getHeads().size()); - assertEquals(3, loop.getTails().size()); + Tuple2 sourceSinkPair = graph.getIterationSourceSinkPairs().iterator().next(); + StreamNode itSource = sourceSinkPair.f0; + StreamNode itSink = sourceSinkPair.f1; - assertEquals(1, loop.getSourceSinkPairs().size()); - Tuple2 pair = loop.getSourceSinkPairs().get(0); + assertEquals(4, itSource.getOutEdges().size()); + assertEquals(3, itSink.getInEdges().size()); - assertEquals(pair.f0.getParallelism(), pair.f1.getParallelism()); - assertEquals(4, pair.f0.getOutEdges().size()); - assertEquals(3, pair.f1.getInEdges().size()); + assertEquals(itSource.getParallelism(), itSink.getParallelism()); - for (StreamEdge edge : pair.f0.getOutEdges()) { - assertTrue(edge.getPartitioner() instanceof ShufflePartitioner); - } - for (StreamEdge edge : pair.f1.getInEdges()) { - assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); + for (StreamEdge edge : itSource.getOutEdges()) { + if (edge.getTargetVertex().getOperatorName().equals("IterRebalanceMap")) { + assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); + } else if (edge.getTargetVertex().getOperatorName().equals("IterForwardMap")) { + assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); + } } + for (StreamEdge edge : itSink.getInEdges()) { + if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapShuffle")) { + assertTrue(edge.getPartitioner() instanceof ShufflePartitioner); + } - assertTrue(loop.getTailSelectedNames().contains(Arrays.asList("even"))); + if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapForward")) { + assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); + } + + if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("EvenOddSourceMap")) { + assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); + assertTrue(edge.getSelectedNames().contains("even")); + } + } // Test co-location @@ -193,142 +277,106 @@ public Iterable select(Integer value) { @Test public void testmultipleHeadsTailsWithTailPartitioning() { - StreamExecutionEnvironment env = new TestStreamEnvironment(4, MEMORYSIZE); - DataStream source1 = env.fromElements(1, 2, 3, 4, 5).shuffle(); - DataStream source2 = env.fromElements(1, 2, 3, 4, 5); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - IterativeDataStream iter1 = source1.union(source2).iterate(); + DataStream source1 = env.fromElements(1, 2, 3, 4, 5) + .shuffle() + .map(NoOpIntMap); - DataStream head1 = iter1.map(NoOpIntMap); - DataStream head2 = iter1.map(NoOpIntMap).setParallelism(2).name("shuffle"); - DataStream head3 = iter1.map(NoOpIntMap).setParallelism(2) - .addSink(new NoOpSink()); - DataStream head4 = iter1.map(NoOpIntMap).addSink(new NoOpSink()); + DataStream source2 = env.fromElements(1, 2, 3, 4, 5) + .map(NoOpIntMap); - SplitDataStream source3 = env.fromElements(1, 2, 3, 4, 5).name("split") - .split(new OutputSelector() { + IterativeDataStream iter1 = source1.union(source2).iterate(); - @Override - public Iterable select(Integer value) { - return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd"); - } - }); + DataStream head1 = iter1.map(NoOpIntMap).name("map1"); + DataStream head2 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).rebalance().name( + "shuffle"); + DataStreamSink head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2) + .addSink(new ReceiveCheckNoOpSink()); + DataStreamSink head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink()); + + SplitDataStream source3 = env.fromElements(1, 2, 3, 4, 5) + .map(NoOpIntMap) + .name("split") + .split(new EvenOddOutputSelector()); iter1.closeWith( source3.select("even").union( - head1.map(NoOpIntMap).broadcast().setParallelism(1).name("bc"), - head2.shuffle()), true); + head1.map(NoOpIntMap).broadcast().name("bc"), + head2.map(NoOpIntMap).shuffle())); StreamGraph graph = env.getStreamGraph(); JobGraph jg = graph.getJobGraph(); - assertEquals(1, graph.getStreamLoops().size()); + assertEquals(1, graph.getIterationSourceSinkPairs().size()); - StreamLoop loop = new ArrayList(graph.getStreamLoops()).get(0); + Tuple2 sourceSinkPair = graph.getIterationSourceSinkPairs().iterator().next(); + StreamNode itSource = sourceSinkPair.f0; + StreamNode itSink = sourceSinkPair.f1; - assertEquals(4, loop.getHeads().size()); - assertEquals(3, loop.getTails().size()); + assertEquals(4, itSource.getOutEdges().size()); + assertEquals(3, itSink.getInEdges().size()); - assertEquals(2, loop.getSourceSinkPairs().size()); - List> pairs = loop.getSourceSinkPairs(); - Tuple2 pair1 = pairs.get(0).f0.getParallelism() == 2 ? pairs.get(0) - : pairs.get(1); - Tuple2 pair2 = pairs.get(0).f0.getParallelism() == 4 ? pairs.get(0) - : pairs.get(1); - assertEquals(pair1.f0.getParallelism(), pair1.f1.getParallelism()); - assertEquals(2, pair1.f0.getParallelism()); - assertEquals(2, pair1.f0.getOutEdges().size()); - assertEquals(3, pair1.f1.getInEdges().size()); + assertEquals(itSource.getParallelism(), itSink.getParallelism()); - for (StreamEdge edge : pair1.f0.getOutEdges()) { - assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); - assertEquals(2, edge.getTargetVertex().getParallelism()); - } - for (StreamEdge edge : pair1.f1.getInEdges()) { - String tailName = edge.getSourceVertex().getOperatorName(); - if (tailName.equals("split")) { + for (StreamEdge edge : itSource.getOutEdges()) { + if (edge.getTargetVertex().getOperatorName().equals("map1")) { + assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); + assertEquals(4, edge.getTargetVertex().getParallelism()); + } else if (edge.getTargetVertex().getOperatorName().equals("shuffle")) { assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); - } else if (tailName.equals("bc")) { - assertTrue(edge.getPartitioner() instanceof BroadcastPartitioner); - } else if (tailName.equals("shuffle")) { - assertTrue(edge.getPartitioner() instanceof ShufflePartitioner); + assertEquals(2, edge.getTargetVertex().getParallelism()); } - } - - assertEquals(pair2.f0.getParallelism(), pair2.f1.getParallelism()); - assertEquals(4, pair2.f0.getParallelism()); - assertEquals(2, pair2.f0.getOutEdges().size()); - assertEquals(3, pair2.f1.getInEdges().size()); - - for (StreamEdge edge : pair2.f0.getOutEdges()) { - assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); - assertEquals(4, edge.getTargetVertex().getParallelism()); - } - for (StreamEdge edge : pair2.f1.getInEdges()) { + for (StreamEdge edge : itSink.getInEdges()) { String tailName = edge.getSourceVertex().getOperatorName(); if (tailName.equals("split")) { - assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); + assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); + assertTrue(edge.getSelectedNames().contains("even")); } else if (tailName.equals("bc")) { assertTrue(edge.getPartitioner() instanceof BroadcastPartitioner); } else if (tailName.equals("shuffle")) { assertTrue(edge.getPartitioner() instanceof ShufflePartitioner); } - } - assertTrue(loop.getTailSelectedNames().contains(Arrays.asList("even"))); - // Test co-location JobVertex itSource1 = null; - JobVertex itSource2 = null; JobVertex itSink1 = null; - JobVertex itSink2 = null; for (JobVertex vertex : jg.getVertices()) { if (vertex.getName().contains("IterationSource")) { - if (vertex.getName().contains("_0")) { - itSource1 = vertex; - } else if (vertex.getName().contains("_1")) { - itSource2 = vertex; - } + itSource1 = vertex; } else if (vertex.getName().contains("IterationSink")) { - if (vertex.getName().contains("_0")) { - itSink1 = vertex; - } else if (vertex.getName().contains("_1")) { - itSink2 = vertex; - } + itSink1 = vertex; } } assertTrue(itSource1.getCoLocationGroup() != null); - assertTrue(itSource2.getCoLocationGroup() != null); + assertTrue(itSink1.getCoLocationGroup() != null); assertEquals(itSource1.getCoLocationGroup(), itSink1.getCoLocationGroup()); - assertEquals(itSource2.getCoLocationGroup(), itSink2.getCoLocationGroup()); - assertNotEquals(itSource1.getCoLocationGroup(), itSource2.getCoLocationGroup()); } @SuppressWarnings("rawtypes") @Test public void testSimpleIteration() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - iterated = new boolean[PARALLELISM]; + iterated = new boolean[DEFAULT_PARALLELISM]; - DataStream source = env - .fromCollection(Collections.nCopies(PARALLELISM * 2, false)); + DataStream source = env.fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false)) + .map(NoOpBoolMap).name("ParallelizeMap"); IterativeDataStream iteration = source.iterate(3000); DataStream increment = iteration.flatMap(new IterationHead()).map(NoOpBoolMap); - iteration.map(NoOpBoolMap).addSink(new NoOpSink()); + iteration.map(NoOpBoolMap).addSink(new ReceiveCheckNoOpSink()); - iteration.closeWith(increment).addSink(new NoOpSink()); + iteration.closeWith(increment).addSink(new ReceiveCheckNoOpSink()); env.execute(); @@ -343,7 +391,13 @@ public void testCoIteration() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(2); - ConnectedIterativeDataStream coIt = env.fromElements(0, 0).iterate(2000) + DataStream otherSource = env.fromElements("1000", "2000") + .map(NoOpStrMap).name("ParallelizeMap"); + + + ConnectedIterativeDataStream coIt = env.fromElements(0, 0) + .map(NoOpIntMap).name("ParallelizeMap") + .iterate(2000) .withFeedbackType("String"); try { @@ -392,25 +446,24 @@ public String map1(Integer value) throws Exception { public String map2(String value) throws Exception { return value; } - }).setParallelism(1).addSink(new NoOpSink()); + }).addSink(new ReceiveCheckNoOpSink()); - coIt.closeWith(head.broadcast().union(env.fromElements("1000", "2000").rebalance())); + coIt.closeWith(head.broadcast().union(otherSource)); head.addSink(new TestSink()).setParallelism(1); + assertEquals(1, env.getStreamGraph().getIterationSourceSinkPairs().size()); + env.execute(); Collections.sort(TestSink.collected); assertEquals(Arrays.asList("1", "1", "2", "2", "2", "2"), TestSink.collected); - assertEquals(2, new ArrayList(env.getStreamGraph().getStreamLoops()).get(0) - .getSourceSinkPairs().size()); - } @Test public void testGroupByFeedback() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); + env.setParallelism(DEFAULT_PARALLELISM - 1); KeySelector key = new KeySelector() { @@ -420,7 +473,8 @@ public Integer getKey(Integer value) throws Exception { } }; - DataStream source = env.fromElements(1, 2, 3); + DataStream source = env.fromElements(1, 2, 3) + .map(NoOpIntMap).name("ParallelizeMap"); IterativeDataStream it = source.groupBy(key).iterate(3000); @@ -448,8 +502,7 @@ public void close() { } }); - it.closeWith(head.groupBy(key).union(head.map(NoOpIntMap).setParallelism(2).groupBy(key)), - true).addSink(new NoOpSink()); + it.closeWith(head.groupBy(key).union(head.map(NoOpIntMap).groupBy(key))).addSink(new ReceiveCheckNoOpSink()); env.execute(); } @@ -457,15 +510,17 @@ public void close() { @SuppressWarnings("deprecation") @Test public void testWithCheckPointing() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(); - DataStream source = env - .fromCollection(Collections.nCopies(PARALLELISM * 2, false)); + DataStream source = env .fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false)) + .map(NoOpBoolMap).name("ParallelizeMap"); + IterativeDataStream iteration = source.iterate(3000); - iteration.closeWith(iteration.flatMap(new IterationHead())); + iteration.closeWith(iteration.flatMap(new IterationHead())).addSink(new ReceiveCheckNoOpSink()); try { env.execute(); @@ -503,22 +558,6 @@ public void flatMap(Boolean value, Collector out) throws Exception { } } - public static final class NoOpSink extends RichSinkFunction { - private List received; - - public void invoke(T tuple) { - received.add(tuple); - } - - public void open(Configuration conf) { - received = new ArrayList(); - } - - public void close() { - assertTrue(received.size() > 0); - } - } - public static CoMapFunction NoOpCoMap = new CoMapFunction() { public String map1(Integer value) throws Exception { @@ -530,9 +569,23 @@ public String map2(String value) throws Exception { } }; - public static MapFunction NoOpIntMap = new MapFunction() { + public static MapFunction NoOpIntMap = new NoOpIntMap(); + + public static MapFunction NoOpStrMap = new MapFunction() { + + public String map(String value) throws Exception { + return value; + } + + }; + + public static CoMapFunction NoOpIntCoMap = new CoMapFunction() { + + public Integer map1(Integer value) throws Exception { + return value; + } - public Integer map(Integer value) throws Exception { + public Integer map2(Integer value) throws Exception { return value; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java index fc78d27d456d0..8525d37b787cc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java @@ -26,13 +26,12 @@ import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; -public class OutputSplitterTest { - - private static final long MEMORYSIZE = 32; +public class OutputSplitterTest extends StreamingMultipleProgramsTestBase { private static ArrayList expectedSplitterResult = new ArrayList(); @@ -42,7 +41,8 @@ public void testOnMergedDataStream() throws Exception { TestListResultSink splitterResultSink1 = new TestListResultSink(); TestListResultSink splitterResultSink2 = new TestListResultSink(); - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); env.setBufferTimeout(1); DataStream d1 = env.fromElements(0, 2, 4, 6, 8); @@ -95,7 +95,8 @@ public void testOnSingleDataStream() throws Exception { TestListResultSink splitterResultSink1 = new TestListResultSink(); TestListResultSink splitterResultSink2 = new TestListResultSink(); - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); env.setBufferTimeout(1); DataStream ds = env.fromElements(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java index c8588341d57d5..987a8fb3fc228 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java @@ -25,14 +25,18 @@ import java.util.HashSet; import java.util.List; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.NoOpIntMap; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; @@ -40,10 +44,33 @@ /** * IT case that tests the different stream partitioning schemes. */ -public class PartitionerTest { +public class PartitionerTest extends StreamingMultipleProgramsTestBase { + + @Test(expected = UnsupportedOperationException.class) + public void testForwardFailsLowToHighParallelism() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream src = env.fromElements(1, 2, 3); + + // this doesn't work because it goes from 1 to 3 + src.forward().map(new NoOpIntMap()); + + env.execute(); + } + + @Test(expected = UnsupportedOperationException.class) + public void testForwardFailsHightToLowParallelism() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // this does a rebalance that works + DataStream src = env.fromElements(1, 2, 3).map(new NoOpIntMap()); + + // this doesn't work because it goes from 3 to 1 + src.forward().map(new NoOpIntMap()).setParallelism(1); + + env.execute(); + } - public static final int PARALLELISM = 3; - public static final int MEMORY_SIZE = 32; @Test public void partitionerTest() { @@ -62,7 +89,9 @@ public void partitionerTest() { new TestListResultSink>(); - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORY_SIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(3); + DataStream> src = env.fromElements( new Tuple1("a"), new Tuple1("b"), @@ -98,12 +127,21 @@ public int partition(String key, int numPartitions) { // partition broadcast src.broadcast().map(new SubtaskIndexAssigner()).addSink(broadcastPartitionResultSink); - // partition forward - src.map(new SubtaskIndexAssigner()).addSink(forwardPartitionResultSink); - // partition rebalance src.rebalance().map(new SubtaskIndexAssigner()).addSink(rebalancePartitionResultSink); + // partition forward + src.map(new MapFunction, Tuple1>() { + private static final long serialVersionUID = 1L; + @Override + public Tuple1 map(Tuple1 value) throws Exception { + return value; + } + }) + .forward() + .map(new SubtaskIndexAssigner()) + .addSink(forwardPartitionResultSink); + // partition global src.global().map(new SubtaskIndexAssigner()).addSink(globalPartitionResultSink); @@ -209,8 +247,8 @@ private static void verifyGlobalPartitioning(List> globa new HashSet>(globalPartitionResult)); } - private static class SubtaskIndexAssigner - extends RichMapFunction, Tuple2> { + private static class SubtaskIndexAssigner extends RichMapFunction, Tuple2> { + private static final long serialVersionUID = 1L; private int indexOfSubtask; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java index e2fe599129cf5..606259e852eec 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java @@ -37,24 +37,23 @@ import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.util.NoOpSink; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.SplittableIterator; import org.junit.Test; -public class StreamExecutionEnvironmentTest { - - private static final long MEMORYSIZE = 32; - private static int PARALLELISM = 4; +public class StreamExecutionEnvironmentTest extends StreamingMultipleProgramsTestBase { @Test @SuppressWarnings("unchecked") public void testFromCollectionParallelism() { try { TypeInformation typeInfo = BasicTypeInfo.INT_TYPE_INFO; - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource dataStream1 = env.fromCollection(new DummySplittableIterator(), typeInfo); - + try { dataStream1.setParallelism(4); fail("should throw an exception"); @@ -62,15 +61,20 @@ public void testFromCollectionParallelism() { catch (IllegalArgumentException e) { // expected } + + dataStream1.addSink(new NoOpSink()); - env.fromParallelCollection(new DummySplittableIterator(), typeInfo).setParallelism(4); - + DataStreamSource dataStream2 = env.fromParallelCollection(new DummySplittableIterator(), + typeInfo).setParallelism(4); + + dataStream2.addSink(new NoOpSink()); + String plan = env.getExecutionPlan(); - - assertTrue("Parallelism for dataStream1 is not right.", - plan.contains("\"contents\":\"Collection Source\",\"parallelism\":1")); - assertTrue("Parallelism for dataStream2 is not right.", - plan.contains("\"contents\":\"Parallel Collection Source\",\"parallelism\":4")); + + assertEquals("Parallelism of collection source must be 1.", 1, env.getStreamGraph().getStreamNode(dataStream1.getId()).getParallelism()); + assertEquals("Parallelism of parallel collection source must be 4.", + 4, + env.getStreamGraph().getStreamNode(dataStream2.getId()).getParallelism()); } catch (Exception e) { e.printStackTrace(); @@ -80,7 +84,7 @@ public void testFromCollectionParallelism() { @Test public void testSources() { - StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); SourceFunction srcFun = new SourceFunction() { private static final long serialVersionUID = 1L; @@ -94,6 +98,7 @@ public void cancel() { } }; DataStreamSource src1 = env.addSource(srcFun); + src1.addSink(new NoOpSink()); assertEquals(srcFun, getFunctionFromDataSource(src1)); List list = Arrays.asList(0L, 1L, 2L); @@ -120,6 +125,7 @@ private static StreamOperator getOperatorFromDataStream(DataStream dataStr } private static SourceFunction getFunctionFromDataSource(DataStreamSource dataStreamSource) { + dataStreamSource.addSink(new NoOpSink()); AbstractUdfStreamOperator operator = (AbstractUdfStreamOperator) getOperatorFromDataStream(dataStreamSource); return (SourceFunction) operator.getUserFunction(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java index 7d08709893bb4..0989128999f86 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java @@ -33,11 +33,12 @@ import org.apache.flink.streaming.api.functions.co.CoReduceFunction; import org.apache.flink.streaming.api.functions.co.CoWindowFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.util.Collector; import org.junit.Test; @SuppressWarnings("serial") -public class TypeFillTest { +public class TypeFillTest extends StreamingMultipleProgramsTestBase { @Test public void test() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java index cccec4045a874..508f1a24b6f6f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java @@ -30,15 +30,12 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.windowing.helper.Timestamp; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; -public class WindowCrossJoinTest implements Serializable { - - private static final long serialVersionUID = 1L; - - private static final long MEMORYSIZE = 32; +public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase { private static ArrayList, Integer>> joinExpectedResults = new ArrayList, Integer>>(); private static ArrayList, Integer>> crossExpectedResults = new ArrayList, Integer>>(); @@ -55,7 +52,8 @@ public long getTimestamp(T value) { @Test public void test() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); env.setBufferTimeout(1); TestListResultSink, Integer>> joinResultSink = diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java index fc3e36fb3753c..b7df2ecfe4dd9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java @@ -30,11 +30,11 @@ import org.apache.flink.streaming.api.datastream.SplitDataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestListResultSink; -import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; -public class DirectedOutputTest { +public class DirectedOutputTest extends StreamingMultipleProgramsTestBase { private static final String TEN = "ten"; private static final String ODD = "odd"; @@ -94,7 +94,8 @@ private void readObject(java.io.ObjectInputStream in) throws IOException, @Test public void outputSelectorTest() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(3, 32); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(3); TestListResultSink evenSink = new TestListResultSink(); TestListResultSink oddAndTenSink = new TestListResultSink(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java index 46e43272a8fa1..f2c253c55202b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.FoldFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -44,7 +43,6 @@ import org.apache.flink.streaming.api.windowing.helper.Time; import org.apache.flink.streaming.api.windowing.helper.Timestamp; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; -import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.Collector; import org.junit.After; import org.junit.Before; @@ -131,7 +129,6 @@ public Tuple2> map( } }) - .setParallelism(1).filter(new FilterFunction >>() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java index 767eaa42df283..39a13b3ab91aa 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java @@ -25,14 +25,15 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.junit.Test; -public class SlotAllocationTest { +public class SlotAllocationTest extends StreamingMultipleProgramsTestBase{ @SuppressWarnings("serial") @Test public void test() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(8); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); FilterFunction dummyFilter = new FilterFunction() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java new file mode 100644 index 0000000000000..fb2ef56f2adcc --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java @@ -0,0 +1,179 @@ +/** + * 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.streaming.api.graph; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; +import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; +import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; +import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; +import org.apache.flink.streaming.util.EvenOddOutputSelector; +import org.apache.flink.streaming.util.NoOpIntMap; +import org.apache.flink.streaming.util.NoOpSink; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link StreamGraphGenerator}. This only tests correct translation of split/select, + * union, partitioning since the other translation routines are tested already in operation + * specific tests, for example in {@link org.apache.flink.streaming.api.IterateTest} for + * iterations. + */ +public class StreamGraphGeneratorTest extends StreamingMultipleProgramsTestBase { + + /** + * This tests whether virtual Transformations behave correctly. + * + *

    + * Verifies that partitioning, output selector, selected names are correctly set in the + * StreamGraph when they are intermixed. + */ + @Test + public void testVirtualTransformations() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream source = env.fromElements(1, 10); + + DataStream rebalanceMap = source.rebalance().map(new NoOpIntMap()); + + // verify that only the partitioning that was set last is used + DataStream broadcastMap = rebalanceMap + .forward() + .global() + .broadcast() + .map(new NoOpIntMap()); + + broadcastMap.addSink(new NoOpSink()); + + // verify that partitioning is preserved across union and split/select + EvenOddOutputSelector selector1 = new EvenOddOutputSelector(); + EvenOddOutputSelector selector2 = new EvenOddOutputSelector(); + EvenOddOutputSelector selector3 = new EvenOddOutputSelector(); + + DataStream map1Operator = rebalanceMap + .map(new NoOpIntMap()); + + DataStream map1 = map1Operator + .broadcast() + .split(selector1) + .select("even"); + + DataStream map2Operator = rebalanceMap + .map(new NoOpIntMap()); + + DataStream map2 = map2Operator + .split(selector2) + .select("odd") + .global(); + + DataStream map3Operator = rebalanceMap + .map(new NoOpIntMap()); + + DataStream map3 = map3Operator + .global() + .split(selector3) + .select("even") + .shuffle(); + + + SingleOutputStreamOperator unionedMap = map1.union(map2).union(map3) + .map(new NoOpIntMap()); + + unionedMap.addSink(new NoOpSink()); + + StreamGraph graph = env.getStreamGraph(); + + // rebalanceMap + assertTrue(graph.getStreamNode(rebalanceMap.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner); + + // verify that only last partitioning takes precedence + assertTrue(graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); + assertEquals(rebalanceMap.getId(), graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getSourceVertex().getId()); + + // verify that partitioning in unions is preserved and that it works across split/select + assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); + assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even")); + assertTrue(graph.getStreamNode(map1Operator.getId()).getOutputSelectors().contains(selector1)); + + assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof GlobalPartitioner); + assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("odd")); + assertTrue(graph.getStreamNode(map2Operator.getId()).getOutputSelectors().contains(selector2)); + + assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof ShufflePartitioner); + assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even")); + assertTrue(graph.getStreamNode(map3Operator.getId()).getOutputSelectors().contains(selector3)); + } + + /** + * This tests whether virtual Transformations behave correctly. + * + * Checks whether output selector, partitioning works correctly when applied on a union. + */ + @Test + public void testVirtualTransformations2() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream source = env.fromElements(1, 10); + + DataStream rebalanceMap = source.rebalance().map(new NoOpIntMap()); + + DataStream map1 = rebalanceMap + .map(new NoOpIntMap()); + + DataStream map2 = rebalanceMap + .map(new NoOpIntMap()); + + DataStream map3 = rebalanceMap + .map(new NoOpIntMap()); + + EvenOddOutputSelector selector = new EvenOddOutputSelector(); + + SingleOutputStreamOperator unionedMap = map1.union(map2).union(map3) + .broadcast() + .split(selector) + .select("foo") + .map(new NoOpIntMap()); + + unionedMap.addSink(new NoOpSink()); + + StreamGraph graph = env.getStreamGraph(); + + // verify that the properties are correctly set on all input operators + assertTrue(graph.getStreamNode(map1.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); + assertTrue(graph.getStreamNode(map1.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo")); + assertTrue(graph.getStreamNode(map1.getId()).getOutputSelectors().contains(selector)); + + assertTrue(graph.getStreamNode(map2.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); + assertTrue(graph.getStreamNode(map2.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo")); + assertTrue(graph.getStreamNode(map2.getId()).getOutputSelectors().contains(selector)); + + assertTrue(graph.getStreamNode(map3.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); + assertTrue(graph.getStreamNode(map3.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo")); + assertTrue(graph.getStreamNode(map3.getId()).getOutputSelectors().contains(selector)); + + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java index 4e7c963c4829f..e8064289188d7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java @@ -21,6 +21,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; @@ -28,7 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class StreamingJobGraphGeneratorTest { +public class StreamingJobGraphGeneratorTest extends StreamingMultipleProgramsTestBase { private static final Logger LOG = LoggerFactory.getLogger(StreamingJobGraphGeneratorTest.class); @Test @@ -36,8 +38,9 @@ public void testExecutionConfigSerialization() throws IOException, ClassNotFound final long seed = System.currentTimeMillis(); LOG.info("Test seed: {}", new Long(seed)); final Random r = new Random(seed); - - TestStreamEnvironment env = new TestStreamEnvironment(4, 32); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamGraph streamingJob = new StreamGraph(env); StreamingJobGraphGenerator compiler = new StreamingJobGraphGenerator(streamingJob); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java index e8f0a036f0d52..bb9dad7e771eb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java @@ -40,6 +40,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; @@ -52,8 +53,7 @@ *

  • Watermarks are correctly forwarded
  • * */ -public class StreamProjectTest implements Serializable { - private static final long serialVersionUID = 1L; +public class StreamProjectTest extends StreamingMultipleProgramsTestBase { @Test public void testProject() throws Exception { @@ -95,7 +95,6 @@ public void testProject() throws Exception { // tests using projection from the API without explicitly specifying the types - private static final long MEMORY_SIZE = 32; private static HashSet> expected = new HashSet>(); private static HashSet> actual = new HashSet>(); @@ -106,7 +105,8 @@ public void APIWithoutTypesTest() { expected.add(new Tuple2(i, i.doubleValue())); } - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); env.generateSequence(1, 10).map(new MapFunction>() { private static final long serialVersionUID = 1L; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java index b8e9619e7eead..4c644a97d08b4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java @@ -30,10 +30,12 @@ import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.NoOpSink; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness; import org.apache.flink.util.Collector; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; /** @@ -169,21 +171,4 @@ public void flatMap2(Integer value, Collector out) throws Exception { } } - - @SuppressWarnings("unchecked") - @Test - public void multipleInputTest() { - LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1); - - DataStream ds1 = env.fromElements(1, 3, 5); - DataStream ds2 = env.fromElements(2, 4).union(ds1); - - try { - ds1.forward().union(ds2); - fail(); - } catch (RuntimeException e) { - // expected - } - - } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java index 5986a30929c79..512a0df77a28d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java @@ -31,19 +31,17 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.windowing.helper.Timestamp; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.Collector; import org.junit.Test; -public class SelfConnectionTest implements Serializable { - - private static final long serialVersionUID = 1L; - - private final int MEMORY_SIZE = 32; +public class SelfConnectionTest extends StreamingMultipleProgramsTestBase { private static List expected; @@ -51,20 +49,12 @@ public class SelfConnectionTest implements Serializable { @Test public void sameDataStreamTest() { - StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(3); TestListResultSink resultSink = new TestListResultSink(); - Timestamp timeStamp = new Timestamp() { - - private static final long serialVersionUID = 1L; - - @Override - public long getTimestamp(Integer value) { - return value; - } - - }; + Timestamp timeStamp = new IntegerTimestamp(); KeySelector keySelector = new KeySelector() { @@ -79,7 +69,7 @@ public Integer getKey(Integer value) throws Exception { DataStream src = env.fromElements(1, 3, 5); @SuppressWarnings("unused") - DataStream> dataStream = + DataStreamSink> dataStream = src.join(src).onWindow(50L, timeStamp, timeStamp).where(keySelector).equalTo(keySelector) .map(new MapFunction, String>() { @@ -107,8 +97,8 @@ public String map(Tuple2 value) throws Exception { assertEquals(expected, result); } catch (Exception e) { - fail(); e.printStackTrace(); + fail(); } } @@ -120,7 +110,8 @@ public void differentDataStreamSameChain() { TestListResultSink resultSink = new TestListResultSink(); - StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); DataStream src = env.fromElements(1, 3, 5); @@ -175,7 +166,8 @@ public void differentDataStreamDifferentChain() { TestListResultSink resultSink = new TestListResultSink(); - StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(3); DataStream src = env.fromElements(1, 3, 5).disableChaining(); @@ -248,4 +240,15 @@ public String map2(Long value) { assertEquals(expected, result); } + + private static class IntegerTimestamp implements Timestamp { + + private static final long serialVersionUID = 1L; + + @Override + public long getTimestamp(Integer value) { + return value; + } + + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java index 5e6ffa28fb882..db093739bc08a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java @@ -40,14 +40,12 @@ import org.apache.flink.streaming.api.windowing.helper.FullStream; import org.apache.flink.streaming.api.windowing.helper.Time; import org.apache.flink.streaming.api.windowing.helper.Timestamp; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.Collector; import org.junit.Test; -public class WindowingITCase implements Serializable { - - private static final long serialVersionUID = 1L; - private static final Integer MEMORYSIZE = 32; +public class WindowingITCase extends StreamingMultipleProgramsTestBase { @SuppressWarnings("serial") public static class ModKey implements KeySelector { @@ -98,17 +96,10 @@ public void test() throws Exception { KeySelector key = new ModKey(2); - Timestamp ts = new Timestamp() { - - private static final long serialVersionUID = 1L; + Timestamp ts = new IntegerTimestamp(); - @Override - public long getTimestamp(Integer value) { - return value; - } - }; - - StreamExecutionEnvironment env = new TestStreamEnvironment(2, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); env.disableOperatorChaining(); DataStream source = env.fromCollection(inputs); @@ -116,14 +107,18 @@ public long getTimestamp(Integer value) { source.window(Time.of(3, ts, 1)).every(Time.of(2, ts, 1)).sum(0).getDiscretizedStream() .addSink(new TestSink1()); - source.window(Time.of(4, ts, 1)).groupBy(new ModKey(2)).mapWindow(new IdentityWindowMap()) - .flatten().addSink(new TestSink2()); + source.window(Time.of(4, ts, 1)).groupBy(new ModKey(2)) + .mapWindow(new IdentityWindowMap()) + .flatten() + .addSink(new TestSink2()).name("TESTSIUNK2"); source.groupBy(key).window(Time.of(4, ts, 1)).sum(0).getDiscretizedStream() .addSink(new TestSink4()); source.groupBy(new ModKey(3)).window(Count.of(2)).groupBy(new ModKey(2)) - .mapWindow(new IdentityWindowMap()).flatten().addSink(new TestSink5()); + .mapWindow(new IdentityWindowMap()) + .flatten() + .addSink(new TestSink5()); source.window(Time.of(2, ts)).every(Time.of(3, ts)).min(0).getDiscretizedStream() .addSink(new TestSink3()); @@ -131,11 +126,13 @@ public long getTimestamp(Integer value) { source.groupBy(key).window(Time.of(4, ts, 1)).max(0).getDiscretizedStream() .addSink(new TestSink6()); - source.window(Time.of(5, ts, 1)).mapWindow(new IdentityWindowMap()).flatten() + source.window(Time.of(5, ts, 1)).mapWindow(new IdentityWindowMap()) + .flatten() .addSink(new TestSink7()); source.window(Time.of(5, ts, 1)).every(Time.of(4, ts, 1)).groupBy(new ModKey(2)).sum(0) - .getDiscretizedStream().addSink(new TestSink8()); + .getDiscretizedStream() + .addSink(new TestSink8()); try { source.window(FullStream.window()).every(Count.of(2)).getDiscretizedStream(); @@ -156,7 +153,8 @@ public long getTimestamp(Integer value) { source.every(Count.of(4)).sum(0).getDiscretizedStream().addSink(new TestSink11()); source.window(FullStream.window()).every(Count.of(4)).groupBy(key).sum(0) - .getDiscretizedStream().addSink(new TestSink12()); + .getDiscretizedStream() + .addSink(new TestSink12()); DataStream source2 = env.addSource(new ParallelSourceFunction() { private static final long serialVersionUID = 1L; @@ -202,12 +200,15 @@ public void run(SourceContext ctx) throws Exception { source3.window(Time.of(5, ts, 1)).groupBy(new ModKey(2)).sum(0).getDiscretizedStream() .addSink(new TestSink10()); - source.map(new MapFunction() { - @Override - public Integer map(Integer value) throws Exception { - return value; - } - }).every(Time.of(5, ts, 1)).sum(0).getDiscretizedStream().addSink(new TestSink13()); + source + .map(new MapFunction() { + @Override + public Integer map(Integer value) throws Exception { + return value; + } + }) + .every(Time.of(5, ts, 1)).sum(0).getDiscretizedStream() + .addSink(new TestSink13()); env.execute(); @@ -516,4 +517,13 @@ public void invoke(StreamWindow value) throws Exception { } + private static class IntegerTimestamp implements Timestamp { + + private static final long serialVersionUID = 1L; + + @Override + public long getTimestamp(Integer value) { + return value; + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java index 68e2a75109f0e..7ac5616bbfccb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java @@ -40,9 +40,9 @@ protected void testProgram() throws Exception { DataStream text = env.fromElements(WordCountData.TEXT); - DataStream> counts = - text.flatMap(new Tokenizer()) - .groupBy(0).sum(1); + DataStream> counts = text + .flatMap(new Tokenizer()) + .groupBy(0).sum(1); counts.writeAsCsv(resultPath); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java index 3c48b3f475b1b..6bbcea8134df7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java @@ -38,9 +38,9 @@ protected void testProgram() throws Exception { DataStream text = env.fromElements(WordCountData.TEXT); - DataStream> counts = - text.flatMap(new CsvOutputFormatITCase.Tokenizer()) - .groupBy(0).sum(1); + DataStream> counts = text + .flatMap(new CsvOutputFormatITCase.Tokenizer()) + .groupBy(0).sum(1); counts.writeAsText(resultPath); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java index 60db798ddcbf1..32b345573f9e2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java @@ -51,6 +51,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.InstantiationUtil; import org.junit.Test; @@ -62,7 +63,7 @@ * partitioned and non-partitioned user states. This test mimics the runtime * behavior of stateful stream operators. */ -public class StatefulOperatorTest { +public class StatefulOperatorTest extends StreamingMultipleProgramsTestBase { @Test public void simpleStateTest() throws Exception { @@ -104,7 +105,8 @@ public void simpleStateTest() throws Exception { @Test public void apiTest() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(3, 32); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(3); KeyedDataStream keyedStream = env.fromCollection(Arrays.asList(0, 1, 2, 3, 4, 5, 6)).keyBy(new ModKey(4)); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java index f45125b9c6bd0..122aa8a11a71e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java @@ -36,10 +36,11 @@ import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; -public class StreamVertexTest { +public class StreamVertexTest extends StreamingMultipleProgramsTestBase { private static Map data = new HashMap(); @@ -86,14 +87,12 @@ public void invoke(Tuple2 tuple) { } @SuppressWarnings("unused") - private static final int PARALLELISM = 1; private static final int SOURCE_PARALELISM = 1; - private static final long MEMORYSIZE = 32; @Test public void wrongJobGraph() { - LocalStreamEnvironment env = StreamExecutionEnvironment - .createLocalEnvironment(SOURCE_PARALELISM); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(SOURCE_PARALELISM); try { env.fromCollection(null); @@ -155,7 +154,8 @@ public void invoke(String value) { @Test public void coTest() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(SOURCE_PARALELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(SOURCE_PARALELISM); DataStream fromStringElements = env.fromElements("aa", "bb", "cc"); DataStream generatedSequence = env.generateSequence(0, 3); @@ -171,7 +171,8 @@ public void coTest() throws Exception { @Test public void runStream() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(SOURCE_PARALELISM, MEMORYSIZE); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(SOURCE_PARALELISM); env.addSource(new MySource()).setParallelism(SOURCE_PARALELISM).map(new MyTask()) .addSink(new MySink()); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java index 89679ea12bd31..9d9d47b486856 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java @@ -24,12 +24,13 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.junit.Test; import static org.junit.Assert.*; @SuppressWarnings("serial") -public class TranslationTest { +public class TranslationTest extends StreamingMultipleProgramsTestBase { @Test public void testCheckpointModeTranslation() { @@ -66,7 +67,8 @@ private static StreamExecutionEnvironment getSimpleJob() { env.generateSequence(1, 10000000) .addSink(new SinkFunction() { @Override - public void invoke(Long value) {} + public void invoke(Long value) { + } }); return env; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java index 856f7aa78d603..f7bd739769d24 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java @@ -27,14 +27,14 @@ public class ForwardPartitionerTest { - private RebalancePartitioner forwardPartitioner; + private ForwardPartitioner forwardPartitioner; private StreamRecord streamRecord = new StreamRecord(null); private SerializationDelegate> sd = new SerializationDelegate>( null); @Before public void setPartitioner() { - forwardPartitioner = new RebalancePartitioner(true); + forwardPartitioner = new ForwardPartitioner(); } @Test @@ -49,7 +49,7 @@ public void testSelectChannelsLength() { public void testSelectChannelsInterval() { sd.setInstance(streamRecord); assertEquals(0, forwardPartitioner.selectChannels(sd, 1)[0]); - assertEquals(1, forwardPartitioner.selectChannels(sd, 2)[0]); - assertEquals(2, forwardPartitioner.selectChannels(sd, 1024)[0]); + assertEquals(0, forwardPartitioner.selectChannels(sd, 2)[0]); + assertEquals(0, forwardPartitioner.selectChannels(sd, 1024)[0]); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java similarity index 74% rename from flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java rename to flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java index 05541f5eec2c6..6dbf932da23f7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java @@ -27,9 +27,9 @@ import org.junit.Before; import org.junit.Test; -public class FieldsPartitionerTest { +public class HashPartitionerTest { - private FieldsPartitioner> fieldsPartitioner; + private HashPartitioner> hashPartitioner; private StreamRecord> streamRecord1 = new StreamRecord>(new Tuple2("test", 0)); private StreamRecord> streamRecord2 = new StreamRecord>(new Tuple2("test", 42)); private SerializationDelegate>> sd1 = new SerializationDelegate>>(null); @@ -37,7 +37,7 @@ public class FieldsPartitionerTest { @Before public void setPartitioner() { - fieldsPartitioner = new FieldsPartitioner>(new KeySelector, String>() { + hashPartitioner = new HashPartitioner>(new KeySelector, String>() { private static final long serialVersionUID = 1L; @@ -51,9 +51,9 @@ public String getKey(Tuple2 value) throws Exception { @Test public void testSelectChannelsLength() { sd1.setInstance(streamRecord1); - assertEquals(1, fieldsPartitioner.selectChannels(sd1, 1).length); - assertEquals(1, fieldsPartitioner.selectChannels(sd1, 2).length); - assertEquals(1, fieldsPartitioner.selectChannels(sd1, 1024).length); + assertEquals(1, hashPartitioner.selectChannels(sd1, 1).length); + assertEquals(1, hashPartitioner.selectChannels(sd1, 2).length); + assertEquals(1, hashPartitioner.selectChannels(sd1, 1024).length); } @Test @@ -61,11 +61,11 @@ public void testSelectChannelsGrouping() { sd1.setInstance(streamRecord1); sd2.setInstance(streamRecord2); - assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 1), - fieldsPartitioner.selectChannels(sd2, 1)); - assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 2), - fieldsPartitioner.selectChannels(sd2, 2)); - assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 1024), - fieldsPartitioner.selectChannels(sd2, 1024)); + assertArrayEquals(hashPartitioner.selectChannels(sd1, 1), + hashPartitioner.selectChannels(sd2, 1)); + assertArrayEquals(hashPartitioner.selectChannels(sd1, 2), + hashPartitioner.selectChannels(sd2, 2)); + assertArrayEquals(hashPartitioner.selectChannels(sd1, 1024), + hashPartitioner.selectChannels(sd2, 1024)); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java similarity index 95% rename from flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java rename to flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java index 2643bbac11a09..aa70e8ac6b6d5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java @@ -25,7 +25,7 @@ import org.junit.Before; import org.junit.Test; -public class DistributePartitionerTest { +public class RebalancePartitionerTest { private RebalancePartitioner distributePartitioner; private StreamRecord streamRecord = new StreamRecord(null); @@ -34,7 +34,7 @@ public class DistributePartitionerTest { @Before public void setPartitioner() { - distributePartitioner = new RebalancePartitioner(false); + distributePartitioner = new RebalancePartitioner(); } @Test diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java index 52de8aacd1d14..a41cb8cd13753 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.NoOpSink; import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.junit.AfterClass; import org.junit.Assert; @@ -111,7 +112,8 @@ public void testWatermarkPropagation() throws Exception { source1 .map(new IdentityMap()) .connect(source2).map(new IdentityCoMap()) - .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator()); + .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator()) + .addSink(new NoOpSink()); env.execute(); @@ -149,7 +151,9 @@ public void testTimestampHandling() throws Exception { source1 .map(new IdentityMap()) .connect(source2).map(new IdentityCoMap()) - .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator()); + .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator()) + .addSink(new NoOpSink()); + env.execute(); } @@ -176,7 +180,9 @@ public void testDisabledTimestamps() throws Exception { source1 .map(new IdentityMap()) .connect(source2).map(new IdentityCoMap()) - .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator()); + .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator()) + .addSink(new NoOpSink()); + env.execute(); } @@ -194,7 +200,8 @@ public void testEventTimeSourceEmitWithoutTimestamp() throws Exception { DataStream source1 = env.addSource(new MyErroneousTimestampSource()); source1 - .map(new IdentityMap()); + .map(new IdentityMap()) + .addSink(new NoOpSink()); env.execute(); } @@ -212,7 +219,8 @@ public void testSourceEmitWithTimestamp() throws Exception { DataStream source1 = env.addSource(new MyErroneousSource()); source1 - .map(new IdentityMap()); + .map(new IdentityMap()) + .addSink(new NoOpSink()); env.execute(); } @@ -230,7 +238,8 @@ public void testSourceEmitWatermark() throws Exception { DataStream source1 = env.addSource(new MyErroneousWatermarkSource()); source1 - .map(new IdentityMap()); + .map(new IdentityMap()) + .addSink(new NoOpSink()); env.execute(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java new file mode 100644 index 0000000000000..9820ef81a2ef2 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java @@ -0,0 +1,31 @@ +/** + * 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.streaming.util; + +import org.apache.flink.streaming.api.collector.selector.OutputSelector; + +import java.util.Arrays; + +public class EvenOddOutputSelector implements OutputSelector { + private static final long serialVersionUID = 1L; + + @Override + public Iterable select(Integer value) { + return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd"); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java new file mode 100644 index 0000000000000..bcb569121d66a --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java @@ -0,0 +1,28 @@ +/** + * 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.streaming.util; + +import org.apache.flink.api.common.functions.MapFunction; + +public class NoOpIntMap implements MapFunction { + private static final long serialVersionUID = 1L; + + public Integer map(Integer value) throws Exception { + return value; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java new file mode 100644 index 0000000000000..d39812146ffee --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java @@ -0,0 +1,26 @@ +/** + * 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.streaming.util; + +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; + +public final class NoOpSink extends RichSinkFunction { + public void invoke(T tuple) { + + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java new file mode 100644 index 0000000000000..a46ff55694e62 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java @@ -0,0 +1,42 @@ +/** + * 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.streaming.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertTrue; + +public final class ReceiveCheckNoOpSink extends RichSinkFunction { + private List received; + + public void invoke(T tuple) { + received.add(tuple); + } + + public void open(Configuration conf) { + received = new ArrayList(); + } + + public void close() { + assertTrue(received.size() > 0); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java index 1a6ef0fa24429..fbcda1c89b1e9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.util; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -44,7 +45,7 @@ public TestStreamEnvironment(int parallelism, long memorySize){ } public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism){ - this.executor = executor; + this.executor = Preconditions.checkNotNull(executor); setDefaultLocalParallelism(parallelism); setParallelism(parallelism); } @@ -56,7 +57,8 @@ public JobExecutionResult execute() throws Exception { @Override public JobExecutionResult execute(String jobName) throws Exception { - return execute(streamGraph.getJobGraph(jobName)); + JobExecutionResult result = execute(getStreamGraph().getJobGraph(jobName)); + return result; } public JobExecutionResult execute(JobGraph jobGraph) throws Exception { @@ -81,6 +83,7 @@ public JobExecutionResult execute(JobGraph jobGraph) throws Exception { throw e; } } finally { + transformations.clear(); if (internalExecutor){ executor.shutdown(); } @@ -161,7 +164,7 @@ public StreamExecutionEnvironment createExecutionEnvironment() { } }; - initializeFromFactory(factory); + initializeContextEnvironment(factory); } } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 5e62331a3dda5..2bb6a6a6d2c63 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -82,10 +82,6 @@ class DataStream[T](javaStream: JavaStream[T]) { */ def getParallelism = javaStream.getParallelism - def getPartitioner = javaStream.getPartitioner - - def getSelectedNames = javaStream.getSelectedNames - /** * Returns the execution config. */ @@ -403,7 +399,7 @@ class DataStream[T](javaStream: JavaStream[T]) { val iterativeStream = javaStream.iterate(maxWaitTimeMillis) val (feedback, output) = stepFunction(new DataStream[T](iterativeStream)) - iterativeStream.closeWith(feedback.getJavaStream, keepPartitioning) + iterativeStream.closeWith(feedback.getJavaStream) output } @@ -703,7 +699,7 @@ class DataStream[T](javaStream: JavaStream[T]) { * written. * */ - def print(): DataStream[T] = javaStream.print() + def print(): DataStreamSink[T] = javaStream.print() /** * Writes a DataStream to the standard output stream (stderr). @@ -722,7 +718,7 @@ class DataStream[T](javaStream: JavaStream[T]) { * is written. * */ - def writeAsText(path: String, millis: Long = 0): DataStream[T] = + def writeAsText(path: String, millis: Long = 0): DataStreamSink[T] = javaStream.writeAsText(path, millis) /** @@ -737,7 +733,7 @@ class DataStream[T](javaStream: JavaStream[T]) { millis: Long = 0, rowDelimiter: String = ScalaCsvOutputFormat.DEFAULT_LINE_DELIMITER, fieldDelimiter: String = ScalaCsvOutputFormat.DEFAULT_FIELD_DELIMITER, - writeMode: FileSystem.WriteMode = null): DataStream[T] = { + writeMode: FileSystem.WriteMode = null): DataStreamSink[T] = { require(javaStream.getType.isTupleType, "CSV output can only be used with Tuple DataSets.") val of = new ScalaCsvOutputFormat[Product](new Path(path), rowDelimiter, fieldDelimiter) if (writeMode != null) { @@ -758,8 +754,12 @@ class DataStream[T](javaStream: JavaStream[T]) { * Writes the DataStream to a socket as a byte array. The format of the output is * specified by a [[SerializationSchema]]. */ - def writeToSocket(hostname: String, port: Integer, schema: SerializationSchema[T, Array[Byte]]): - DataStream[T] = javaStream.writeToSocket(hostname, port, schema) + def writeToSocket( + hostname: String, + port: Integer, + schema: SerializationSchema[T, Array[Byte]]): DataStreamSink[T] = { + javaStream.writeToSocket(hostname, port, schema) + } /** * Adds the given sink to this DataStream. Only streams with sinks added @@ -767,7 +767,7 @@ class DataStream[T](javaStream: JavaStream[T]) { * method is called. * */ - def addSink(sinkFunction: SinkFunction[T]): DataStream[T] = + def addSink(sinkFunction: SinkFunction[T]): DataStreamSink[T] = javaStream.addSink(sinkFunction) /** @@ -776,7 +776,7 @@ class DataStream[T](javaStream: JavaStream[T]) { * method is called. * */ - def addSink(fun: T => Unit): DataStream[T] = { + def addSink(fun: T => Unit): DataStreamSink[T] = { if (fun == null) { throw new NullPointerException("Sink function must not be null.") } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala index 703321861b46c..0e01eee0da72c 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala @@ -82,18 +82,14 @@ object StreamCrossOperator { def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = { val cleanCrossWindowFunction = clean(getCrossWindowFunction(op, fun)) - val operator = new CoStreamWindow[I1, I2, R]( + + op.input1.connect(op.input2).addGeneralWindowCombine( cleanCrossWindowFunction, + implicitly[TypeInformation[R]], op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2) - - javaStream.getExecutionEnvironment().getStreamGraph().setOperator(javaStream.getId(), - operator) - - val js = javaStream.asInstanceOf[SingleOutputStreamOperator[R,_]] - js.returns(implicitly[TypeInformation[R]]).asInstanceOf[SingleOutputStreamOperator[R,_]] } override def every(length: Long, timeUnit: TimeUnit): CrossWindow[I1, I2] = { diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala index 09329ca1df5ea..e872851b81548 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala @@ -182,8 +182,9 @@ object StreamJoinOperator { } } - class JoinedStream[I1, I2](jp: JoinPredicate[I1, I2], javaStream: JavaStream[(I1, I2)]) extends - DataStream[(I1, I2)](javaStream) { + class JoinedStream[I1, I2]( + jp: JoinPredicate[I1, I2], + javaStream: JavaStream[(I1, I2)]) extends DataStream[(I1, I2)](javaStream) { private val op = jp.op @@ -194,18 +195,15 @@ object StreamJoinOperator { def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = { val cleanFun = clean(getJoinWindowFunction(jp, fun)) - val operator = new CoStreamWindow[I1, I2, R]( - cleanFun, - op.windowSize, - op.slideInterval, - op.timeStamp1, - op.timeStamp2) - - javaStream.getExecutionEnvironment().getStreamGraph().setOperator(javaStream.getId(), - operator) - - val js = javaStream.asInstanceOf[SingleOutputStreamOperator[R,_]] - js.returns(implicitly[TypeInformation[R]]).asInstanceOf[SingleOutputStreamOperator[R,_]] + + op.input1.groupBy(jp.keys1).connect(op.input2.groupBy(jp.keys2)) + .addGeneralWindowCombine[R]( + cleanFun, + implicitly[TypeInformation[R]], + op.windowSize, + op.slideInterval, + op.timeStamp1, + op.timeStamp2) } } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala index 39a8fe6ffd059..8b4d527439273 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala @@ -28,21 +28,21 @@ import org.apache.flink.streaming.api.graph.{StreamEdge, StreamGraph} import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, StreamOperator} import org.apache.flink.streaming.api.windowing.helper.Count import org.apache.flink.streaming.runtime.partitioner._ +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.apache.flink.util.Collector import org.junit.Assert.fail import org.junit.Test import org.apache.flink.streaming.api.scala.function.StatefulFunction -class DataStreamTest { - - private val parallelism = 2 +class DataStreamTest extends StreamingMultipleProgramsTestBase { @Test def testNaming(): Unit = { - val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism) + val env = StreamExecutionEnvironment.getExecutionEnvironment - val source1 = env.generateSequence(0, 0).name("testSource1") - assert("testSource1" == source1.getName) + val source1Operator = env.generateSequence(0, 0).name("testSource1") + val source1 = source1Operator + assert("testSource1" == source1Operator.getName) val dataStream1 = source1 .map(x => 0L) @@ -70,7 +70,7 @@ class DataStreamTest { windowed.name("testWindowFold") assert("testWindowFold" == windowed.getName) - windowed.flatten() + windowed.flatten().print() val plan = env.getExecutionPlan @@ -88,8 +88,7 @@ class DataStreamTest { */ @Test def testPartitioning(): Unit = { - val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism); - val graph: StreamGraph = env.getStreamGraph; + val env = StreamExecutionEnvironment.getExecutionEnvironment val src1: DataStream[(Long, Long)] = env.fromElements((0L, 0L)) val src2: DataStream[(Long, Long)] = env.fromElements((0L, 0L)) @@ -101,10 +100,14 @@ class DataStreamTest { val group3 = src1.groupBy("_1") val group4 = src1.groupBy(x => x._1) - assert(isPartitioned(graph.getStreamEdge(group1.getId, createDownStreamId(group1)))) - assert(isPartitioned(graph.getStreamEdge(group2.getId, createDownStreamId(group2)))) - assert(isPartitioned(graph.getStreamEdge(group3.getId, createDownStreamId(group3)))) - assert(isPartitioned(graph.getStreamEdge(group4.getId, createDownStreamId(group4)))) + val gid1 = createDownStreamId(group1) + val gid2 = createDownStreamId(group2) + val gid3 = createDownStreamId(group3) + val gid4 = createDownStreamId(group4) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid1))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid2))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid3))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid4))) //Testing DataStream partitioning val partition1: DataStream[_] = src1.partitionByHash(0) @@ -112,10 +115,15 @@ class DataStreamTest { val partition3: DataStream[_] = src1.partitionByHash("_1") val partition4: DataStream[_] = src1.partitionByHash((x : (Long, Long)) => x._1); - assert(isPartitioned(graph.getStreamEdge(partition1.getId, createDownStreamId(partition1)))) - assert(isPartitioned(graph.getStreamEdge(partition2.getId, createDownStreamId(partition2)))) - assert(isPartitioned(graph.getStreamEdge(partition3.getId, createDownStreamId(partition3)))) - assert(isPartitioned(graph.getStreamEdge(partition4.getId, createDownStreamId(partition4)))) + val pid1 = createDownStreamId(partition1) + val pid2 = createDownStreamId(partition2) + val pid3 = createDownStreamId(partition3) + val pid4 = createDownStreamId(partition4) + + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid1))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid2))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid3))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid4))) // Testing DataStream custom partitioning val longPartitioner: Partitioner[Long] = new Partitioner[Long] { @@ -129,12 +137,12 @@ class DataStreamTest { val customPartition4: DataStream[_] = src1.partitionCustom(longPartitioner, (x : (Long, Long)) => x._1) - assert(isCustomPartitioned( - graph.getStreamEdge(customPartition1.getId, createDownStreamId(customPartition1)))) - assert(isCustomPartitioned( - graph.getStreamEdge(customPartition3.getId, createDownStreamId(customPartition3)))) - assert(isCustomPartitioned( - graph.getStreamEdge(customPartition4.getId, createDownStreamId(customPartition4)))) + val cpid1 = createDownStreamId(customPartition1) + val cpid2 = createDownStreamId(customPartition3) + val cpid3 = createDownStreamId(customPartition4) + assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid1))) + assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid2))) + assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid3))) //Testing ConnectedDataStream grouping val connectedGroup1: ConnectedDataStream[_, _] = connected.groupBy(0, 0) @@ -153,20 +161,20 @@ class DataStreamTest { val connectedGroup5: ConnectedDataStream[_, _] = connected.groupBy(x => x._1, x => x._1) val downStreamId5: Integer = createDownStreamId(connectedGroup5) - assert(isPartitioned(graph.getStreamEdge(connectedGroup1.getFirst.getId, downStreamId1))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup1.getSecond.getId, downStreamId1))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId1))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId1))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup2.getFirst.getId, downStreamId2))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup2.getSecond.getId, downStreamId2))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId2))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId2))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup3.getFirst.getId, downStreamId3))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup3.getSecond.getId, downStreamId3))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId3))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId3))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup4.getFirst.getId, downStreamId4))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup4.getSecond.getId, downStreamId4))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId4))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId4))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup5.getFirst.getId, downStreamId5))) - assert(isPartitioned(graph.getStreamEdge(connectedGroup5.getSecond.getId, downStreamId5))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId5))) + assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId5))) //Testing ConnectedDataStream partitioning val connectedPartition1: ConnectedDataStream[_, _] = connected.partitionByHash(0, 0) @@ -188,38 +196,38 @@ class DataStreamTest { val connectDownStreamId5: Integer = createDownStreamId(connectedPartition5) assert( - isPartitioned(graph.getStreamEdge(connectedPartition1.getFirst.getId, connectDownStreamId1)) + isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId1)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition1.getSecond.getId, connectDownStreamId1)) + isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId1)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition2.getFirst.getId, connectDownStreamId2)) + isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId2)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition2.getSecond.getId, connectDownStreamId2)) + isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId2)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition3.getFirst.getId, connectDownStreamId3)) + isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId3)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition3.getSecond.getId, connectDownStreamId3)) + isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId3)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition4.getFirst.getId, connectDownStreamId4)) + isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId4)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition4.getSecond.getId, connectDownStreamId4)) + isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId4)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition5.getFirst.getId, connectDownStreamId5)) + isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId5)) ) assert( - isPartitioned(graph.getStreamEdge(connectedPartition5.getSecond.getId, connectDownStreamId5)) + isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId5)) ) } @@ -230,20 +238,19 @@ class DataStreamTest { def testParallelism { val env: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment(10) - val graph: StreamGraph = env.getStreamGraph - val src = env.fromElements(new Tuple2[Long, Long](0L, 0L)) val map = src.map(x => 0L) val windowed: DataStream[Long] = map .window(Count.of(10)) .foldWindow(0L, (x: Long, y: Long) => 0L) .flatten + windowed.print() val sink = map.addSink(x => {}) - assert(1 == graph.getStreamNode(src.getId).getParallelism) - assert(10 == graph.getStreamNode(map.getId).getParallelism) - assert(10 == graph.getStreamNode(windowed.getId).getParallelism) - assert(10 == graph.getStreamNode(sink.getId).getParallelism) + assert(1 == env.getStreamGraph.getStreamNode(src.getId).getParallelism) + assert(10 == env.getStreamGraph.getStreamNode(map.getId).getParallelism) + assert(1 == env.getStreamGraph.getStreamNode(windowed.getId).getParallelism) + assert(10 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism) try { src.setParallelism(3) @@ -255,28 +262,31 @@ class DataStreamTest { } env.setParallelism(7) - assert(1 == graph.getStreamNode(src.getId).getParallelism) - assert(7 == graph.getStreamNode(map.getId).getParallelism) - assert(7 == graph.getStreamNode(windowed.getId).getParallelism) - assert(7 == graph.getStreamNode(sink.getId).getParallelism) + // the parallelism does not change since some windowing code takes the parallelism from + // input operations and that cannot change dynamically + assert(1 == env.getStreamGraph.getStreamNode(src.getId).getParallelism) + assert(10 == env.getStreamGraph.getStreamNode(map.getId).getParallelism) + assert(1 == env.getStreamGraph.getStreamNode(windowed.getId).getParallelism) + assert(10 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism) val parallelSource = env.generateSequence(0, 0) + parallelSource.print() - assert(7 == graph.getStreamNode(parallelSource.getId).getParallelism) + assert(7 == env.getStreamGraph.getStreamNode(parallelSource.getId).getParallelism) parallelSource.setParallelism(3) - assert(3 == graph.getStreamNode(parallelSource.getId).getParallelism) + assert(3 == env.getStreamGraph.getStreamNode(parallelSource.getId).getParallelism) map.setParallelism(2) - assert(2 == graph.getStreamNode(map.getId).getParallelism) + assert(2 == env.getStreamGraph.getStreamNode(map.getId).getParallelism) sink.setParallelism(4) - assert(4 == graph.getStreamNode(sink.getId).getParallelism) + assert(4 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism) } @Test def testTypeInfo { - val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism) + val env = StreamExecutionEnvironment.getExecutionEnvironment val src1: DataStream[Long] = env.generateSequence(0, 0) assert(TypeExtractor.getForClass(classOf[Long]) == src1.getType) @@ -299,15 +309,14 @@ class DataStreamTest { } @Test def operatorTest { - val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism) - - val streamGraph = env.getStreamGraph + val env = StreamExecutionEnvironment.getExecutionEnvironment val src = env.generateSequence(0, 0) val mapFunction = new MapFunction[Long, Int] { override def map(value: Long): Int = 0 - }; + } + val map = src.map(mapFunction) assert(mapFunction == getFunctionForDataStream(map)) assert(getFunctionForDataStream(map.map(x => 0)).isInstanceOf[MapFunction[_, _]]) @@ -364,7 +373,7 @@ class DataStreamTest { asInstanceOf[StatefulFunction[_, _, _]].partitioned) try { - streamGraph.getStreamEdge(map.getId, unionFilter.getId) + env.getStreamGraph.getStreamEdge(map.getId, unionFilter.getId) } catch { case e: Throwable => { @@ -373,7 +382,7 @@ class DataStreamTest { } try { - streamGraph.getStreamEdge(flatMap.getId, unionFilter.getId) + env.getStreamGraph.getStreamEdge(flatMap.getId, unionFilter.getId) } catch { case e: Throwable => { @@ -386,17 +395,19 @@ class DataStreamTest { } val split = unionFilter.split(outputSelector) - val outputSelectors = streamGraph.getStreamNode(split.getId).getOutputSelectors + split.print() + val outputSelectors = env.getStreamGraph.getStreamNode(unionFilter.getId).getOutputSelectors assert(1 == outputSelectors.size) assert(outputSelector == outputSelectors.get(0)) - unionFilter.split(x => List("a")) - val moreOutputSelectors = streamGraph.getStreamNode(split.getId).getOutputSelectors + unionFilter.split(x => List("a")).print() + val moreOutputSelectors = env.getStreamGraph.getStreamNode(unionFilter.getId).getOutputSelectors assert(2 == moreOutputSelectors.size) val select = split.select("a") val sink = select.print - val splitEdge = streamGraph.getStreamEdge(select.getId, sink.getId) + val splitEdge = + env.getStreamGraph.getStreamEdge(unionFilter.getId, sink.getTransformation.getId) assert("a" == splitEdge.getSelectedNames.get(0)) val foldFunction = new FoldFunction[Int, String] { @@ -421,7 +432,7 @@ class DataStreamTest { assert(coMapFunction == getFunctionForDataStream(coMap)) try { - streamGraph.getStreamEdge(fold.getId, coMap.getId) + env.getStreamGraph.getStreamEdge(fold.getId, coMap.getId) } catch { case e: Throwable => { @@ -429,7 +440,7 @@ class DataStreamTest { } } try { - streamGraph.getStreamEdge(flatMap.getId, coMap.getId) + env.getStreamGraph.getStreamEdge(flatMap.getId, coMap.getId) } catch { case e: Throwable => { @@ -440,51 +451,51 @@ class DataStreamTest { @Test def testChannelSelectors { - val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism) + val env = StreamExecutionEnvironment.getExecutionEnvironment - val streamGraph = env.getStreamGraph val src = env.generateSequence(0, 0) val broadcast = src.broadcast val broadcastSink = broadcast.print - val broadcastPartitioner = streamGraph - .getStreamEdge(broadcast.getId, broadcastSink.getId).getPartitioner + val broadcastPartitioner = env.getStreamGraph + .getStreamEdge(src.getId, broadcastSink.getTransformation.getId).getPartitioner assert(broadcastPartitioner.isInstanceOf[BroadcastPartitioner[_]]) val shuffle: DataStream[Long] = src.shuffle val shuffleSink = shuffle.print - val shufflePartitioner = streamGraph - .getStreamEdge(shuffle.getId, shuffleSink.getId).getPartitioner + val shufflePartitioner = env.getStreamGraph + .getStreamEdge(src.getId, shuffleSink.getTransformation.getId).getPartitioner assert(shufflePartitioner.isInstanceOf[ShufflePartitioner[_]]) val forward: DataStream[Long] = src.forward val forwardSink = forward.print - val forwardPartitioner = streamGraph - .getStreamEdge(forward.getId, forwardSink.getId).getPartitioner - assert(forwardPartitioner.isInstanceOf[RebalancePartitioner[_]]) + val forwardPartitioner = env.getStreamGraph + .getStreamEdge(src.getId, forwardSink.getTransformation.getId).getPartitioner + assert(forwardPartitioner.isInstanceOf[ForwardPartitioner[_]]) val rebalance: DataStream[Long] = src.rebalance val rebalanceSink = rebalance.print - val rebalancePartitioner = streamGraph - .getStreamEdge(rebalance.getId, rebalanceSink.getId).getPartitioner + val rebalancePartitioner = env.getStreamGraph + .getStreamEdge(src.getId, rebalanceSink.getTransformation.getId).getPartitioner assert(rebalancePartitioner.isInstanceOf[RebalancePartitioner[_]]) val global: DataStream[Long] = src.global val globalSink = global.print - val globalPartitioner = streamGraph - .getStreamEdge(global.getId, globalSink.getId).getPartitioner + val globalPartitioner = env.getStreamGraph + .getStreamEdge(src.getId, globalSink.getTransformation.getId).getPartitioner assert(globalPartitioner.isInstanceOf[GlobalPartitioner[_]]) } @Test def testIterations { val env = StreamExecutionEnvironment.getExecutionEnvironment - val source = env.fromElements(1, 2, 3) + // we need to rebalance before iteration + val source = env.fromElements(1, 2, 3).map { t: Int => t } val iterated = source.iterate((input: ConnectedDataStream[Int, String]) => { val head = input.map(i => (i + 1).toString, s => s) (head.filter(_ == "2"), head.filter(_ != "2")) - }, 1000) + }, 1000).print() val iterated2 = source.iterate((input: DataStream[Int]) => (input.map(_ + 1), input.map(_.toString)), 2000) @@ -493,7 +504,7 @@ class DataStreamTest { val invalid = source.iterate((input: ConnectedDataStream[Int, String]) => { val head = input.partitionByHash(1, 1).map(i => (i + 1).toString, s => s) (head.filter(_ == "2"), head.filter(_ != "2")) - }, 1000) + }, 1000).print() fail } catch { case uoe: UnsupportedOperationException => @@ -502,7 +513,7 @@ class DataStreamTest { val sg = env.getStreamGraph - assert(sg.getStreamLoops().size() == 2) + assert(sg.getIterationSourceSinkPairs().size() == 2) } ///////////////////////////////////////////////////////////// @@ -510,19 +521,21 @@ class DataStreamTest { ///////////////////////////////////////////////////////////// private def getFunctionForDataStream(dataStream: DataStream[_]): Function = { + dataStream.print() val operator = getOperatorForDataStream(dataStream) .asInstanceOf[AbstractUdfStreamOperator[_, _]] return operator.getUserFunction.asInstanceOf[Function] } private def getOperatorForDataStream(dataStream: DataStream[_]): StreamOperator[_] = { + dataStream.print() val env = dataStream.getJavaStream.getExecutionEnvironment val streamGraph: StreamGraph = env.getStreamGraph streamGraph.getStreamNode(dataStream.getId).getOperator } private def isPartitioned(edge: StreamEdge): Boolean = { - return edge.getPartitioner.isInstanceOf[FieldsPartitioner[_]] + return edge.getPartitioner.isInstanceOf[HashPartitioner[_]] } private def isCustomPartitioned(edge: StreamEdge): Boolean = { @@ -530,11 +543,13 @@ class DataStreamTest { } private def createDownStreamId(dataStream: DataStream[_]): Integer = { - return dataStream.print.getId + return dataStream.print.getTransformation.getId } private def createDownStreamId(dataStream: ConnectedDataStream[_, _]): Integer = { - return dataStream.map(x => 0, x => 0).getId + val m = dataStream.map(x => 0, x => 0) + m.print() + m.getId } } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala index 88b0f4f803283..0d573a9889b45 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala @@ -18,14 +18,16 @@ package org.apache.flink.streaming.api.scala import org.apache.flink.streaming.util.SocketOutputTestBase.DummyStringSchema +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema +import org.apache.flink.test.util.MultipleProgramsTestBase import scala.language.existentials /** * Test programs for built in output formats. Invoked from {@link OutputFormatTest}. */ -object OutputFormatTestPrograms { +object OutputFormatTestPrograms { def wordCountToText(input : String, outputPath : String) : Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala index 5fa007ceaecf2..650fd7ebef734 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala @@ -29,13 +29,13 @@ object StateTestPrograms { val env = StreamExecutionEnvironment.getExecutionEnvironment // test stateful map - env.generateSequence(0, 10).setParallelism(1). - mapWithState((in, count: Option[Long]) => + env.generateSequence(0, 10).setParallelism(1) + .mapWithState((in, count: Option[Long]) => count match { case Some(c) => ((in - c), Some(c + 1)) case None => (in, Some(1L)) - }).setParallelism(1). - addSink(new RichSinkFunction[Long]() { + }).setParallelism(1) + .addSink(new RichSinkFunction[Long]() { var allZero = true override def invoke(in: Long) = { if (in != 0) allZero = false @@ -50,8 +50,8 @@ object StateTestPrograms { s match { case Some(s) => (w.split(" ").toList.map(s + _), Some(w)) case None => (List(w), Some(w)) - }).setParallelism(1). - addSink(new RichSinkFunction[String]() { + }).setParallelism(1) + .addSink(new RichSinkFunction[String]() { val received = new HashSet[String]() override def invoke(in: String) = { received.add(in) } override def close() = { diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala index 7ebc16147831f..e7d99535c2eee 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala @@ -41,6 +41,7 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase { "org.apache.flink.streaming.api.datastream.DataStream.getType", "org.apache.flink.streaming.api.datastream.DataStream.copy", "org.apache.flink.streaming.api.datastream.DataStream.transform", + "org.apache.flink.streaming.api.datastream.DataStream.getTransformation", "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.copy", "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getExecutionEnvironment", "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getType1", @@ -50,6 +51,16 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase { "org.apache.flink.streaming.api.datastream.WindowedDataStream.getType", "org.apache.flink.streaming.api.datastream.WindowedDataStream.getExecutionConfig", + "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.isChainingEnabled", + "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." + + "getStateHandleProvider", + "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getCheckpointInterval", + "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.addOperator", + "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getCheckpointingMode", + "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." + + "isForceCheckpointing", + + // TypeHints are only needed for Java API, Scala API doesn't need them "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.returns" ) diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 020919e7a496f..ae31d95f4729c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -24,7 +24,6 @@ import akka.pattern.Patterns; import akka.testkit.JavaTestKit; import akka.util.Timeout; -import org.apache.flink.api.common.ExecutionMode; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; @@ -163,7 +162,7 @@ public void testBatch() throws Exception { @Test public void testStreaming() throws Exception { - StreamExecutionEnvironment env = new StreamJobExtractor(); + StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment(); env.setParallelism(1); DataStream input = env.fromCollection(inputData); @@ -171,9 +170,8 @@ public void testStreaming() throws Exception { .flatMap(new NotifyingMapper()) .write(new NotifyingOutputFormat(), 1000).disableChaining(); - env.execute(); - jobGraph = ((StreamJobExtractor) env).graph; + jobGraph = env.getStreamGraph().getJobGraph(); jobID = jobGraph.getJobID(); verifyResults(); @@ -346,6 +344,7 @@ private static boolean checkFlinkAccumulators(ExecutionAttemptID taskKey, int ex * UDF that notifies when it changes the accumulator values */ private static class NotifyingMapper extends RichFlatMapFunction { + private static final long serialVersionUID = 1L; private IntCounter counter = new IntCounter(); @@ -376,6 +375,7 @@ public void close() throws Exception { * Outputs format which notifies of accumulator changes and waits for the previous mapper. */ private static class NotifyingOutputFormat implements OutputFormat { + private static final long serialVersionUID = 1L; @Override public void configure(Configuration parameters) { @@ -439,9 +439,11 @@ public JobExecutionResult execute(String jobName) throws Exception { } - private static class StreamJobExtractor extends StreamExecutionEnvironment { - - private JobGraph graph = null; + /** + * This is used to for creating the example topology. {@link #execute} is never called, we + * only use this to call {@link #getStreamGraph()}. + */ + private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { @Override public JobExecutionResult execute() throws Exception { @@ -450,8 +452,7 @@ public JobExecutionResult execute() throws Exception { @Override public JobExecutionResult execute(String jobName) throws Exception { - graph = this.streamGraph.getJobGraph(); - return new JobExecutionResult(new JobID(), -1, null); + throw new RuntimeException("This should not be called."); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java index 04f9f3456d690..76c8e54a10606 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java @@ -39,7 +39,7 @@ public static void main(String[] args) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile); env.getConfig().disableSysoutLogging(); - DataStream text = env.fromElements(WordCountData.TEXT); + DataStream text = env.fromElements(WordCountData.TEXT).rebalance(); DataStream counts = text.flatMap(new Tokenizer()).groupBy("word").sum("frequency"); From d05d386aaa50fe88a18dbde8557d12eea874f592 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 20 Aug 2015 19:09:56 +0200 Subject: [PATCH 153/175] [hotfix] Some Java 7 cleanups in InstantiationUtil --- .../apache/flink/util/InstantiationUtil.java | 23 ++++--------------- 1 file changed, 5 insertions(+), 18 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index 6857c310151e3..9955422682a2a 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -138,7 +138,7 @@ public static T instantiate(Class clazz) { try { return clazz.newInstance(); } - catch (InstantiationException iex) { + catch (InstantiationException | IllegalAccessException iex) { // check for the common problem causes checkForInstantiation(clazz); @@ -147,15 +147,6 @@ public static T instantiate(Class clazz) { throw new RuntimeException("Could not instantiate type '" + clazz.getName() + "' due to an unspecified exception: " + iex.getMessage(), iex); } - catch (IllegalAccessException iaex) { - // check for the common problem causes - checkForInstantiation(clazz); - - // here we are, if non of the common causes was the problem. then the error was - // most likely an exception in the constructor or field initialization - throw new RuntimeException("Could not instantiate type '" + clazz.getName() + - "' due to an unspecified exception: " + iaex.getMessage(), iaex); - } catch (Throwable t) { String message = t.getMessage(); throw new RuntimeException("Could not instantiate type '" + clazz.getName() + @@ -172,9 +163,9 @@ public static T instantiate(Class clazz) { */ public static boolean hasPublicNullaryConstructor(Class clazz) { Constructor[] constructors = clazz.getConstructors(); - for (int i = 0; i < constructors.length; i++) { - if (constructors[i].getParameterTypes().length == 0 && - Modifier.isPublic(constructors[i].getModifiers())) { + for (Constructor constructor : constructors) { + if (constructor.getParameterTypes().length == 0 && + Modifier.isPublic(constructor.getModifiers())) { return true; } } @@ -310,14 +301,10 @@ public static Object deserializeObject(byte[] bytes, ClassLoader cl) throws IOEx public static byte[] serializeObject(Object o) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos); - try { + try (ObjectOutputStream oos = new ObjectOutputStream(baos)) { oos.writeObject(o); } - finally { - oos.close(); - } return baos.toByteArray(); } From 58421b848ac9190db3b7c86b5ee4f8a9fc977d90 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Thu, 20 Aug 2015 00:38:37 +0200 Subject: [PATCH 154/175] [FLINK-2540] [optimizer] [runtime] Propagate union batch exchanges to union inputs The DataExchangeMode of union nodes was not respected when translating an OptimizedPlan to a JobGraph. This could result in deadlocks, when a branched data flow was closed. Union nodes with a batch exchange will propagate their exchange mode to all inputs of their inputs when the JobGraph is generated. This closes #1036 --- .../apache/flink/optimizer/plan/Channel.java | 14 +- .../plantranslate/JobGraphGenerator.java | 10 + .../flink/test/UnionClosedBranchingTest.java | 192 ++++++++++++++++++ 3 files changed, 215 insertions(+), 1 deletion(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java index 4f8b1bee2ea5d..b139b621e9d58 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java @@ -36,6 +36,8 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; +import static com.google.common.base.Preconditions.checkNotNull; + /** * A Channel represents the result produced by an operator and the data exchange * before the consumption by the target operator. @@ -181,7 +183,17 @@ public void setShipStrategy(ShipStrategyType strategy, FieldList keys, } /** - * Gets the data exchange mode (batch / streaming) to use for the data + * Sets the data exchange mode (batch / pipelined) to use for the data + * exchange of this channel. + * + * @return The data exchange mode of this channel. + */ + public void setDataExchangeMode(DataExchangeMode dataExchangeMode) { + this.dataExchangeMode = checkNotNull(dataExchangeMode); + } + + /** + * Gets the data exchange mode (batch / pipelined) to use for the data * exchange of this channel. * * @return The data exchange mode of this channel. diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index d440063838776..943ec2e248fa2 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -593,6 +593,16 @@ private int translateChannel(Channel input, int inputIndex, JobVertex targetVert if (inputPlanNode instanceof NAryUnionPlanNode) { allInChannels = ((NAryUnionPlanNode) inputPlanNode).getListOfInputs().iterator(); + + // If the union node has a batch data exchange, we have to adopt the exchange mode of + // the inputs of the union as well, because the optimizer has a separate union + // node, which does not exist in the JobGraph. Otherwise, this can result in + // deadlocks when closing a branching flow at runtime. + if (input.getDataExchangeMode().equals(DataExchangeMode.BATCH)) { + for (Channel in : inputPlanNode.getInputs()) { + in.setDataExchangeMode(DataExchangeMode.BATCH); + } + } } else if (inputPlanNode instanceof BulkPartialSolutionPlanNode) { if (this.vertices.get(inputPlanNode) == null) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java b/flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java new file mode 100644 index 0000000000000..f7ea9119d7984 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java @@ -0,0 +1,192 @@ +/* + * 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.test; + +import org.apache.flink.api.common.ExecutionMode; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.optimizer.plan.Channel; +import org.apache.flink.optimizer.plan.DualInputPlanNode; +import org.apache.flink.optimizer.plan.NAryUnionPlanNode; +import org.apache.flink.optimizer.plan.OptimizedPlan; +import org.apache.flink.optimizer.plan.SinkPlanNode; +import org.apache.flink.optimizer.plan.SourcePlanNode; +import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; +import org.apache.flink.optimizer.util.CompilerTestBase; +import org.apache.flink.runtime.io.network.DataExchangeMode; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import static org.apache.flink.runtime.io.network.DataExchangeMode.BATCH; +import static org.apache.flink.runtime.io.network.DataExchangeMode.PIPELINED; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * This tests a fix for FLINK-2540. + * + *

    This test is necessary, because {@link NAryUnionPlanNode}s are not directly translated + * to runtime tasks by the {@link JobGraphGenerator}. Instead, the network stack unions the + * inputs by directly reading from multiple inputs (via {@link UnionInputGate}). + * + *

    + *   (source)-\        /-\
    + *            (union)-+  (join)
    + *   (source)-/        \-/
    + * 
    + * + * @see FLINK-2540 + */ +@RunWith(Parameterized.class) +@SuppressWarnings({"serial","unchecked"}) +public class UnionClosedBranchingTest extends CompilerTestBase { + + @Parameterized.Parameters + public static Collection params() { + Collection params = Arrays.asList(new Object[][]{ + {ExecutionMode.PIPELINED, PIPELINED, BATCH}, + {ExecutionMode.PIPELINED_FORCED, PIPELINED, PIPELINED}, + {ExecutionMode.BATCH, BATCH, BATCH}, + {ExecutionMode.BATCH_FORCED, BATCH, BATCH}, + }); + + // Make sure that changes to ExecutionMode are reflected in this test. + assertEquals(ExecutionMode.values().length, params.size()); + + return params; + } + + private final ExecutionMode executionMode; + + /** Expected {@link DataExchangeMode} from sources to union. */ + private final DataExchangeMode sourceToUnion; + + /** Expected {@link DataExchangeMode} from union to join. */ + private final DataExchangeMode unionToJoin; + + public UnionClosedBranchingTest( + ExecutionMode executionMode, + DataExchangeMode sourceToUnion, + DataExchangeMode unionToJoin) { + + this.executionMode = executionMode; + this.sourceToUnion = sourceToUnion; + this.unionToJoin = unionToJoin; + } + + @Test + public void testUnionClosedBranchingTest() throws Exception { + + // ----------------------------------------------------------------------------------------- + // Build test program + // ----------------------------------------------------------------------------------------- + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().setExecutionMode(executionMode); + env.setParallelism(4); + + DataSet> src1 = env.fromElements(new Tuple1<>(0), new Tuple1<>(1)); + + DataSet> src2 = env.fromElements(new Tuple1<>(0), new Tuple1<>(1)); + + DataSet> union = src1.union(src2); + + DataSet> join = union + .join(union).where(0).equalTo(0) + .projectFirst(0).projectSecond(0); + + join.output(new DiscardingOutputFormat>()); + + // ----------------------------------------------------------------------------------------- + // Verify optimized plan + // ----------------------------------------------------------------------------------------- + + OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); + + SinkPlanNode sinkNode = optimizedPlan.getDataSinks().iterator().next(); + + DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); + + // Verify that the compiler correctly sets the expected data exchange modes. + for (Channel channel : joinNode.getInputs()) { + assertEquals("Unexpected data exchange mode between union and join node.", + unionToJoin, channel.getDataExchangeMode()); + } + + for (SourcePlanNode src : optimizedPlan.getDataSources()) { + for (Channel channel : src.getOutgoingChannels()) { + assertEquals("Unexpected data exchange mode between source and union node.", + sourceToUnion, channel.getDataExchangeMode()); + } + } + + // ----------------------------------------------------------------------------------------- + // Verify generated JobGraph + // ----------------------------------------------------------------------------------------- + + JobGraphGenerator jgg = new JobGraphGenerator(); + JobGraph jobGraph = jgg.compileJobGraph(optimizedPlan); + + List vertices = jobGraph.getVerticesSortedTopologicallyFromSources(); + + // Sanity check for the test setup + assertEquals("Unexpected number of vertices created.", 4, vertices.size()); + + // Verify all sources + JobVertex[] sources = new JobVertex[]{vertices.get(0), vertices.get(1)}; + + for (JobVertex src : sources) { + // Sanity check + assertTrue("Unexpected vertex type. Test setup is broken.", src.isInputVertex()); + + // The union is not translated to an extra union task, but the join uses a union + // input gate to read multiple inputs. The source create a single result per consumer. + assertEquals("Unexpected number of created results.", 2, + src.getNumberOfProducedIntermediateDataSets()); + + for (IntermediateDataSet dataSet : src.getProducedDataSets()) { + ResultPartitionType dsType = dataSet.getResultType(); + + // The result type is determined by the channel between the union and the join node + // and *not* the channel between source and union. + if (unionToJoin.equals(BATCH)) { + assertTrue("Expected batch exchange, but result type is " + dsType + ".", + dsType.isBlocking()); + } else { + assertFalse("Expected non-batch exchange, but result type is " + dsType + ".", + dsType.isBlocking()); + } + } + } + } + +} From c9cfb17cb095def8b8ea0ed1b598fc78b890b874 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Wed, 22 Jul 2015 11:38:13 +0800 Subject: [PATCH 155/175] [FLINK-1901] [core] Create sample operator for Dataset. [FLINK-1901] [core] enable sample with fixed size on the whole dataset. [FLINK-1901] [core] add more comments for RandomSamplerTest. [FLINK-1901] [core] refactor PoissonSampler output Iterator. [FLINK-1901] [core] move sample/sampleWithSize operator to DataSetUtils. Adds notes for commons-math3 to LICENSE and NOTICE file This closes #949. --- flink-dist/src/main/flink-bin/LICENSE | 1 + flink-dist/src/main/flink-bin/NOTICE | 15 + flink-java/pom.xml | 6 + .../org/apache/flink/api/java/DataSet.java | 2 +- .../java/org/apache/flink/api/java/Utils.java | 4 + .../java/functions/SampleInCoordinator.java | 71 +++ .../api/java/functions/SampleInPartition.java | 71 +++ .../java/functions/SampleWithFraction.java | 68 +++ .../api/java/sampling/BernoulliSampler.java | 117 +++++ .../sampling/DistributedRandomSampler.java | 125 +++++ .../java/sampling/IntermediateSampleData.java | 47 ++ .../api/java/sampling/PoissonSampler.java | 122 +++++ .../api/java/sampling/RandomSampler.java | 63 +++ .../ReservoirSamplerWithReplacement.java | 110 +++++ .../ReservoirSamplerWithoutReplacement.java | 106 ++++ .../flink/api/java/utils/DataSetUtils.java | 95 ++++ .../api/java/sampling/RandomSamplerTest.java | 452 ++++++++++++++++++ .../apache/flink/api/scala/DataSetUtils.scala | 40 +- .../apache/flink/test/util/TestBaseUtils.java | 31 ++ .../test/javaApiOperators/SampleITCase.java | 167 +++++++ .../api/scala/operators/SampleITCase.scala | 167 +++++++ pom.xml | 6 + 22 files changed, 1884 insertions(+), 2 deletions(-) create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInCoordinator.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInPartition.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/functions/SampleWithFraction.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/DistributedRandomSampler.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/IntermediateSampleData.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/RandomSampler.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SampleITCase.scala diff --git a/flink-dist/src/main/flink-bin/LICENSE b/flink-dist/src/main/flink-bin/LICENSE index 281b8f0b35702..e79ff7115faf6 100644 --- a/flink-dist/src/main/flink-bin/LICENSE +++ b/flink-dist/src/main/flink-bin/LICENSE @@ -277,6 +277,7 @@ under the Apache License (v 2.0): - Uncommons Math (org.uncommons.maths:uncommons-maths:1.2.2a - https://github.com/dwdyer/uncommons-maths) - Jansi (org.fusesource.jansi:jansi:1.4 - https://github.com/fusesource/jansi) - Apache Camel Core (org.apache.camel:camel-core:2.10.3 - http://camel.apache.org/camel-core.html) + - Apache Commons Math (org.apache.commons:commons-math3:3.5 - http://commons.apache.org/proper/commons-math/index.html) ----------------------------------------------------------------------- diff --git a/flink-dist/src/main/flink-bin/NOTICE b/flink-dist/src/main/flink-bin/NOTICE index a71e61d86ad80..7b0fe72209773 100644 --- a/flink-dist/src/main/flink-bin/NOTICE +++ b/flink-dist/src/main/flink-bin/NOTICE @@ -68,6 +68,21 @@ This product contains code derived from JSON in Java code from JSON.org. Copyright (c) 2002 JSON.org +----------------------------------------------------------------------- + Apache Commons Math +----------------------------------------------------------------------- + +Apache Commons Math +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This product includes software developed for Orekit by +CS Systèmes d'Information (http://www.c-s.fr/) +Copyright 2010-2012 CS Systèmes d'Information + + ----------------------------------------------------------------------- Akka ----------------------------------------------------------------------- diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 683304f08f0f5..d7770480533ef 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -92,6 +92,12 @@ under the License. guava ${guava.version} + + + org.apache.commons + commons-math3 + + org.apache.flink diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index 81ba2792acb55..98a94c6426a21 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -1057,7 +1057,7 @@ public DataSet runOperation(CustomUnaryOperation operation) { public UnionOperator union(DataSet other){ return new UnionOperator(this, other, Utils.getCallLocationName()); } - + // -------------------------------------------------------------------------------------------- // Partitioning // -------------------------------------------------------------------------------------------- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java index a1e3d257afdc7..785f3ce0bd494 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java @@ -28,6 +28,8 @@ import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.List; +import java.util.Random; + import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; @@ -36,6 +38,8 @@ public class Utils { + + public static final Random RNG = new Random(); public static String getCallLocationName() { return getCallLocationName(4); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInCoordinator.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInCoordinator.java new file mode 100644 index 0000000000000..528d746d5d254 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInCoordinator.java @@ -0,0 +1,71 @@ +/* + * 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.api.java.functions; + +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.java.sampling.IntermediateSampleData; +import org.apache.flink.api.java.sampling.DistributedRandomSampler; +import org.apache.flink.api.java.sampling.ReservoirSamplerWithReplacement; +import org.apache.flink.api.java.sampling.ReservoirSamplerWithoutReplacement; +import org.apache.flink.util.Collector; + +import java.util.Iterator; + +/** + * SampleInCoordinator wraps the sample logic of the coordinator side (the second phase of + * distributed sample algorithm). It executes the coordinator side sample logic in an all reduce + * function. The user needs to make sure that the operator parallelism of this function is 1 to + * make sure this is a central coordinator. Besides, we do not need the task index information for + * random generator seed as the parallelism must be 1. + * + * @param the data type wrapped in ElementWithRandom as input. + */ +public class SampleInCoordinator implements GroupReduceFunction, T> { + + private boolean withReplacement; + private int numSample; + private long seed; + + /** + * Create a function instance of SampleInCoordinator. + * + * @param withReplacement Whether element can be selected more than once. + * @param numSample Fixed sample size. + * @param seed Random generator seed. + */ + public SampleInCoordinator(boolean withReplacement, int numSample, long seed) { + this.withReplacement = withReplacement; + this.numSample = numSample; + this.seed = seed; + } + + @Override + public void reduce(Iterable> values, Collector out) throws Exception { + DistributedRandomSampler sampler; + if (withReplacement) { + sampler = new ReservoirSamplerWithReplacement<>(numSample, seed); + } else { + sampler = new ReservoirSamplerWithoutReplacement<>(numSample, seed); + } + + Iterator sampled = sampler.sampleInCoordinator(values.iterator()); + while (sampled.hasNext()) { + out.collect(sampled.next()); + } + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInPartition.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInPartition.java new file mode 100644 index 0000000000000..295fb441ded95 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleInPartition.java @@ -0,0 +1,71 @@ +/* + * 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.api.java.functions; + +import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.api.java.sampling.IntermediateSampleData; +import org.apache.flink.api.java.sampling.DistributedRandomSampler; +import org.apache.flink.api.java.sampling.ReservoirSamplerWithReplacement; +import org.apache.flink.api.java.sampling.ReservoirSamplerWithoutReplacement; +import org.apache.flink.util.Collector; + +import java.util.Iterator; + +/** + * SampleInPartition wraps the sample logic on the partition side (the first phase of distributed + * sample algorithm). It executes the partition side sample logic in a mapPartition function. + * + * @param The type of input data + */ +public class SampleInPartition extends RichMapPartitionFunction> { + + private boolean withReplacement; + private int numSample; + private long seed; + + /** + * Create a function instance of SampleInPartition. + * + * @param withReplacement Whether element can be selected more than once. + * @param numSample Fixed sample size. + * @param seed Random generator seed. + */ + public SampleInPartition(boolean withReplacement, int numSample, long seed) { + this.withReplacement = withReplacement; + this.numSample = numSample; + this.seed = seed; + } + + @Override + public void mapPartition(Iterable values, Collector> out) throws Exception { + DistributedRandomSampler sampler; + long seedAndIndex = seed + getRuntimeContext().getIndexOfThisSubtask(); + if (withReplacement) { + sampler = new ReservoirSamplerWithReplacement(numSample, seedAndIndex); + } else { + sampler = new ReservoirSamplerWithoutReplacement(numSample, seedAndIndex); + } + + Iterator> sampled = sampler.sampleInPartition(values.iterator()); + while (sampled.hasNext()) { + out.collect(sampled.next()); + } + } +} + + diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleWithFraction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleWithFraction.java new file mode 100644 index 0000000000000..4ef9aa0f37f6f --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/SampleWithFraction.java @@ -0,0 +1,68 @@ +/* + * 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.api.java.functions; + +import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.api.java.sampling.BernoulliSampler; +import org.apache.flink.api.java.sampling.PoissonSampler; +import org.apache.flink.api.java.sampling.RandomSampler; +import org.apache.flink.util.Collector; + +import java.util.Iterator; + +/** + * A map partition function wrapper for sampling algorithms with fraction, the sample algorithm + * takes the partition iterator as input. + * + * @param + */ +public class SampleWithFraction extends RichMapPartitionFunction { + + private boolean withReplacement; + private double fraction; + private long seed; + + /** + * Create a function instance of SampleWithFraction. + * + * @param withReplacement Whether element can be selected more than once. + * @param fraction Probability that each element is selected. + * @param seed random number generator seed. + */ + public SampleWithFraction(boolean withReplacement, double fraction, long seed) { + this.withReplacement = withReplacement; + this.fraction = fraction; + this.seed = seed; + } + + @Override + public void mapPartition(Iterable values, Collector out) throws Exception { + RandomSampler sampler; + long seedAndIndex = seed + getRuntimeContext().getIndexOfThisSubtask(); + if (withReplacement) { + sampler = new PoissonSampler<>(fraction, seedAndIndex); + } else { + sampler = new BernoulliSampler<>(fraction, seedAndIndex); + } + + Iterator sampled = sampler.sample(values.iterator()); + while (sampled.hasNext()) { + out.collect(sampled.next()); + } + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java new file mode 100644 index 0000000000000..0f5ecc67a19a2 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java @@ -0,0 +1,117 @@ +/* + * 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.api.java.sampling; + +import com.google.common.base.Preconditions; + +import java.util.Iterator; +import java.util.Random; + +/** + * A sampler implementation built upon a Bernoulli trail. This sampler is used to sample with + * fraction and without replacement. Whether an element is sampled or not is determined by a + * Bernoulli experiment. + * + * @param The type of sample. + */ +public class BernoulliSampler extends RandomSampler { + + private final double fraction; + private final Random random; + + /** + * Create a Bernoulli sampler with sample fraction and default random number generator. + * + * @param fraction Sample fraction, aka the Bernoulli sampler possibility. + */ + public BernoulliSampler(double fraction) { + this(fraction, new Random()); + } + + /** + * Create a Bernoulli sampler with sample fraction and random number generator seed. + * + * @param fraction Sample fraction, aka the Bernoulli sampler possibility. + * @param seed Random number generator seed. + */ + public BernoulliSampler(double fraction, long seed) { + this(fraction, new Random(seed)); + } + + /** + * Create a Bernoulli sampler with sample fraction and random number generator. + * + * @param fraction Sample fraction, aka the Bernoulli sampler possibility. + * @param random The random number generator. + */ + public BernoulliSampler(double fraction, Random random) { + Preconditions.checkArgument(fraction >= 0 && fraction <= 1.0d, "fraction fraction must between [0, 1]."); + this.fraction = fraction; + this.random = random; + } + + /** + * Sample the input elements, for each input element, take a Bernoulli trail for sampling. + * + * @param input Elements to be sampled. + * @return The sampled result which is lazy computed upon input elements. + */ + @Override + public Iterator sample(final Iterator input) { + if (fraction == 0) { + return EMPTY_ITERABLE; + } + + return new SampledIterator() { + T current = null; + + @Override + public boolean hasNext() { + if (current == null) { + current = getNextSampledElement(); + } + + return current != null; + } + + @Override + public T next() { + if (current == null) { + return getNextSampledElement(); + } else { + T result = current; + current = null; + + return result; + } + } + + private T getNextSampledElement() { + while (input.hasNext()) { + T element = input.next(); + + if (random.nextDouble() <= fraction) { + return element; + } + } + + return null; + } + }; + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/DistributedRandomSampler.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/DistributedRandomSampler.java new file mode 100644 index 0000000000000..e5a719f9d1da1 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/DistributedRandomSampler.java @@ -0,0 +1,125 @@ +/* + * 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.api.java.sampling; + +import java.util.Iterator; +import java.util.PriorityQueue; + +/** + * For sampling with fraction, the sample algorithms are natively distributed, while it's not + * true for fixed size sample algorithms. The fixed size sample algorithms require two-phases + * sampling (according to our current implementation). In the first phase, each distributed + * partition is sampled independently. The partial sampling results are handled by a central + * coordinator. The central coordinator combines the partial sampling results to form the final + * result. + * + * @param The input data type. + */ +public abstract class DistributedRandomSampler extends RandomSampler { + + protected final int numSamples; + + public DistributedRandomSampler(int numSamples) { + this.numSamples = numSamples; + } + + protected final Iterator> EMPTY_INTERMEDIATE_ITERABLE = + new SampledIterator>() { + @Override + public boolean hasNext() { + return false; + } + + @Override + public IntermediateSampleData next() { + return null; + } + }; + + /** + * Sample algorithm for the first phase. It operates on a single partition. + * + * @param input The DataSet input of each partition. + * @return Intermediate sample output which will be used as the input of the second phase. + */ + public abstract Iterator> sampleInPartition(Iterator input); + + /** + * Sample algorithm for the second phase. This operation should be executed as the UDF of + * an all reduce operation. + * + * @param input The intermediate sample output generated in the first phase. + * @return The sampled output. + */ + public Iterator sampleInCoordinator(Iterator> input) { + if (numSamples == 0) { + return EMPTY_ITERABLE; + } + + // This queue holds fixed number elements with the top K weight for the coordinator. + PriorityQueue> reservoir = new PriorityQueue>(numSamples); + int index = 0; + IntermediateSampleData smallest = null; + while (input.hasNext()) { + IntermediateSampleData element = input.next(); + if (index < numSamples) { + // Fill the queue with first K elements from input. + reservoir.add(element); + smallest = reservoir.peek(); + } else { + // If current element weight is larger than the smallest one in queue, remove the element + // with the smallest weight, and append current element into the queue. + if (element.getWeight() > smallest.getWeight()) { + reservoir.remove(); + reservoir.add(element); + smallest = reservoir.peek(); + } + } + index++; + } + final Iterator> itr = reservoir.iterator(); + + return new Iterator() { + @Override + public boolean hasNext() { + return itr.hasNext(); + } + + @Override + public T next() { + return itr.next().getElement(); + } + + @Override + public void remove() { + itr.remove(); + } + }; + } + + /** + * Combine the first phase and second phase in sequence, implemented for test purpose only. + * + * @param input Source data. + * @return Sample result in sequence. + */ + @Override + public Iterator sample(Iterator input) { + return sampleInCoordinator(sampleInPartition(input)); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/IntermediateSampleData.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/IntermediateSampleData.java new file mode 100644 index 0000000000000..1d70f19fd0997 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/IntermediateSampleData.java @@ -0,0 +1,47 @@ +/* + * 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.api.java.sampling; + +/** + * The data structure which is transferred between partitions and the coordinator for distributed + * random sampling. + * + * @param The type of sample data. + */ +public class IntermediateSampleData implements Comparable> { + private double weight; + private T element; + + public IntermediateSampleData(double weight, T element) { + this.weight = weight; + this.element = element; + } + + public double getWeight() { + return weight; + } + + public T getElement() { + return element; + } + + @Override + public int compareTo(IntermediateSampleData other) { + return this.weight >= other.getWeight() ? 1 : -1; + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java new file mode 100644 index 0000000000000..3834d2414bee0 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java @@ -0,0 +1,122 @@ +/* + * 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.api.java.sampling; + +import com.google.common.base.Preconditions; +import org.apache.commons.math3.distribution.PoissonDistribution; + +import java.util.Iterator; + +/** + * A sampler implementation based on the Poisson Distribution. While sampling elements with fraction + * and replacement, the selected number of each element follows a given poisson distribution. + * + * @param The type of sample. + * @see https://en.wikipedia.org/wiki/Poisson_distribution + */ +public class PoissonSampler extends RandomSampler { + + private PoissonDistribution poissonDistribution; + private final double fraction; + + /** + * Create a poisson sampler which can sample elements with replacement. + * + * @param fraction The expected count of each element. + * @param seed Random number generator seed for internal PoissonDistribution. + */ + public PoissonSampler(double fraction, long seed) { + Preconditions.checkArgument(fraction >= 0, "fraction should be positive."); + this.fraction = fraction; + if (this.fraction > 0) { + this.poissonDistribution = new PoissonDistribution(fraction); + this.poissonDistribution.reseedRandomGenerator(seed); + } + } + + /** + * Create a poisson sampler which can sample elements with replacement. + * + * @param fraction The expected count of each element. + */ + public PoissonSampler(double fraction) { + Preconditions.checkArgument(fraction >= 0, "fraction should be non-negative."); + this.fraction = fraction; + if (this.fraction > 0) { + this.poissonDistribution = new PoissonDistribution(fraction); + } + } + + /** + * Sample the input elements, for each input element, generate its count following a poisson + * distribution. + * + * @param input Elements to be sampled. + * @return The sampled result which is lazy computed upon input elements. + */ + @Override + public Iterator sample(final Iterator input) { + if (fraction == 0) { + return EMPTY_ITERABLE; + } + + return new SampledIterator() { + T currentElement; + int currentCount = 0; + + @Override + public boolean hasNext() { + if (currentCount > 0) { + return true; + } else { + moveToNextElement(); + + if (currentCount > 0) { + return true; + } else { + return false; + } + } + } + + private void moveToNextElement() { + while (input.hasNext()) { + currentElement = input.next(); + currentCount = poissonDistribution.sample(); + if (currentCount > 0) { + break; + } + } + } + + @Override + public T next() { + if (currentCount == 0) { + moveToNextElement(); + } + + if (currentCount == 0) { + return null; + } else { + currentCount--; + return currentElement; + } + } + }; + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/RandomSampler.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/RandomSampler.java new file mode 100644 index 0000000000000..5fe292025b1b4 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/RandomSampler.java @@ -0,0 +1,63 @@ +/* + * 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.api.java.sampling; + +import java.util.Iterator; + +/** + * A data sample is a set of data selected from a statistical population by a defined procedure. + * RandomSampler helps to create data sample randomly. + * + * @param The type of sampler data. + */ +public abstract class RandomSampler { + + protected final Iterator EMPTY_ITERABLE = new SampledIterator() { + @Override + public boolean hasNext() { + return false; + } + + @Override + public T next() { + return null; + } + }; + + /** + * Randomly sample the elements from input in sequence, and return the result iterator. + * + * @param input Source data + * @return The sample result. + */ + public abstract Iterator sample(Iterator input); + +} + +/** + * A simple abstract iterator which implements the remove method as unsupported operation. + * + * @param The type of iterator data. + */ +abstract class SampledIterator implements Iterator { + @Override + public void remove() { + throw new UnsupportedOperationException("Do not support this operation."); + } + +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java new file mode 100644 index 0000000000000..9c371541dde34 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java @@ -0,0 +1,110 @@ +/* + * 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.api.java.sampling; + +import com.google.common.base.Preconditions; + +import java.util.Iterator; +import java.util.PriorityQueue; +import java.util.Random; + +/** + * A simple in memory implementation of Reservoir Sampling with replacement and with only one pass + * through the input iteration whose size is unpredictable. The basic idea behind this sampler + * implementation is quite similar to {@link ReservoirSamplerWithoutReplacement}. The main + * difference is that, in the first phase, we generate weights for each element K times, so that + * each element can get selected multiple times. + * + * This implementation refers to the algorithm described in + * "Optimal Random Sampling from Distributed Streams Revisited". + * + * @param The type of sample. + */ +public class ReservoirSamplerWithReplacement extends DistributedRandomSampler { + + private final Random random; + + /** + * Create a sampler with fixed sample size and default random number generator. + * + * @param numSamples Number of selected elements, must be non-negative. + */ + public ReservoirSamplerWithReplacement(int numSamples) { + this(numSamples, new Random()); + } + + /** + * Create a sampler with fixed sample size and random number generator seed. + * + * @param numSamples Number of selected elements, must be non-negative. + * @param seed Random number generator seed + */ + public ReservoirSamplerWithReplacement(int numSamples, long seed) { + this(numSamples, new Random(seed)); + } + + /** + * Create a sampler with fixed sample size and random number generator. + * + * @param numSamples Number of selected elements, must be non-negative. + * @param random Random number generator + */ + public ReservoirSamplerWithReplacement(int numSamples, Random random) { + super(numSamples); + Preconditions.checkArgument(numSamples >= 0, "numSamples should be non-negative."); + this.random = random; + } + + @Override + public Iterator> sampleInPartition(Iterator input) { + if (numSamples == 0) { + return EMPTY_INTERMEDIATE_ITERABLE; + } + + // This queue holds a fixed number of elements with the top K weight for current partition. + PriorityQueue> queue = new PriorityQueue>(numSamples); + + IntermediateSampleData smallest = null; + + if (input.hasNext()) { + T element = input.next(); + // Initiate the queue with the first element and random weights. + for (int i = 0; i < numSamples; i++) { + queue.add(new IntermediateSampleData(random.nextDouble(), element)); + smallest = queue.peek(); + } + } + + while (input.hasNext()) { + T element = input.next(); + // To sample with replacement, we generate K random weights for each element, so that it's + // possible to be selected multi times. + for (int i = 0; i < numSamples; i++) { + // If current element weight is larger than the smallest one in queue, remove the element + // with the smallest weight, and append current element into the queue. + double rand = random.nextDouble(); + if (rand > smallest.getWeight()) { + queue.remove(); + queue.add(new IntermediateSampleData(rand, element)); + smallest = queue.peek(); + } + } + } + return queue.iterator(); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java new file mode 100644 index 0000000000000..b953bfff0f24b --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java @@ -0,0 +1,106 @@ +/* + * 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.api.java.sampling; + +import com.google.common.base.Preconditions; + +import java.util.Iterator; +import java.util.PriorityQueue; +import java.util.Random; + +/** + * A simple in memory implementation of Reservoir Sampling without replacement, and with only one + * pass through the input iteration whose size is unpredictable. The basic idea behind this sampler + * implementation is to generate a random number for each input element as its weight, select the + * top K elements with max weight. As the weights are generated randomly, so are the selected + * top K elements. The algorithm is implemented using the {@link DistributedRandomSampler} + * interface. In the first phase, we generate random numbers as the weights for each element and + * select top K elements as the output of each partitions. In the second phase, we select top K + * elements from all the outputs of the first phase. + * + * This implementation refers to the algorithm described in + * "Optimal Random Sampling from Distributed Streams Revisited". + * + * @param The type of the sampler. + */ +public class ReservoirSamplerWithoutReplacement extends DistributedRandomSampler { + + private final Random random; + + /** + * Create a new sampler with reservoir size and a supplied random number generator. + * + * @param numSamples Maximum number of samples to retain in reservoir, must be non-negative. + * @param random Instance of random number generator for sampling. + */ + public ReservoirSamplerWithoutReplacement(int numSamples, Random random) { + super(numSamples); + Preconditions.checkArgument(numSamples >= 0, "numSamples should be non-negative."); + this.random = random; + } + + /** + * Create a new sampler with reservoir size and a default random number generator. + * + * @param numSamples Maximum number of samples to retain in reservoir, must be non-negative. + */ + public ReservoirSamplerWithoutReplacement(int numSamples) { + this(numSamples, new Random()); + } + + /** + * Create a new sampler with reservoir size and the seed for random number generator. + * + * @param numSamples Maximum number of samples to retain in reservoir, must be non-negative. + * @param seed Random number generator seed. + */ + public ReservoirSamplerWithoutReplacement(int numSamples, long seed) { + + this(numSamples, new Random(seed)); + } + + @Override + public Iterator> sampleInPartition(Iterator input) { + if (numSamples == 0) { + return EMPTY_INTERMEDIATE_ITERABLE; + } + + // This queue holds fixed number elements with the top K weight for current partition. + PriorityQueue> queue = new PriorityQueue>(numSamples); + int index = 0; + IntermediateSampleData smallest = null; + while (input.hasNext()) { + T element = input.next(); + if (index < numSamples) { + // Fill the queue with first K elements from input. + queue.add(new IntermediateSampleData(random.nextDouble(), element)); + smallest = queue.peek(); + } else { + double rand = random.nextDouble(); + // Remove the element with the smallest weight, and append current element into the queue. + if (rand > smallest.getWeight()) { + queue.remove(); + queue.add(new IntermediateSampleData(rand, element)); + smallest = queue.peek(); + } + } + index++; + } + return queue.iterator(); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java index 142e7cf84bfd7..d26892592572d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java @@ -19,7 +19,14 @@ package org.apache.flink.api.java.utils; import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.api.java.sampling.IntermediateSampleData; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils; +import org.apache.flink.api.java.functions.SampleInCoordinator; +import org.apache.flink.api.java.functions.SampleInPartition; +import org.apache.flink.api.java.functions.SampleWithFraction; +import org.apache.flink.api.java.operators.GroupReduceOperator; +import org.apache.flink.api.java.operators.MapPartitionOperator; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; @@ -142,6 +149,94 @@ public void mapPartition(Iterable values, Collector> out) thr }); } + // -------------------------------------------------------------------------------------------- + // Sample + // -------------------------------------------------------------------------------------------- + + /** + * Generate a sample of DataSet by the probability fraction of each element. + * + * @param withReplacement Whether element can be selected more than once. + * @param fraction Probability that each element is chosen, should be [0,1] without replacement, + * and [0, ∞) with replacement. While fraction is larger than 1, the elements are + * expected to be selected multi times into sample on average. + * @return The sampled DataSet + */ + public static MapPartitionOperator sample( + DataSet input, + final boolean withReplacement, + final double fraction) { + + return sample(input, withReplacement, fraction, Utils.RNG.nextLong()); + } + + /** + * Generate a sample of DataSet by the probability fraction of each element. + * + * @param withReplacement Whether element can be selected more than once. + * @param fraction Probability that each element is chosen, should be [0,1] without replacement, + * and [0, ∞) with replacement. While fraction is larger than 1, the elements are + * expected to be selected multi times into sample on average. + * @param seed random number generator seed. + * @return The sampled DataSet + */ + public static MapPartitionOperator sample( + DataSet input, + final boolean withReplacement, + final double fraction, + final long seed) { + + return input.mapPartition(new SampleWithFraction(withReplacement, fraction, seed)); + } + + /** + * Generate a sample of DataSet which contains fixed size elements. + *

    + * NOTE: Sample with fixed size is not as efficient as sample with fraction, use sample with + * fraction unless you need exact precision. + *

    + * + * @param withReplacement Whether element can be selected more than once. + * @param numSample The expected sample size. + * @return The sampled DataSet + */ + public static DataSet sampleWithSize( + DataSet input, + final boolean withReplacement, + final int numSample) { + + return sampleWithSize(input, withReplacement, numSample, Utils.RNG.nextLong()); + } + + /** + * Generate a sample of DataSet which contains fixed size elements. + *

    + * NOTE: Sample with fixed size is not as efficient as sample with fraction, use sample with + * fraction unless you need exact precision. + *

    + * + * @param withReplacement Whether element can be selected more than once. + * @param numSample The expected sample size. + * @param seed Random number generator seed. + * @return The sampled DataSet + */ + public static DataSet sampleWithSize( + DataSet input, + final boolean withReplacement, + final int numSample, + final long seed) { + + SampleInPartition sampleInPartition = new SampleInPartition(withReplacement, numSample, seed); + MapPartitionOperator mapPartitionOperator = input.mapPartition(sampleInPartition); + + // There is no previous group, so the parallelism of GroupReduceOperator is always 1. + String callLocation = Utils.getCallLocationName(); + SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(withReplacement, numSample, seed); + return new GroupReduceOperator, T>(mapPartitionOperator, + input.getType(), sampleInCoordinator, callLocation); + } + + // ************************************************************************* // UTIL METHODS // ************************************************************************* diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java new file mode 100644 index 0000000000000..83e5b419e98f6 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java @@ -0,0 +1,452 @@ +/* + * 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.api.java.sampling; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * This test suite try to verify whether all the random samplers work as we expected, which mainly focus on: + *

      + *
    • Does sampled result fit into input parameters? we check parameters like sample fraction, sample size, + * w/o replacement, and so on.
    • + *
    • Does sampled result randomly selected? we verify this by measure how much does it distributed on source data. + * Run Kolmogorov-Smirnov (KS) test between the random samplers and default reference samplers which is distributed + * well-proportioned on source data. If random sampler select elements randomly from source, it would distributed + * well-proportioned on source data as well. The KS test will fail to strongly reject the null hypothesis that + * the distributions of sampling gaps are the same. + *
    • + *
    + * + * @see Kolmogorov Smirnov test + */ +public class RandomSamplerTest { + private final static int SOURCE_SIZE = 10000; + private static KolmogorovSmirnovTest ksTest; + private static List source; + private final static int DEFFAULT_PARTITION_NUMBER=10; + private List[] sourcePartitions = new List[DEFFAULT_PARTITION_NUMBER]; + + @BeforeClass + public static void init() { + // initiate source data set. + source = new ArrayList(SOURCE_SIZE); + for (int i = 0; i < SOURCE_SIZE; i++) { + source.add((double) i); + } + + ksTest = new KolmogorovSmirnovTest(); + } + + private void initSourcePartition() { + for (int i=0; i(); + } + for (int i = 0; i< SOURCE_SIZE; i++) { + int index = i % DEFFAULT_PARTITION_NUMBER; + sourcePartitions[index].add((double)i); + } + } + + @Test(expected = java.lang.IllegalArgumentException.class) + public void testBernoulliSamplerWithUnexpectedFraction1() { + verifySamplerFraction(-1, false); + } + + @Test(expected = java.lang.IllegalArgumentException.class) + public void testBernoulliSamplerWithUnexpectedFraction2() { + verifySamplerFraction(2, false); + } + + @Test + public void testBernoulliSamplerFraction() { + verifySamplerFraction(0.01, false); + verifySamplerFraction(0.05, false); + verifySamplerFraction(0.1, false); + verifySamplerFraction(0.3, false); + verifySamplerFraction(0.5, false); + verifySamplerFraction(0.854, false); + verifySamplerFraction(0.99, false); + } + + @Test + public void testBernoulliSamplerDuplicateElements() { + verifyRandomSamplerDuplicateElements(new BernoulliSampler(0.01)); + verifyRandomSamplerDuplicateElements(new BernoulliSampler(0.1)); + verifyRandomSamplerDuplicateElements(new BernoulliSampler(0.5)); + } + + @Test(expected = java.lang.IllegalArgumentException.class) + public void testPoissonSamplerWithUnexpectedFraction1() { + verifySamplerFraction(-1, true); + } + + @Test + public void testPoissonSamplerFraction() { + verifySamplerFraction(0.01, true); + verifySamplerFraction(0.05, true); + verifySamplerFraction(0.1, true); + verifySamplerFraction(0.5, true); + verifySamplerFraction(0.854, true); + verifySamplerFraction(0.99, true); + verifySamplerFraction(1.5, true); + } + + @Test(expected = java.lang.IllegalArgumentException.class) + public void testReservoirSamplerUnexpectedSize1() { + verifySamplerFixedSampleSize(-1, true); + } + + @Test(expected = java.lang.IllegalArgumentException.class) + public void testReservoirSamplerUnexpectedSize2() { + verifySamplerFixedSampleSize(-1, false); + } + + @Test + public void testBernoulliSamplerDistribution() { + verifyBernoulliSampler(0.01d); + verifyBernoulliSampler(0.05d); + verifyBernoulliSampler(0.1d); + verifyBernoulliSampler(0.5d); + } + + @Test + public void testPoissonSamplerDistribution() { + verifyPoissonSampler(0.01d); + verifyPoissonSampler(0.05d); + verifyPoissonSampler(0.1d); + verifyPoissonSampler(0.5d); + } + + @Test + public void testReservoirSamplerSampledSize() { + verifySamplerFixedSampleSize(1, true); + verifySamplerFixedSampleSize(10, true); + verifySamplerFixedSampleSize(100, true); + verifySamplerFixedSampleSize(1234, true); + verifySamplerFixedSampleSize(9999, true); + verifySamplerFixedSampleSize(20000, true); + + verifySamplerFixedSampleSize(1, false); + verifySamplerFixedSampleSize(10, false); + verifySamplerFixedSampleSize(100, false); + verifySamplerFixedSampleSize(1234, false); + verifySamplerFixedSampleSize(9999, false); + } + + @Test + public void testReservoirSamplerSampledSize2() { + RandomSampler sampler = new ReservoirSamplerWithoutReplacement(20000); + Iterator sampled = sampler.sample(source.iterator()); + assertTrue("ReservoirSamplerWithoutReplacement sampled output size should not beyond the source size.", getSize(sampled) == SOURCE_SIZE); + } + + @Test + public void testReservoirSamplerDuplicateElements() { + verifyRandomSamplerDuplicateElements(new ReservoirSamplerWithoutReplacement(100)); + verifyRandomSamplerDuplicateElements(new ReservoirSamplerWithoutReplacement(1000)); + verifyRandomSamplerDuplicateElements(new ReservoirSamplerWithoutReplacement(5000)); + } + + @Test + public void testReservoirSamplerWithoutReplacement() { + verifyReservoirSamplerWithoutReplacement(100, false); + verifyReservoirSamplerWithoutReplacement(500, false); + verifyReservoirSamplerWithoutReplacement(1000, false); + verifyReservoirSamplerWithoutReplacement(5000, false); + } + + @Test + public void testReservoirSamplerWithReplacement() { + verifyReservoirSamplerWithReplacement(100, false); + verifyReservoirSamplerWithReplacement(500, false); + verifyReservoirSamplerWithReplacement(1000, false); + verifyReservoirSamplerWithReplacement(5000, false); + } + + @Test + public void testReservoirSamplerWithMultiSourcePartitions1() { + initSourcePartition(); + + verifyReservoirSamplerWithoutReplacement(100, true); + verifyReservoirSamplerWithoutReplacement(500, true); + verifyReservoirSamplerWithoutReplacement(1000, true); + verifyReservoirSamplerWithoutReplacement(5000, true); + } + + @Test + public void testReservoirSamplerWithMultiSourcePartitions2() { + initSourcePartition(); + + verifyReservoirSamplerWithReplacement(100, true); + verifyReservoirSamplerWithReplacement(500, true); + verifyReservoirSamplerWithReplacement(1000, true); + verifyReservoirSamplerWithReplacement(5000, true); + } + + /* + * Sample with fixed size, verify whether the sampled result size equals to input size. + */ + private void verifySamplerFixedSampleSize(int numSample, boolean withReplacement) { + RandomSampler sampler; + if (withReplacement) { + sampler = new ReservoirSamplerWithReplacement(numSample); + } else { + sampler = new ReservoirSamplerWithoutReplacement(numSample); + } + Iterator sampled = sampler.sample(source.iterator()); + assertEquals(numSample, getSize(sampled)); + } + + /* + * Sample with fraction, and verify whether the sampled result close to input fraction. + */ + private void verifySamplerFraction(double fraction, boolean withReplacement) { + RandomSampler sampler; + if (withReplacement) { + sampler = new PoissonSampler(fraction); + } else { + sampler = new BernoulliSampler(fraction); + } + + // take 5 times sample, and take the average result size for next step comparison. + int totalSampledSize = 0; + double sampleCount = 5; + for (int i = 0; i < sampleCount; i++) { + totalSampledSize += getSize(sampler.sample(source.iterator())); + } + double resultFraction = totalSampledSize / ((double) SOURCE_SIZE * sampleCount); + assertTrue(String.format("expected fraction: %f, result fraction: %f", fraction, resultFraction), Math.abs((resultFraction - fraction) / fraction) < 0.1); + } + + /* + * Test sampler without replacement, and verify that there should not exist any duplicate element in sampled result. + */ + private void verifyRandomSamplerDuplicateElements(final RandomSampler sampler) { + List list = Lists.newLinkedList(new Iterable() { + @Override + public Iterator iterator() { + return sampler.sample(source.iterator()); + } + }); + Set set = Sets.newHashSet(list); + assertTrue("There should not have duplicate element for sampler without replacement.", list.size() == set.size()); + } + + private int getSize(Iterator iterator) { + int size = 0; + while (iterator.hasNext()) { + iterator.next(); + size++; + } + return size; + } + + private void verifyBernoulliSampler(double fraction) { + BernoulliSampler sampler = new BernoulliSampler(fraction); + verifyRandomSamplerWithFraction(fraction, sampler, true); + verifyRandomSamplerWithFraction(fraction, sampler, false); + } + + private void verifyPoissonSampler(double fraction) { + PoissonSampler sampler = new PoissonSampler(fraction); + verifyRandomSamplerWithFraction(fraction, sampler, true); + verifyRandomSamplerWithFraction(fraction, sampler, false); + } + + private void verifyReservoirSamplerWithReplacement(int numSamplers, boolean sampleOnPartitions) { + ReservoirSamplerWithReplacement sampler = new ReservoirSamplerWithReplacement(numSamplers); + verifyRandomSamplerWithSampleSize(numSamplers, sampler, true, sampleOnPartitions); + verifyRandomSamplerWithSampleSize(numSamplers, sampler, false, sampleOnPartitions); + } + + private void verifyReservoirSamplerWithoutReplacement(int numSamplers, boolean sampleOnPartitions) { + ReservoirSamplerWithoutReplacement sampler = new ReservoirSamplerWithoutReplacement(numSamplers); + verifyRandomSamplerWithSampleSize(numSamplers, sampler, true, sampleOnPartitions); + verifyRandomSamplerWithSampleSize(numSamplers, sampler, false, sampleOnPartitions); + } + + /* + * Verify whether random sampler sample with fraction from source data randomly. There are two default sample, one is + * sampled from source data with certain interval, the other is sampled only from the first half region of source data, + * If random sampler select elements randomly from source, it would distributed well-proportioned on source data as well, + * so the K-S Test result would accept the first one, while reject the second one. + */ + private void verifyRandomSamplerWithFraction(double fraction, RandomSampler sampler, boolean withDefaultSampler) { + double[] baseSample; + if (withDefaultSampler) { + baseSample = getDefaultSampler(fraction); + } else { + baseSample = getWrongSampler(fraction); + } + + verifyKSTest(sampler, baseSample, withDefaultSampler); + } + + /* + * Verify whether random sampler sample with fixed size from source data randomly. There are two default sample, one is + * sampled from source data with certain interval, the other is sampled only from the first half region of source data, + * If random sampler select elements randomly from source, it would distributed well-proportioned on source data as well, + * so the K-S Test result would accept the first one, while reject the second one. + */ + private void verifyRandomSamplerWithSampleSize(int sampleSize, RandomSampler sampler, boolean withDefaultSampler, boolean sampleWithPartitions) { + double[] baseSample; + if (withDefaultSampler) { + baseSample = getDefaultSampler(sampleSize); + } else { + baseSample = getWrongSampler(sampleSize); + } + + verifyKSTest(sampler, baseSample, withDefaultSampler, sampleWithPartitions); + } + + private void verifyKSTest(RandomSampler sampler, double[] defaultSampler, boolean expectSuccess) { + verifyKSTest(sampler, defaultSampler, expectSuccess, false); + } + + private void verifyKSTest(RandomSampler sampler, double[] defaultSampler, boolean expectSuccess, boolean sampleOnPartitions) { + double[] sampled = getSampledOutput(sampler, sampleOnPartitions); + double pValue = ksTest.kolmogorovSmirnovStatistic(sampled, defaultSampler); + double dValue = getDValue(sampled.length, defaultSampler.length); + if (expectSuccess) { + assertTrue(String.format("KS test result with p value(%f), d value(%f)", pValue, dValue), pValue <= dValue); + } else { + assertTrue(String.format("KS test result with p value(%f), d value(%f)", pValue, dValue), pValue > dValue); + } + } + + private double[] getSampledOutput(RandomSampler sampler, boolean sampleOnPartitions) { + Iterator sampled = null; + if (sampleOnPartitions) { + DistributedRandomSampler reservoirRandomSampler = (DistributedRandomSampler)sampler; + List> intermediateResult = Lists.newLinkedList(); + for (int i=0; i> partialIntermediateResult = reservoirRandomSampler.sampleInPartition(sourcePartitions[i].iterator()); + while (partialIntermediateResult.hasNext()) { + intermediateResult.add(partialIntermediateResult.next()); + } + } + sampled = reservoirRandomSampler.sampleInCoordinator(intermediateResult.iterator()); + } else { + sampled = sampler.sample(source.iterator()); + } + List list = Lists.newArrayList(); + while (sampled.hasNext()) { + list.add(sampled.next()); + } + double[] result = transferFromListToArrayWithOrder(list); + return result; + } + + /* + * Some sample result may not order by the input sequence, we should make it in order to do K-S test. + */ + private double[] transferFromListToArrayWithOrder(List list) { + Collections.sort(list, new Comparator() { + @Override + public int compare(Double o1, Double o2) { + return o1 - o2 >= 0 ? 1 : -1; + } + }); + double[] result = new double[list.size()]; + for (int i = 0; i < list.size(); i++) { + result[i] = list.get(i); + } + return result; + } + + private double[] getDefaultSampler(double fraction) { + Preconditions.checkArgument(fraction > 0, "Sample fraction should be positive."); + int size = (int) (SOURCE_SIZE * fraction); + double step = 1 / fraction; + double[] defaultSampler = new double[size]; + for (int i = 0; i < size; i++) { + defaultSampler[i] = Math.round(step * i); + } + + return defaultSampler; + } + + private double[] getDefaultSampler(int fixSize) { + Preconditions.checkArgument(fixSize > 0, "Sample fraction should be positive."); + int size = fixSize; + double step = SOURCE_SIZE / (double) fixSize; + double[] defaultSampler = new double[size]; + for (int i = 0; i < size; i++) { + defaultSampler[i] = Math.round(step * i); + } + + return defaultSampler; + } + + /* + * Build a failed sample distribution which only contains elements in the first half of source data. + */ + private double[] getWrongSampler(double fraction) { + Preconditions.checkArgument(fraction > 0, "Sample size should be positive."); + int size = (int) (SOURCE_SIZE * fraction); + int halfSourceSize = SOURCE_SIZE / 2; + double[] wrongSampler = new double[size]; + for (int i = 0; i < size; i++) { + wrongSampler[i] = (double) i % halfSourceSize; + } + + return wrongSampler; + } + + /* + * Build a failed sample distribution which only contains elements in the first half of source data. + */ + private double[] getWrongSampler(int fixSize) { + Preconditions.checkArgument(fixSize > 0, "Sample size be positive."); + int halfSourceSize = SOURCE_SIZE / 2; + int size = fixSize; + double[] wrongSampler = new double[size]; + for (int i = 0; i < size; i++) { + wrongSampler[i] = (double) i % halfSourceSize; + } + + return wrongSampler; + } + + /* + * Calculate the D value of K-S test for p-value 0.05, m and n are the sample size + */ + private double getDValue(int m, int n) { + Preconditions.checkArgument(m > 0, "input sample size should be positive."); + Preconditions.checkArgument(n > 0, "input sample size should be positive."); + double first = (double) m; + double second = (double) n; + return 1.36 * Math.sqrt((first + second) / (first * second)); + } +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSetUtils.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSetUtils.scala index b1a1ab6dec20f..266375405118a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSetUtils.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSetUtils.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.{utils => jutils} +import org.apache.flink.api.java.{Utils, utils => jutils} import _root_.scala.language.implicitConversions import _root_.scala.reflect.ClassTag @@ -53,6 +53,44 @@ class DataSetUtils[T](val self: DataSet[T]) extends AnyVal { wrap(jutils.DataSetUtils.zipWithUniqueId(self.javaSet)) .map { t => (t.f0.toLong, t.f1) } } + + // -------------------------------------------------------------------------------------------- + // Sample + // -------------------------------------------------------------------------------------------- + /** + * Generate a sample of DataSet by the probability fraction of each element. + * + * @param withReplacement Whether element can be selected more than once. + * @param fraction Probability that each element is chosen, should be [0,1] without + * replacement, and [0, ∞) with replacement. While fraction is larger + * than 1, the elements are expected to be selected multi times into + * sample on average. + * @param seed Random number generator seed. + * @return The sampled DataSet + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long = Utils.RNG.nextLong()) + (implicit ti: TypeInformation[T], ct: ClassTag[T]): DataSet[T] = { + + wrap(jutils.DataSetUtils.sample(self.javaSet, withReplacement, fraction, seed)) + } + + /** + * Generate a sample of DataSet with fixed sample size. + *

    + * NOTE: Sample with fixed size is not as efficient as sample with fraction, + * use sample with fraction unless you need exact precision. + *

    + * + * @param withReplacement Whether element can be selected more than once. + * @param numSample The expected sample size. + * @param seed Random number generator seed. + * @return The sampled DataSet + */ + def sampleWithSize(withReplacement: Boolean, numSample: Int, seed: Long = Utils.RNG.nextLong()) + (implicit ti: TypeInformation[T], ct: ClassTag[T]): DataSet[T] = { + + wrap(jutils.DataSetUtils.sampleWithSize(self.javaSet, withReplacement, numSample, seed)) + } } object DataSetUtils { diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java index c28347c62f61f..ce022674a3849 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java @@ -19,11 +19,14 @@ package org.apache.flink.test.util; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import akka.actor.ActorRef; import akka.dispatch.Futures; import akka.pattern.Patterns; import akka.util.Timeout; +import com.google.common.collect.Lists; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.flink.api.java.tuple.Tuple; @@ -451,6 +454,34 @@ private static void compareResult(List result, String expected, boolean a assertEquals(extectedStrings[i], resultStrings[i]); } } + + // -------------------------------------------------------------------------------------------- + // Comparison methods for tests using sample + // -------------------------------------------------------------------------------------------- + + /** + * The expected string contains all expected results separate with line break, check whether all elements in result + * are contained in the expected string. + * @param result The test result. + * @param expected The expected string value combination. + * @param The result type. + */ + public static void containsResultAsText(List result, String expected) { + String[] expectedStrings = expected.split("\n"); + List resultStrings = Lists.newLinkedList(); + + for (int i = 0; i < result.size(); i++) { + T val = result.get(i); + String str = (val == null) ? "null" : val.toString(); + resultStrings.add(str); + } + + List expectedStringList = Arrays.asList(expectedStrings); + + for (String element : resultStrings) { + assertTrue(expectedStringList.contains(element)); + } + } // -------------------------------------------------------------------------------------------- // Miscellaneous helper methods diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java new file mode 100644 index 0000000000000..a9c75e5686c1a --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java @@ -0,0 +1,167 @@ +/* + * 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.test.javaApiOperators; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.FlatMapOperator; +import org.apache.flink.api.java.operators.MapPartitionOperator; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.apache.flink.util.Collector; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@SuppressWarnings("serial") +@RunWith(Parameterized.class) +public class SampleITCase extends MultipleProgramsTestBase { + + private static final Random RNG = new Random(); + + public SampleITCase(TestExecutionMode mode) { + super(mode); + } + + @Before + public void initiate() { + ExecutionEnvironment.getExecutionEnvironment().setParallelism(5); + } + + @Test + public void testSamplerWithFractionWithoutReplacement() throws Exception { + verifySamplerWithFractionWithoutReplacement(0d); + verifySamplerWithFractionWithoutReplacement(0.2d); + verifySamplerWithFractionWithoutReplacement(1.0d); + } + + @Test + public void testSamplerWithFractionWithReplacement() throws Exception { + verifySamplerWithFractionWithReplacement(0d); + verifySamplerWithFractionWithReplacement(0.2d); + verifySamplerWithFractionWithReplacement(1.0d); + verifySamplerWithFractionWithReplacement(2.0d); + } + + @Test + public void testSamplerWithSizeWithoutReplacement() throws Exception { + verifySamplerWithFixedSizeWithoutReplacement(0); + verifySamplerWithFixedSizeWithoutReplacement(2); + verifySamplerWithFixedSizeWithoutReplacement(21); + } + + @Test + public void testSamplerWithSizeWithReplacement() throws Exception { + verifySamplerWithFixedSizeWithReplacement(0); + verifySamplerWithFixedSizeWithReplacement(2); + verifySamplerWithFixedSizeWithReplacement(21); + } + + private void verifySamplerWithFractionWithoutReplacement(double fraction) throws Exception { + verifySamplerWithFractionWithoutReplacement(fraction, RNG.nextLong()); + } + + private void verifySamplerWithFractionWithoutReplacement(double fraction, long seed) throws Exception { + verifySamplerWithFraction(false, fraction, seed); + } + + private void verifySamplerWithFractionWithReplacement(double fraction) throws Exception { + verifySamplerWithFractionWithReplacement(fraction, RNG.nextLong()); + } + + private void verifySamplerWithFractionWithReplacement(double fraction, long seed) throws Exception { + verifySamplerWithFraction(true, fraction, seed); + } + + private void verifySamplerWithFraction(boolean withReplacement, double fraction, long seed) throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + FlatMapOperator, String> ds = getSourceDataSet(env); + MapPartitionOperator sampled = DataSetUtils.sample(ds, withReplacement, fraction, seed); + List result = sampled.collect(); + containsResultAsText(result, getSourceStrings()); + } + + private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples) throws Exception { + verifySamplerWithFixedSizeWithoutReplacement(numSamples, RNG.nextLong()); + } + + private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples, long seed) throws Exception { + verifySamplerWithFixedSize(false, numSamples, seed); + } + + private void verifySamplerWithFixedSizeWithReplacement(int numSamples) throws Exception { + verifySamplerWithFixedSizeWithReplacement(numSamples, RNG.nextLong()); + } + + private void verifySamplerWithFixedSizeWithReplacement(int numSamples, long seed) throws Exception { + verifySamplerWithFixedSize(true, numSamples, seed); + } + + private void verifySamplerWithFixedSize(boolean withReplacement, int numSamples, long seed) throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + FlatMapOperator, String> ds = getSourceDataSet(env); + DataSet sampled = DataSetUtils.sampleWithSize(ds, withReplacement, numSamples, seed); + List result = sampled.collect(); + assertEquals(numSamples, result.size()); + containsResultAsText(result, getSourceStrings()); + } + + private FlatMapOperator, String> getSourceDataSet(ExecutionEnvironment env) { + return CollectionDataSets.get3TupleDataSet(env).flatMap( + new FlatMapFunction, String>() { + @Override + public void flatMap(Tuple3 value, Collector out) throws Exception { + out.collect(value.f2); + } + }); + } + + private String getSourceStrings() { + return "Hi\n" + + "Hello\n" + + "Hello world\n" + + "Hello world, how are you?\n" + + "I am fine.\n" + + "Luke Skywalker\n" + + "Comment#1\n" + + "Comment#2\n" + + "Comment#3\n" + + "Comment#4\n" + + "Comment#5\n" + + "Comment#6\n" + + "Comment#7\n" + + "Comment#8\n" + + "Comment#9\n" + + "Comment#10\n" + + "Comment#11\n" + + "Comment#12\n" + + "Comment#13\n" + + "Comment#14\n" + + "Comment#15\n"; + } +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SampleITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SampleITCase.scala new file mode 100644 index 0000000000000..86b0818136c71 --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SampleITCase.scala @@ -0,0 +1,167 @@ +/* + * 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.api.scala.operators + +import java.util.{List => JavaList, Random} + +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.Assert._ +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{Before, After, Test} + +import org.apache.flink.api.scala.DataSetUtils.utilsToDataSet +import scala.collection.JavaConverters._ + +@RunWith(classOf[Parameterized]) +class SampleITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private val RNG: Random = new Random + + private var result: JavaList[String] = null; + + @Before + def initiate { + ExecutionEnvironment.getExecutionEnvironment.setParallelism(5) + } + + @After + def after() = { + TestBaseUtils.containsResultAsText(result, getSourceStrings) + } + + @Test + @throws(classOf[Exception]) + def testSamplerWithFractionWithoutReplacement { + verifySamplerWithFractionWithoutReplacement(0d) + verifySamplerWithFractionWithoutReplacement(0.2d) + verifySamplerWithFractionWithoutReplacement(1.0d) + } + + @Test + @throws(classOf[Exception]) + def testSamplerWithFractionWithReplacement { + verifySamplerWithFractionWithReplacement(0d) + verifySamplerWithFractionWithReplacement(0.2d) + verifySamplerWithFractionWithReplacement(1.0d) + verifySamplerWithFractionWithReplacement(2.0d) + } + + @Test + @throws(classOf[Exception]) + def testSamplerWithSizeWithoutReplacement { + verifySamplerWithFixedSizeWithoutReplacement(0) + verifySamplerWithFixedSizeWithoutReplacement(2) + verifySamplerWithFixedSizeWithoutReplacement(21) + } + + @Test + @throws(classOf[Exception]) + def testSamplerWithSizeWithReplacement { + verifySamplerWithFixedSizeWithReplacement(0) + verifySamplerWithFixedSizeWithReplacement(2) + verifySamplerWithFixedSizeWithReplacement(21) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFractionWithoutReplacement(fraction: Double) { + verifySamplerWithFractionWithoutReplacement(fraction, RNG.nextLong) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFractionWithoutReplacement(fraction: Double, seed: Long) { + verifySamplerWithFraction(false, fraction, seed) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFractionWithReplacement(fraction: Double) { + verifySamplerWithFractionWithReplacement(fraction, RNG.nextLong) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFractionWithReplacement(fraction: Double, seed: Long) { + verifySamplerWithFraction(true, fraction, seed) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFraction(withReplacement: Boolean, fraction: Double, seed: Long) { + val ds = getSourceDataSet() + val sampled = ds.sample(withReplacement, fraction, seed) + result = sampled.collect.asJava + } + + @throws(classOf[Exception]) + private def verifySamplerWithFixedSizeWithoutReplacement(numSamples: Int) { + verifySamplerWithFixedSizeWithoutReplacement(numSamples, RNG.nextLong) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFixedSizeWithoutReplacement(numSamples: Int, seed: Long) { + verifySamplerWithFixedSize(false, numSamples, seed) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFixedSizeWithReplacement(numSamples: Int) { + verifySamplerWithFixedSizeWithReplacement(numSamples, RNG.nextLong) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFixedSizeWithReplacement(numSamples: Int, seed: Long) { + verifySamplerWithFixedSize(true, numSamples, seed) + } + + @throws(classOf[Exception]) + private def verifySamplerWithFixedSize(withReplacement: Boolean, numSamples: Int, seed: Long) { + val ds = getSourceDataSet() + val sampled = ds.sampleWithSize(withReplacement, numSamples, seed) + result = sampled.collect.asJava + assertEquals(numSamples, result.size) + } + + private def getSourceDataSet(): DataSet[String] = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tupleDataSet = CollectionDataSets.get3TupleDataSet(env) + tupleDataSet.map(x => x._3) + } + + private def getSourceStrings: String = { + return "Hi\n" + + "Hello\n" + + "Hello world\n" + + "Hello world, how are you?\n" + + "I am fine.\n" + + "Luke Skywalker\n" + + "Comment#1\n" + + "Comment#2\n" + + "Comment#3\n" + + "Comment#4\n" + + "Comment#5\n" + + "Comment#6\n" + + "Comment#7\n" + + "Comment#8\n" + + "Comment#9\n" + + "Comment#10\n" + + "Comment#11\n" + + "Comment#12\n" + + "Comment#13\n" + + "Comment#14\n" + + "Comment#15\n" + } +} diff --git a/pom.xml b/pom.xml index f215fe4b5fd48..6af03550aed10 100644 --- a/pom.xml +++ b/pom.xml @@ -224,6 +224,12 @@ under the License. 3.2.1 + + org.apache.commons + commons-math3 + 3.5 + + org.javassist From a82bd4311240c3eca1656a05b149d85734fd6196 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 21 Aug 2015 17:58:50 +0200 Subject: [PATCH 156/175] [tests] Move UnionClosedBranchingTest to optimizer tests --- .../flink/optimizer/dataexchange}/UnionClosedBranchingTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename {flink-tests/src/test/java/org/apache/flink/test => flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange}/UnionClosedBranchingTest.java (99%) diff --git a/flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/UnionClosedBranchingTest.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java rename to flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/UnionClosedBranchingTest.java index f7ea9119d7984..b870a9182b593 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/UnionClosedBranchingTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/UnionClosedBranchingTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.test; +package org.apache.flink.optimizer.dataexchange; import org.apache.flink.api.common.ExecutionMode; import org.apache.flink.api.java.DataSet; From 3a8302998d2fc7f38504c238916bc7d0dada2320 Mon Sep 17 00:00:00 2001 From: mjsax Date: Thu, 13 Aug 2015 08:56:47 +0200 Subject: [PATCH 157/175] [FLINK-2306] Add support for named streams in Storm compatibility layer - enabled .declareStream() and connect via stream name - enabled multiplt output streams - added .split() / .select() / strip pattern - added helpers in new package utils - adapted and extended JUnit tests - adapted examples some minor improvements (FlinkClient, integration of Tuple0) This closes #1011 --- docs/apis/storm_compatibility.md | 40 +++- .../flink-storm-compatibility-core/README.md | 1 - .../stormcompatibility/api/FlinkClient.java | 24 ++- .../api/FlinkOutputFieldsDeclarer.java | 70 +++---- .../api/FlinkSubmitter.java | 2 - .../api/FlinkTopologyBuilder.java | 182 ++++++++++++------ .../util/FlinkStormStreamSelector.java | 48 +++++ .../util/SplitStreamMapper.java | 39 ++++ .../util/SplitStreamType.java | 52 +++++ .../wrappers/AbstractStormCollector.java | 118 +++++++----- .../wrappers/AbstractStormSpoutWrapper.java | 41 ++-- .../wrappers/FiniteStormSpoutWrapper.java | 85 +++++--- .../wrappers/StormBoltCollector.java | 21 +- .../wrappers/StormBoltWrapper.java | 136 +++++++++---- .../wrappers/StormFiniteSpoutWrapper.java | 138 ++++++++----- .../wrappers/StormOutputFieldsDeclarer.java | 31 +-- .../wrappers/StormSpoutCollector.java | 24 +-- .../wrappers/StormSpoutWrapper.java | 68 +++++-- .../wrappers/StormWrapperSetupHelper.java | 66 +++---- .../api/FlinkOutputFieldsDeclarerTest.java | 113 ++++++----- .../api/FlinkTopologyBuilderTest.java | 48 +++++ .../stormcompatibility/api/TestBolt.java | 48 +++++ .../stormcompatibility/api/TestSpout.java | 59 ++++++ .../FlinkStormStreamSelectorTest.java | 51 +++++ .../wrappers/StormBoltCollectorTest.java | 26 +-- .../wrappers/StormBoltWrapperTest.java | 118 ++++++++++-- .../StormOutputFieldsDeclarerTest.java | 37 ++-- .../wrappers/StormSpoutCollectorTest.java | 22 ++- .../wrappers/StormTupleTest.java | 2 +- .../wrappers/StormWrapperSetupHelperTest.java | 47 +++-- .../pom.xml | 3 + .../excamation/ExclamationWithStormBolt.java | 7 +- .../excamation/ExclamationWithStormSpout.java | 12 +- .../split/SpoutSplitExample.java | 102 ++++++++++ .../split/stormoperators/RandomSpout.java | 76 ++++++++ .../stormoperators/VerifyAndEnrichBolt.java | 61 ++++++ .../wordcount/SpoutSourceWordCount.java | 9 +- .../split/BoltSplitITCase.java | 28 +++ .../stormcompatibility/split/SplitBolt.java | 61 ++++++ .../split/SplitBoltTopology.java | 87 +++++++++ .../split/SplitSpoutTopology.java | 85 ++++++++ .../split/SpoutSplitITCase.java | 28 +++ .../split/StormSplitStreamBoltLocal.java | 51 +++++ .../split/StormSplitStreamSpoutLocal.java | 51 +++++ 44 files changed, 1918 insertions(+), 500 deletions(-) create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java diff --git a/docs/apis/storm_compatibility.md b/docs/apis/storm_compatibility.md index b38667bfc2e7e..3a0c025eb08e7 100644 --- a/docs/apis/storm_compatibility.md +++ b/docs/apis/storm_compatibility.md @@ -49,7 +49,9 @@ Add the following dependency to your `pom.xml` if you want to execute Storm code ~~~ -**Please note**: `flink-storm-compatibility-core` is not part of the provided binary Flink distribution. Thus, you need to include `flink-storm-compatiblitly-core` classes (and their dependencies) in your program jar that is submitted to Flink's JobManager. +**Please note**: `flink-storm-compatibility-core` is not part of the provided binary Flink distribution. +Thus, you need to include `flink-storm-compatiblitly-core` classes (and their dependencies) in your program jar that is submitted to Flink's JobManager. +See *WordCount Storm* within `flink-storm-compatibility-example/pom.xml` for an example how to package a jar correctly. # Execute Storm Topologies @@ -93,7 +95,7 @@ if(runLocal) { // submit to test cluster As an alternative, Spouts and Bolts can be embedded into regular streaming programs. The Storm compatibility layer offers a wrapper classes for each, namely `StormSpoutWrapper` and `StormBoltWrapper` (`org.apache.flink.stormcompatibility.wrappers`). -Per default, both wrappers convert Storm output tuples to Flink's [Tuple](programming_guide.html#tuples-and-case-classes) types (ie, `Tuple1` to `Tuple25` according to the number of fields of the Storm tuples). +Per default, both wrappers convert Storm output tuples to Flink's [Tuple](programming_guide.html#tuples-and-case-classes) types (ie, `Tuple0` to `Tuple25` according to the number of fields of the Storm tuples). For single field output tuples a conversion to the field's data type is also possible (eg, `String` instead of `Tuple1`). Because Flink cannot infer the output field types of Storm operators, it is required to specify the output type manually. @@ -112,7 +114,7 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm // stream has `raw` type (single field output streams only) DataStream rawInput = env.addSource( - new StormSpoutWrapper(new StormFileSpout(localFilePath), true), // Spout source, 'true' for raw type + new StormSpoutWrapper(new StormFileSpout(localFilePath), new String[] { Utils.DEFAULT_STREAM_ID }), // emit default output stream as raw type TypeExtractor.getForClass(String.class)); // output type // process data stream @@ -167,6 +169,38 @@ The input type is `Tuple1` and `Fields("sentence")` specify that `input. See [BoltTokenizerWordCountPojo](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java) and [BoltTokenizerWordCountWithNames](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java) for examples. +## Multiple Output Streams + +Flink can also handle the declaration of multiple output streams for Spouts and Bolts. +If a whole topology is executed using `FlinkTopologyBuilder` etc., there is no special attention required – it works as in regular Storm. +For embedded usage, the output stream will be of data type `SplitStreamType` and must be split by using `DataStream.split(...)` and `SplitDataStream.select(...)`. +Flink provides the predefined output selector `FlinkStormStreamSelector` for `.split(...)` already. +Furthermore, the wrapper type `SplitStreamTuple` can be removed using `SplitStreamMapper`. +If a data stream of type `SplitStreamTuple` is used as input for a Bolt, `SplitStreamTuple` must **not** be removed – `StormBoltWrapper` removes it automatically. + +

    +
    +~~~java +[...] + +// get DataStream from Spout or Bolt which declares two output streams s1 and s2 with output type SomeType +DataStream> multiStream = ... + +SplitDataStream> splitStream = multiStream.split(new FlinkStormStreamSelector()); + +// remove SplitStreamMapper to get data stream of type SomeType +DataStream s1 = splitStream.select("s1").map(new SplitStreamMapper).returns(SomeType.classs); +// apply Bolt directly, without stripping SplitStreamMapper +DataStream s2 = splitStream.select("s2").transform(/* use Bolt for further processing */); + +// do further processing on s1 and s2 +[...] +~~~ +
    +
    + +See [SpoutSplitExample.java](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java) for a full example. + # Flink Extensions ## Finite Storm Spouts diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md index 04d8934a5005f..aef4847653883 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md @@ -5,7 +5,6 @@ The Storm compatibility layer allows to embed spouts or bolt unmodified within a The following Strom features are not (yet/fully) supported by the compatibility layer right now: * the spout/bolt configuration within `open()`/`prepare()` is not yet supported (ie, `Map conf` parameter) * topology and tuple meta information (ie, `TopologyContext` not fully supported) -* only default stream is supported currently (ie, only a single output stream) * no fault-tolerance guarantees (ie, calls to `ack()`/`fail()` and anchoring is ignored) * for whole Storm topologies the following is not supported by Flink: * direct emit connection pattern diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java index 5a6e8cae3632b..51a4fa115d3fb 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java @@ -30,6 +30,7 @@ import backtype.storm.generated.NotAliveException; import backtype.storm.utils.NimbusClient; import backtype.storm.utils.Utils; + import org.apache.flink.api.common.JobID; import org.apache.flink.client.program.Client; import org.apache.flink.client.program.JobWithJars; @@ -45,6 +46,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus; + import scala.Some; import scala.concurrent.Await; import scala.concurrent.Future; @@ -64,6 +66,9 @@ */ public class FlinkClient { + /** The client's configuration */ + @SuppressWarnings("unused") + private final Map conf; /** The jobmanager's host name */ private final String jobManagerHost; /** The jobmanager's rpc port */ @@ -77,19 +82,24 @@ public class FlinkClient { * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link * Config#NIMBUS_HOST} and {@link Config#NIMBUS_THRIFT_PORT} of the given configuration are ignored. * + * @param conf + * A configuration. * @param host * The jobmanager's host name. * @param port * The jobmanager's rpc port. */ - public FlinkClient(final String host, final int port) { - this(host, port, null); + @SuppressWarnings("rawtypes") + public FlinkClient(final Map conf, final String host, final int port) { + this(conf, host, port, null); } /** * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link * Config#NIMBUS_HOST} and {@link Config#NIMBUS_THRIFT_PORT} of the given configuration are ignored. * + * @param conf + * A configuration. * @param host * The jobmanager's host name. * @param port @@ -97,7 +107,9 @@ public FlinkClient(final String host, final int port) { * @param timeout * Timeout */ - public FlinkClient(final String host, final int port, final Integer timeout) { + @SuppressWarnings("rawtypes") + public FlinkClient(final Map conf, final String host, final int port, final Integer timeout) { + this.conf = conf; this.jobManagerHost = host; this.jobManagerPort = port; if (timeout != null) { @@ -119,7 +131,7 @@ public FlinkClient(final String host, final int port, final Integer timeout) { public static FlinkClient getConfiguredClient(final Map conf) { final String nimbusHost = (String) conf.get(Config.NIMBUS_HOST); final int nimbusPort = Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT)).intValue(); - return new FlinkClient(nimbusHost, nimbusPort); + return new FlinkClient(conf, nimbusHost, nimbusPort); } /** @@ -133,7 +145,7 @@ public FlinkClient getClient() { return this; } - public void close() {/* nothing to do */} + // The following methods are derived from "backtype.storm.generated.Nimubs.Client" /** * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink does not support @@ -220,6 +232,8 @@ public void killTopologyWithOpts(final String name, final KillOptions options) t } } + // Flink specific additional methods + /** * Package internal method to get a Flink {@link JobID} from a Storm topology name. * diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java index 49d73f8d89137..e2f63320d550d 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java @@ -20,10 +20,12 @@ import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; + import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TypeExtractor; +import java.util.HashMap; import java.util.List; /** @@ -36,8 +38,8 @@ */ final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer { - /** the declared output schema */ - Fields outputSchema; + /** the declared output streams and schemas */ + final HashMap outputStreams = new HashMap(); @Override public void declare(final Fields fields) { @@ -57,15 +59,6 @@ public void declare(final boolean direct, final Fields fields) { this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields); } - /** - * {@inheritDoc} - *

    - * Currently, Flink only supports the default output stream. Thus, parameter {@code streamId} must be equals to - * {@link Utils#DEFAULT_STREAM_ID}. - * - * @throws UnsupportedOperationException - * if {@code streamId} is not equal to {@link Utils#DEFAULT_STREAM_ID} - */ @Override public void declareStream(final String streamId, final Fields fields) { this.declareStream(streamId, false, fields); @@ -74,40 +67,45 @@ public void declareStream(final String streamId, final Fields fields) { /** * {@inheritDoc} *

    - * Currently, Flink only supports the default output stream. Thus, parameter {@code streamId} must be equals to - * {@link Utils#DEFAULT_STREAM_ID}. Furthermore, direct emit is no supported by Flink and parameter {@code direct} - * must be {@code false}. + * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}. * * @throws UnsupportedOperationException - * if {@code streamId} is not equal to {@link Utils#DEFAULT_STREAM_ID} or {@code direct} is {@code true} + * if {@code direct} is {@code true} */ @Override public void declareStream(final String streamId, final boolean direct, final Fields fields) { - if (!Utils.DEFAULT_STREAM_ID.equals(streamId)) { - throw new UnsupportedOperationException("Currently, only the default output stream is supported by Flink"); - } if (direct) { throw new UnsupportedOperationException("Direct emit is not supported by Flink"); } - this.outputSchema = fields; + this.outputStreams.put(streamId, fields); } /** - * Returns {@link TypeInformation} for the declared output schema. If no or an empty output schema was declared, - * {@code null} is returned. - * - * @return output type information for the declared output schema; or {@code null} if no output schema was declared + * Returns {@link TypeInformation} for the declared output schema for a specific stream. + * + * @param streamId + * A stream ID. + * + * @return output type information for the declared output schema of the specified stream; or {@code null} if + * {@code streamId == null} + * * @throws IllegalArgumentException - * if more then 25 attributes are declared + * If no output schema was declared for the specified stream or if more then 25 attributes got declared. */ - public TypeInformation getOutputType() throws IllegalArgumentException { - if ((this.outputSchema == null) || (this.outputSchema.size() == 0)) { + public TypeInformation getOutputType(final String streamId) throws IllegalArgumentException { + if (streamId == null) { return null; } + Fields outputSchema = this.outputStreams.get(streamId); + if (outputSchema == null) { + throw new IllegalArgumentException("Stream with ID '" + streamId + + "' was not declared."); + } + Tuple t; - final int numberOfAttributes = this.outputSchema.size(); + final int numberOfAttributes = outputSchema.size(); if (numberOfAttributes == 1) { return TypeExtractor.getForClass(Object.class); @@ -148,16 +146,22 @@ public int compareTo(final DefaultComparable o) { } /** - * Computes the indexes within the declared output schema, for a list of given field-grouping attributes. - * - * @return array of {@code int}s that contains the index without the output schema for each attribute in the given - * list + * Computes the indexes within the declared output schema of the specified stream, for a list of given + * field-grouping attributes. + * + * @param streamId + * A stream ID. + * @param groupingFields + * The names of the key fields. + * + * @return array of {@code int}s that contains the index within the output schema for each attribute in the given + * list */ - public int[] getGroupingFieldIndexes(final List groupingFields) { + public int[] getGroupingFieldIndexes(final String streamId, final List groupingFields) { final int[] fieldIndexes = new int[groupingFields.size()]; for (int i = 0; i < fieldIndexes.length; ++i) { - fieldIndexes[i] = this.outputSchema.fieldIndex(groupingFields.get(i)); + fieldIndexes[i] = this.outputStreams.get(streamId).fieldIndex(groupingFields.get(i)); } return fieldIndexes; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java index 819dbbca428c0..bcc2afb1c6d20 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java @@ -125,8 +125,6 @@ public static void submitTopology(final String name, final Map stormConf, final } catch (final AlreadyAliveException e) { logger.warn("Topology already alive exception", e); throw e; - } finally { - client.close(); } logger.info("Finished submitting topology: " + name); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java index d1462506e21a7..a739c23853791 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java @@ -33,6 +33,9 @@ import backtype.storm.tuple.Fields; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; +import org.apache.flink.stormcompatibility.util.SplitStreamType; import org.apache.flink.stormcompatibility.wrappers.AbstractStormSpoutWrapper; import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; @@ -41,6 +44,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.datastream.SplitDataStream; import java.util.HashMap; import java.util.HashSet; @@ -54,8 +58,7 @@ * topology. Most methods (except {@link #createTopology()} are copied from the original {@link TopologyBuilder} * implementation to ensure equal behavior.
    *
    - * CAUTION: {@link IRichStateSpout StateSpout}s and multiple output streams per spout/bolt are currently not - * supported. + * CAUTION: {@link IRichStateSpout StateSpout}s are currently not supported. */ public class FlinkTopologyBuilder { @@ -65,13 +68,13 @@ public class FlinkTopologyBuilder { private final HashMap spouts = new HashMap(); /** All user bolts by their ID */ private final HashMap bolts = new HashMap(); - /** All declared output schemas by operator ID */ - private final HashMap outputSchemas = new HashMap(); + /** All declared streams and output schemas by operator ID */ + private final HashMap> outputStreams = new HashMap>(); /** All spouts&bolts declarers by their ID */ private final HashMap declarers = new HashMap(); /** - * Creates a Flink program that used the specified spouts and bolts. + * Creates a Flink program that uses the specified spouts and bolts. */ @SuppressWarnings({"rawtypes", "unchecked"}) public FlinkTopology createTopology() { @@ -79,8 +82,7 @@ public FlinkTopology createTopology() { final FlinkTopology env = new FlinkTopology(stormTopolgoy); env.setParallelism(1); - final HashMap availableOperators = - new HashMap(); + final HashMap> availableInputs = new HashMap>(); for (final Entry spout : this.spouts.entrySet()) { final String spoutId = spout.getKey(); @@ -88,14 +90,10 @@ public FlinkTopology createTopology() { final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); userSpout.declareOutputFields(declarer); - this.outputSchemas.put(spoutId, declarer.outputSchema); + final HashMap sourceStreams = declarer.outputStreams; + this.outputStreams.put(spoutId, sourceStreams); declarers.put(spoutId, declarer); - /* TODO in order to support multiple output streams, use an additional wrapper (or modify StormSpoutWrapper - * and StormCollector) - * -> add an additional output attribute tagging the output stream, and use .split() and .select() to split - * the streams - */ AbstractStormSpoutWrapper spoutWrapper; if (userSpout instanceof FiniteStormSpout) { @@ -104,8 +102,23 @@ public FlinkTopology createTopology() { spoutWrapper = new StormSpoutWrapper(userSpout); } - final DataStreamSource source = env.addSource(spoutWrapper, declarer.getOutputType()); - availableOperators.put(spoutId, source); + DataStreamSource source; + HashMap outputStreams = new HashMap(); + if (sourceStreams.size() == 1) { + final String outputStreamId = (String) sourceStreams.keySet().toArray()[0]; + source = env.addSource(spoutWrapper, spoutId, + declarer.getOutputType(outputStreamId)); + outputStreams.put(outputStreamId, source); + } else { + source = env.addSource(spoutWrapper, spoutId, + TypeExtractor.getForClass(SplitStreamType.class)); + SplitDataStream splitSource = source.split(new FlinkStormStreamSelector()); + + for (String streamId : sourceStreams.keySet()) { + outputStreams.put(streamId, splitSource.select(streamId)); + } + } + availableInputs.put(spoutId, outputStreams); int dop = 1; final ComponentCommon common = stormTopolgoy.get_spouts().get(spoutId).get_common(); @@ -126,7 +139,14 @@ public FlinkTopology createTopology() { * its producer * ->thus, we might need to repeat multiple times */ + boolean makeProgress = true; while (unprocessedBolts.size() > 0) { + if (!makeProgress) { + throw new RuntimeException( + "Unable to build Topology. Could not connect the following bolts: " + + unprocessedBolts.keySet()); + } + makeProgress = false; final Iterator> boltsIterator = unprocessedBolts.entrySet().iterator(); while (boltsIterator.hasNext()) { @@ -135,11 +155,6 @@ public FlinkTopology createTopology() { final String boltId = bolt.getKey(); final IRichBolt userBolt = bolt.getValue(); - final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); - userBolt.declareOutputFields(declarer); - this.outputSchemas.put(boltId, declarer.outputSchema); - declarers.put(boltId, declarer); - final ComponentCommon common = stormTopolgoy.get_bolts().get(boltId).get_common(); Set> unprocessedInputs = unprocessdInputsPerBolt.get(boltId); @@ -153,51 +168,98 @@ public FlinkTopology createTopology() { final Iterator> inputStreamsIterator = unprocessedInputs.iterator(); while (inputStreamsIterator.hasNext()) { - final Entry inputStream = inputStreamsIterator.next(); - final String producerId = inputStream.getKey().get_componentId(); - - DataStream inputDataStream = availableOperators.get(producerId); - - if (inputDataStream != null) { - // if producer was processed already - final Grouping grouping = inputStream.getValue(); - if (grouping.is_set_shuffle()) { - // Storm uses a round-robin shuffle strategy - inputDataStream = inputDataStream.rebalance(); - } else if (grouping.is_set_fields()) { - // global grouping is emulated in Storm via an empty fields grouping list - final List fields = grouping.get_fields(); - if (fields.size() > 0) { - FlinkOutputFieldsDeclarer procDeclarer = this.declarers.get(producerId); - inputDataStream = inputDataStream.groupBy(procDeclarer.getGroupingFieldIndexes(grouping - .get_fields())); - } else { - inputDataStream = inputDataStream.global(); + final Entry stormInputStream = inputStreamsIterator.next(); + final String producerId = stormInputStream.getKey().get_componentId(); + final String inputStreamId = stormInputStream.getKey().get_streamId(); + + HashMap producer = availableInputs.get(producerId); + if (producer != null) { + makeProgress = true; + + DataStream inputStream = producer.get(inputStreamId); + if (inputStream != null) { + final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); + userBolt.declareOutputFields(declarer); + final HashMap boltOutputStreams = declarer.outputStreams; + this.outputStreams.put(boltId, boltOutputStreams); + this.declarers.put(boltId, declarer); + + // if producer was processed already + final Grouping grouping = stormInputStream.getValue(); + if (grouping.is_set_shuffle()) { + // Storm uses a round-robin shuffle strategy + inputStream = inputStream.rebalance(); + } else if (grouping.is_set_fields()) { + // global grouping is emulated in Storm via an empty fields grouping list + final List fields = grouping.get_fields(); + if (fields.size() > 0) { + FlinkOutputFieldsDeclarer prodDeclarer = this.declarers.get(producerId); + inputStream = inputStream.groupBy(prodDeclarer + .getGroupingFieldIndexes(inputStreamId, + grouping.get_fields())); + } else { + inputStream = inputStream.global(); + } + } else if (grouping.is_set_all()) { + inputStream = inputStream.broadcast(); + } else if (!grouping.is_set_local_or_shuffle()) { + throw new UnsupportedOperationException( + "Flink only supports (local-or-)shuffle, fields, all, and global grouping"); } - } else if (grouping.is_set_all()) { - inputDataStream = inputDataStream.broadcast(); - } else if (!grouping.is_set_local_or_shuffle()) { - throw new UnsupportedOperationException( - "Flink only supports (local-or-)shuffle, fields, all, and global grouping"); - } - final TypeInformation outType = declarer.getOutputType(); + SingleOutputStreamOperator outputStream; + if (boltOutputStreams.size() < 2) { // single output stream or sink + String outputStreamId = null; + if (boltOutputStreams.size() == 1) { + outputStreamId = (String) boltOutputStreams.keySet().toArray()[0]; + } + final TypeInformation outType = declarer + .getOutputType(outputStreamId); + + outputStream = inputStream.transform( + boltId, + outType, + new StormBoltWrapper(userBolt, this.outputStreams.get( + producerId).get(inputStreamId))); + + if (outType != null) { + // only for non-sink nodes + HashMap op = new HashMap(); + op.put(outputStreamId, outputStream); + availableInputs.put(boltId, op); + } + } else { + final TypeInformation outType = TypeExtractor + .getForClass(SplitStreamType.class); + + outputStream = inputStream.transform( + boltId, + outType, + new StormBoltWrapper(userBolt, this.outputStreams.get( + producerId).get(inputStreamId))); + + SplitDataStream splitStreams = outputStream + .split(new FlinkStormStreamSelector()); + + HashMap op = new HashMap(); + for (String outputStreamId : boltOutputStreams.keySet()) { + op.put(outputStreamId, splitStreams.select(outputStreamId)); + } + availableInputs.put(boltId, op); + } - final SingleOutputStreamOperator operator = inputDataStream.transform(boltId, outType, - new StormBoltWrapper(userBolt, this.outputSchemas.get(producerId))); - if (outType != null) { - // only for non-sink nodes - availableOperators.put(boltId, operator); - } + int dop = 1; + if (common.is_set_parallelism_hint()) { + dop = common.get_parallelism_hint(); + outputStream.setParallelism(dop); + } + env.increaseNumberOfTasks(dop); - int dop = 1; - if (common.is_set_parallelism_hint()) { - dop = common.get_parallelism_hint(); - operator.setParallelism(dop); + inputStreamsIterator.remove(); + } else { + throw new RuntimeException("Cannot connect '" + boltId + "' to '" + + producerId + "'. Stream '" + inputStreamId + "' not found."); } - env.increaseNumberOfTasks(dop); - - inputStreamsIterator.remove(); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java new file mode 100644 index 0000000000000..7ca45d6db2714 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java @@ -0,0 +1,48 @@ +/* + * 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.stormcompatibility.util; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; + +/** + * Used by {@link FlinkTopologyBuilder} to split multiple declared output streams within Flink. + */ +final public class FlinkStormStreamSelector implements OutputSelector> { + private static final long serialVersionUID = 2553423379715401023L; + + /** internal cache to avoid short living ArrayList objects. */ + private final HashMap> streams = new HashMap>(); + + @Override + public Iterable select(SplitStreamType value) { + String sid = value.streamId; + List streamId = this.streams.get(sid); + if (streamId == null) { + streamId = new ArrayList(1); + streamId.add(sid); + this.streams.put(sid, streamId); + } + return streamId; + } + +} \ No newline at end of file diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java new file mode 100644 index 0000000000000..afcdcaed3a40f --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java @@ -0,0 +1,39 @@ +/* + * 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.stormcompatibility.util; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SplitDataStream; + +/** + * Strips {@link SplitStreamType}{@code } away, ie, extracts the wrapped record of type {@code T}. Can be used to get + * a "clean" stream from a Spout/Bolt that declared multiple output streams (after the streams got separated using + * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} and + * {@link SplitDataStream#select(String...) .select(...)}). + * + * @param + */ +public class SplitStreamMapper implements MapFunction, T> { + private static final long serialVersionUID = 3550359150160908564L; + + @Override + public T map(SplitStreamType value) throws Exception { + return value.value; + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java new file mode 100644 index 0000000000000..9c7e47759de70 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java @@ -0,0 +1,52 @@ +/* + * 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.stormcompatibility.util; + +import org.apache.flink.streaming.api.datastream.DataStream; + +/** + * Used by {@link org.apache.flink.stormcompatibility.wrappers.AbstractStormCollector AbstractStormCollector} to wrap + * output tuples if multiple output streams are declared. For this case, the Flink output data stream must be split via + * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} using + * {@link FlinkStormStreamSelector}. + */ +public class SplitStreamType { + + /** The stream ID this tuple belongs to. */ + public String streamId; + /** The actual data value. */ + public T value; + + @Override + public String toString() { + return ""; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SplitStreamType other = (SplitStreamType) o; + + return this.streamId.equals(other.streamId) && this.value.equals(other.value); + } +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java index 4a8fb7de929fe..7b35a64dd117a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java @@ -17,82 +17,114 @@ package org.apache.flink.stormcompatibility.wrappers; import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple25; +import org.apache.flink.stormcompatibility.util.SplitStreamType; + +import java.util.HashMap; import java.util.List; +import java.util.Map.Entry; /** * A {@link AbstractStormCollector} transforms Storm tuples to Flink tuples. */ abstract class AbstractStormCollector { + /** Flink output tuple of concrete type {@link Tuple0} to {@link Tuple25} per output stream. */ + protected final HashMap outputTuple = new HashMap(); + /** Flink split tuple. Used, if multiple output streams are declared. */ + private final SplitStreamType splitTuple = new SplitStreamType(); /** - * Flink output tuple of concrete type {@link Tuple1} to {@link Tuple25}. - */ - protected final Tuple outputTuple; - /** - * The number of attributes of the output tuples. (Determines the concrete type of - * {@link #outputTuple}). If {@link #numberOfAttributes} is zero, {@link #outputTuple} is not - * used and "raw" data type is used. - */ - protected final int numberOfAttributes; - /** - * Is set to {@code true} each time a tuple is emitted. + * The number of attributes of the output tuples per stream. (Determines the concrete type of {@link #outputTuple}). + * If {@link #numberOfAttributes} is zero, {@link #outputTuple} is not used and "raw" data type is used. */ + protected final HashMap numberOfAttributes; + /** Indicates of multiple output stream are declared and thus {@link SplitStreamType} must be used as output. */ + private final boolean split; + /** Is set to {@code true} each time a tuple is emitted. */ boolean tupleEmitted = false; /** - * Instantiates a new {@link AbstractStormCollector} that emits Flink tuples via - * {@link #doEmit(Object)}. If the number of attributes is specified as zero, any output type is - * supported. If the number of attributes is between 1 to 25, the output type is {@link Tuple1} - * to {@link Tuple25}. + * Instantiates a new {@link AbstractStormCollector} that emits Flink tuples via {@link #doEmit(Object)}. If the + * number of attributes is negative, any output type is supported (ie, raw type). If the number of attributes is + * between 0 and 25, the output type is {@link Tuple0} to {@link Tuple25}, respectively. * * @param numberOfAttributes - * The number of attributes of the emitted tuples. + * The number of attributes of the emitted tuples per output stream. * @throws UnsupportedOperationException - * if the specified number of attributes is not in the valid range of [0,25] + * if the specified number of attributes is greater than 25 */ - public AbstractStormCollector(final int numberOfAttributes) throws UnsupportedOperationException { + public AbstractStormCollector(final HashMap numberOfAttributes) + throws UnsupportedOperationException { + assert (numberOfAttributes != null); + this.numberOfAttributes = numberOfAttributes; + this.split = this.numberOfAttributes.size() > 1; + + for (Entry outputStream : numberOfAttributes.entrySet()) { + final int numAtt = outputStream.getValue(); + assert (numAtt >= -1); + + if (numAtt > 25) { + throw new UnsupportedOperationException( + "Flink cannot handle more then 25 attributes, but " + numAtt + + " are declared for stream '" + outputStream.getKey() + + "' by the given bolt"); + } else if (numAtt >= 0) { + try { + this.outputTuple.put(outputStream.getKey(), + org.apache.flink.api.java.tuple.Tuple.getTupleClass(numAtt) + .newInstance()); + } catch (final InstantiationException e) { + throw new RuntimeException(e); + } catch (final IllegalAccessException e) { + throw new RuntimeException(e); + } - if (this.numberOfAttributes <= 0) { - this.outputTuple = null; - } else if (this.numberOfAttributes <= 25) { - try { - this.outputTuple = org.apache.flink.api.java.tuple.Tuple - .getTupleClass(this.numberOfAttributes).newInstance(); - } catch (final InstantiationException e) { - throw new RuntimeException(e); - } catch (final IllegalAccessException e) { - throw new RuntimeException(e); } - } else { - throw new UnsupportedOperationException( - "Flink cannot handle more then 25 attributes, but " - + this.numberOfAttributes + " are declared by the given bolt"); } } /** - * Transforms a Storm tuple into a Flink tuple of type {@code OUT} and emits this tuple via - * {@link #doEmit(Object)}. + * Transforms a Storm tuple into a Flink tuple of type {@code OUT} and emits this tuple via {@link #doEmit(Object)} + * to the specified output stream. * + * @param The + * The output stream id. * @param tuple - * The Storm tuple to be emitted. + * The Storm tuple to be emitted. * @return the return value of {@link #doEmit(Object)} */ @SuppressWarnings("unchecked") - protected final List transformAndEmit(final List tuple) { + protected final List tansformAndEmit(final String streamId, final List tuple) { List taskIds; - if (this.numberOfAttributes > 0) { - assert (tuple.size() == this.numberOfAttributes); - for (int i = 0; i < this.numberOfAttributes; ++i) { - this.outputTuple.setField(tuple.get(i), i); + + final int numAtt = this.numberOfAttributes.get(streamId); + if (numAtt > -1) { + assert (tuple.size() == numAtt); + Tuple out = this.outputTuple.get(streamId); + for (int i = 0; i < numAtt; ++i) { + out.setField(tuple.get(i), i); + } + if (this.split) { + this.splitTuple.streamId = streamId; + this.splitTuple.value = out; + + taskIds = doEmit((OUT) this.splitTuple); + } else { + taskIds = doEmit((OUT) out); } - taskIds = doEmit((OUT) this.outputTuple); + } else { assert (tuple.size() == 1); - taskIds = doEmit((OUT) tuple.get(0)); + if (split) { + this.splitTuple.streamId = streamId; + this.splitTuple.value = tuple.get(0); + + taskIds = doEmit((OUT) this.splitTuple); + } else { + taskIds = doEmit((OUT) tuple.get(0)); + } } this.tupleEmitted = true; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java index 4e43a8a55d55a..62059feaddef5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java @@ -17,8 +17,13 @@ package org.apache.flink.stormcompatibility.wrappers; +import java.util.Collection; +import java.util.HashMap; + import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.topology.IRichSpout; + +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; @@ -37,9 +42,9 @@ public abstract class AbstractStormSpoutWrapper extends RichParallelSourceF private static final long serialVersionUID = 4993283609095408765L; /** - * Number of attributes of the bolt's output tuples. + * Number of attributes of the bolt's output tuples per stream. */ - private final int numberOfAttributes; + private final HashMap numberOfAttributes; /** * The wrapped Storm {@link IRichSpout spout}. */ @@ -55,38 +60,40 @@ public abstract class AbstractStormSpoutWrapper extends RichParallelSourceF /** * Instantiates a new {@link AbstractStormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such - * that it can be used within a Flink streaming program. The output type will be one of {@link Tuple1} to + * that it can be used within a Flink streaming program. The output type will be one of {@link Tuple0} to * {@link Tuple25} depending on the spout's declared number of attributes. * * @param spout * The Storm {@link IRichSpout spout} to be used. * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [1;25]. + * If the number of declared output attributes is not with range [0;25]. */ public AbstractStormSpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { - this(spout, false); + this(spout, null); } /** * Instantiates a new {@link AbstractStormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such * that it can be used within a Flink streaming program. The output type can be any type if parameter * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is - * {@code false} the output type will be one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared * number of attributes. - * + * * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public AbstractStormSpoutWrapper(final IRichSpout spout, final boolean rawOutput) throws IllegalArgumentException { + public AbstractStormSpoutWrapper(final IRichSpout spout, + final Collection rawOutputs) + throws IllegalArgumentException { this.spout = spout; - this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(spout, rawOutput); + this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(spout, rawOutputs); } @Override @@ -94,7 +101,7 @@ public final void run(final SourceContext ctx) throws Exception { this.collector = new StormSpoutCollector(this.numberOfAttributes, ctx); this.spout.open(null, StormWrapperSetupHelper - .convertToTopologyContext((StreamingRuntimeContext) super.getRuntimeContext(), true), + .convertToTopologyContext((StreamingRuntimeContext) super.getRuntimeContext(), true), new SpoutOutputCollector(this.collector)); this.spout.activate(); this.execute(); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java index 79135109b6e39..1912afc26e957 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java @@ -17,6 +17,14 @@ package org.apache.flink.stormcompatibility.wrappers; +import java.util.Collection; + +import org.apache.flink.api.java.tuple.Tuple0; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple25; + +import com.google.common.collect.Sets; + /** * A {@link FiniteStormSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls the wrapped * {@link FiniteStormSpout}'s {@link FiniteStormSpout#nextTuple()} method until {@link @@ -28,16 +36,14 @@ public class FiniteStormSpoutWrapper extends AbstractStormSpoutWrapper private FiniteStormSpout finiteSpout; /** - * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link - * FiniteStormSpout spout} such that it can be used within a Flink streaming program. The - * output - * type will be one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared - * number of attributes. - * + * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link FiniteStormSpout spout} such + * that it can be used within a Flink streaming program. The output type will be one of {@link Tuple0} to + * {@link Tuple25} depending on the spout's declared number of attributes. + * * @param spout - * The Storm {@link FiniteStormSpout spout} to be used. @throws - * IllegalArgumentException If - * the number of declared output attributes is not with range [1;25]. + * The Storm {@link FiniteStormSpout spout} to be used. + * @throws IllegalArgumentException + * If the number of declared output attributes is not with range [0;25]. */ public FiniteStormSpoutWrapper(FiniteStormSpout spout) throws IllegalArgumentException { @@ -46,36 +52,53 @@ public FiniteStormSpoutWrapper(FiniteStormSpout spout) } /** - * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link - * FiniteStormSpout spout} such that it can be used within a Flink streaming program. The - * output - * type can be any type if parameter {@code rawOutput} is {@code true} and the spout's - * number of - * declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be - * one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared number of - * attributes. - * + * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link FiniteStormSpout spout} such + * that it can be used within a Flink streaming program. The output type can be any type if parameter + * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * + * @param spout + * The Storm {@link FiniteStormSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public FiniteStormSpoutWrapper(final FiniteStormSpout spout, final String[] rawOutputs) + throws IllegalArgumentException { + this(spout, Sets.newHashSet(rawOutputs)); + } + + /** + * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link FiniteStormSpout spout} such + * that it can be used within a Flink streaming program. The output type can be any type if parameter + * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * * @param spout - * The Storm {@link FiniteStormSpout spout} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type {@link - * Tuple1} but be of a raw type. + * The Storm {@link FiniteStormSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is - * not 1 - * or if {@code rawOuput} is {@code false} and the number of declared output attributes - * is not - * with range [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public FiniteStormSpoutWrapper(final FiniteStormSpout spout, final boolean rawOutput) + public FiniteStormSpoutWrapper(final FiniteStormSpout spout, final Collection rawOutputs) throws IllegalArgumentException { - super(spout, rawOutput); + super(spout, rawOutputs); this.finiteSpout = spout; } /** - * Calls the {@link FiniteStormSpout#nextTuple()} method until {@link - * FiniteStormSpout#reachedEnd()} is true or {@link FiniteStormSpout#cancel()} is called. + * Calls the {@link FiniteStormSpout#nextTuple()} method until {@link FiniteStormSpout#reachedEnd()} is true or + * {@link FiniteStormSpout#cancel()} is called. */ @Override protected void execute() { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java index 81ad9a6921a12..e810214fe92e5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java @@ -20,12 +20,13 @@ import backtype.storm.task.IOutputCollector; import backtype.storm.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.util.Collector; import java.util.Collection; +import java.util.HashMap; import java.util.List; /** @@ -39,19 +40,19 @@ class StormBoltCollector extends AbstractStormCollector implements IOu private final Collector flinkOutput; /** - * Instantiates a new {@link StormBoltCollector} that emits Flink tuples to the given Flink - * output object. If the number of attributes is specified as zero, any output type is - * supported. If the number of attributes is between 1 to 25, the output type is {@link Tuple1} - * to {@link Tuple25}. + * Instantiates a new {@link StormBoltCollector} that emits Flink tuples to the given Flink output object. If the + * number of attributes is negative, any output type is supported (ie, raw type). If the number of attributes is + * between 0 and 25, the output type is {@link Tuple0} to {@link Tuple25}, respectively. * * @param numberOfAttributes - * The number of attributes of the emitted tuples. + * The number of attributes of the emitted tuples per output stream. * @param flinkOutput - * The Flink output object to be used. + * The Flink output object to be used. * @throws UnsupportedOperationException - * if the specified number of attributes is not in the valid range of [0,25] + * if the specified number of attributes is greater than 25 */ - public StormBoltCollector(final int numberOfAttributes, final Collector flinkOutput) throws UnsupportedOperationException { + public StormBoltCollector(final HashMap numberOfAttributes, + final Collector flinkOutput) throws UnsupportedOperationException { super(numberOfAttributes); assert (flinkOutput != null); this.flinkOutput = flinkOutput; @@ -72,7 +73,7 @@ public void reportError(final Throwable error) { @Override public List emit(final String streamId, final Collection anchors, final List tuple) { - return this.transformAndEmit(tuple); + return this.tansformAndEmit(streamId, tuple); } @Override diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java index 8bcdae06a78a2..05a490220e3a5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java @@ -16,23 +16,26 @@ */ package org.apache.flink.stormcompatibility.wrappers; +import java.util.Collection; +import java.util.HashMap; + import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichBolt; import backtype.storm.tuple.Fields; +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.configuration.Configuration; +import org.apache.flink.stormcompatibility.util.SplitStreamType; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; - - - +import com.google.common.collect.Sets; /** * A {@link StormBoltWrapper} wraps an {@link IRichBolt} in order to execute the Storm bolt within a Flink Streaming @@ -48,10 +51,10 @@ public class StormBoltWrapper extends AbstractStreamOperator implements OneInputStreamOperator { private static final long serialVersionUID = -4788589118464155835L; - /** The wrapped Storm {@link IRichBolt bolt} */ + /** The wrapped Storm {@link IRichBolt bolt}. */ private final IRichBolt bolt; - /** Number of attributes of the bolt's output tuples */ - private final int numberOfAttributes; + /** Number of attributes of the bolt's output tuples per stream. */ + private final HashMap numberOfAttributes; /** The schema (ie, ordered field names) of the input stream. */ private final Fields inputSchema; @@ -64,34 +67,34 @@ public class StormBoltWrapper extends AbstractStreamOperator imple /** * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. As no input schema is defined, attribute-by-name access in only possible - * for POJO input types. The output type will be one of {@link Tuple1} to {@link Tuple25} depending on the bolt's + * for POJO input types. The output type will be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's * declared number of attributes. * * @param bolt - * The Storm {@link IRichBolt bolt} to be used. + * The Storm {@link IRichBolt bolt} to be used. * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [1;25]. + * If the number of declared output attributes is not with range [0;25]. */ public StormBoltWrapper(final IRichBolt bolt) throws IllegalArgumentException { - this(bolt, null, false); + this(bolt, null, (Collection) null); } /** * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. The given input schema enable attribute-by-name access for input types - * {@link Tuple1} to {@link Tuple25}. The output type will be one of {@link Tuple1} to {@link Tuple25} depending on + * {@link Tuple0} to {@link Tuple25}. The output type will be one of {@link Tuple0} to {@link Tuple25} depending on * the bolt's declared number of attributes. * * @param bolt - * The Storm {@link IRichBolt bolt} to be used. + * The Storm {@link IRichBolt bolt} to be used. * @param inputSchema - * The schema (ie, ordered field names) of the input stream. + * The schema (ie, ordered field names) of the input stream. * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [1;25]. + * If the number of declared output attributes is not with range [0;25]. */ public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema) throws IllegalArgumentException { - this(bolt, inputSchema, false); + this(bolt, inputSchema, (Collection) null); } /** @@ -99,47 +102,93 @@ public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema) * used within a Flink streaming program. As no input schema is defined, attribute-by-name access in only possible * for POJO input types. The output type can be any type if parameter {@code rawOutput} is {@code true} and the * bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one - * of {@link Tuple1} to {@link Tuple25} depending on the bolt's declared number of attributes. + * of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes. * * @param bolt - * The Storm {@link IRichBolt bolt} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type - * {@link Tuple1} but be of a raw type. + * The Storm {@link IRichBolt bolt} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is - * not 1 or if {@code rawOuput} is {@code false} and the number of declared output - * attributes is not with range [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [1;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final boolean rawOutput) + public StormBoltWrapper(final IRichBolt bolt, final String[] rawOutputs) throws IllegalArgumentException { - this(bolt, null, rawOutput); + this(bolt, null, Sets.newHashSet(rawOutputs)); + } + + /** + * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * used within a Flink streaming program. As no input schema is defined, attribute-by-name access in only possible + * for POJO input types. The output type can be any type if parameter {@code rawOutput} is {@code true} and the + * bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one + * of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes. + * + * @param bolt + * The Storm {@link IRichBolt bolt} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [1;25]. + */ + public StormBoltWrapper(final IRichBolt bolt, final Collection rawOutputs) + throws IllegalArgumentException { + this(bolt, null, rawOutputs); } /** * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. The given input schema enable attribute-by-name access for input types - * {@link Tuple1} to {@link Tuple25}. The output type can be any type if parameter {@code rawOutput} is {@code true} + * {@link Tuple0} to {@link Tuple25}. The output type can be any type if parameter {@code rawOutput} is {@code true} * and the bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will - * be one of {@link Tuple1} to {@link Tuple25} depending on the bolt's declared number of attributes. + * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes. * * @param bolt - * The Storm {@link IRichBolt bolt} to be used. + * The Storm {@link IRichBolt bolt} to be used. * @param inputSchema - * The schema (ie, ordered field names) of the input stream. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type - * {@link Tuple1} but be of a raw type. + * The schema (ie, ordered field names) of the input stream. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is - * not 1 or if {@code rawOuput} is {@code false} and the number of declared output - * attributes is not with range [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, final boolean rawOutput) - throws IllegalArgumentException { + public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, + final String[] rawOutputs) throws IllegalArgumentException { + this(bolt, inputSchema, Sets.newHashSet(rawOutputs)); + } + + /** + * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * used within a Flink streaming program. The given input schema enable attribute-by-name access for input types + * {@link Tuple0} to {@link Tuple25}. The output type can be any type if parameter {@code rawOutput} is {@code true} + * and the bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will + * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes. + * + * @param bolt + * The Storm {@link IRichBolt bolt} to be used. + * @param inputSchema + * The schema (ie, ordered field names) of the input stream. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, + final Collection rawOutputs) throws IllegalArgumentException { this.bolt = bolt; this.inputSchema = inputSchema; - this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(bolt, rawOutput); + this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(bolt, rawOutputs); } @Override @@ -151,7 +200,7 @@ public void open(final Configuration parameters) throws Exception { flinkCollector = new TimestampedCollector(output); OutputCollector stormCollector = null; - if (this.numberOfAttributes != -1) { + if (this.numberOfAttributes.size() > 0) { stormCollector = new OutputCollector(new StormBoltCollector( this.numberOfAttributes, flinkCollector)); } @@ -165,10 +214,17 @@ public void close() throws Exception { this.bolt.cleanup(); } + @SuppressWarnings("unchecked") @Override public void processElement(final StreamRecord element) throws Exception { flinkCollector.setTimestamp(element.getTimestamp()); - this.bolt.execute(new StormTuple(element.getValue(), inputSchema)); + IN value = element.getValue(); + if (value instanceof SplitStreamType) { + this.bolt.execute(new StormTuple(((SplitStreamType) value).value, + inputSchema)); + } else { + this.bolt.execute(new StormTuple(value, inputSchema)); + } } @Override diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java index 44b3f683ea16a..45eb56c71fc19 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java @@ -17,17 +17,22 @@ package org.apache.flink.stormcompatibility.wrappers; +import java.util.Collection; + import backtype.storm.topology.IRichSpout; + +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; +import com.google.common.collect.Sets; + /** - * A {@link StormFiniteSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls - * {@link IRichSpout#nextTuple() nextTuple()} for finite number of times before - * {@link #run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext)} - * returns. The number of {@code nextTuple()} calls can be specified as a certain number of - * invocations or can be undefined. In the undefined case, the {@code run(...)} method return if no - * record was emitted to the output collector for the first time. + * A {@link StormFiniteSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls {@link IRichSpout#nextTuple() + * nextTuple()} for finite number of times before + * {@link #run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext)} returns. The number of + * {@code nextTuple()} calls can be specified as a certain number of invocations or can be undefined. In the undefined + * case, the {@code run(...)} method return if no record was emitted to the output collector for the first time. */ public class StormFiniteSpoutWrapper extends AbstractStormSpoutWrapper { private static final long serialVersionUID = 3883246587044801286L; @@ -38,79 +43,126 @@ public class StormFiniteSpoutWrapper extends AbstractStormSpoutWrapper /** * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} * method of the given Storm {@link IRichSpout spout} as long as it emits records to the output collector. The - * output type will be one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared number of + * output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of * attributes. - * + * * @param spout - * The Storm {@link IRichSpout spout} to be used. + * The Storm {@link IRichSpout spout} to be used. * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [1;25]. + * If the number of declared output attributes is not with range [0;25]. */ public StormFiniteSpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { - this(spout, false, -1); + this(spout, (Collection) null, -1); } /** * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type will be - * one - * of {@link Tuple1} to {@link Tuple25} depending on the spout's declared number of attributes. - * + * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type will be one + * of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of attributes. + * * @param spout - * The Storm {@link IRichSpout spout} to be used. + * The Storm {@link IRichSpout spout} to be used. * @param numberOfInvocations - * The number of calls to {@link IRichSpout#nextTuple()}. + * The number of calls to {@link IRichSpout#nextTuple()}. * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [1;25]. + * If the number of declared output attributes is not with range [0;25]. */ public StormFiniteSpoutWrapper(final IRichSpout spout, final int numberOfInvocations) throws IllegalArgumentException { - this(spout, false, numberOfInvocations); + this(spout, (Collection) null, numberOfInvocations); + } + + /** + * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} + * method of the given Storm {@link IRichSpout spout} as long as it emits records to the output collector. The + * output type can be any type if parameter {@code rawOutput} is {@code true} and the spout's number of declared + * output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to + * {@link Tuple25} depending on the spout's declared number of attributes. + * + * @param spout + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public StormFiniteSpoutWrapper(final IRichSpout spout, final String[] rawOutputs) + throws IllegalArgumentException { + this(spout, Sets.newHashSet(rawOutputs), -1); } /** * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} * method of the given Storm {@link IRichSpout spout} as long as it emits records to the output collector. The * output type can be any type if parameter {@code rawOutput} is {@code true} and the spout's number of declared - * output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one of {@link Tuple1} to + * output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to * {@link Tuple25} depending on the spout's declared number of attributes. - * + * * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final boolean rawOutput) throws IllegalArgumentException { - this(spout, rawOutput, -1); + public StormFiniteSpoutWrapper(final IRichSpout spout, final Collection rawOutputs) + throws IllegalArgumentException { + this(spout, rawOutputs, -1); } /** * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type can be any * type if parameter {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If - * {@code rawOutput} is {@code false} the output type will be one of {@link Tuple1} to {@link Tuple25} depending on + * {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on * the spout's declared number of attributes. - * + * * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. * @param numberOfInvocations - * The number of calls to {@link IRichSpout#nextTuple()}. + * The number of calls to {@link IRichSpout#nextTuple()}. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final boolean rawOutput, final int numberOfInvocations) - throws IllegalArgumentException { - super(spout, rawOutput); + public StormFiniteSpoutWrapper(final IRichSpout spout, final String[] rawOutputs, + final int numberOfInvocations) throws IllegalArgumentException { + super(spout, Sets.newHashSet(rawOutputs)); + this.numberOfInvocations = numberOfInvocations; + } + + /** + * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} + * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type can be any + * type if parameter {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If + * {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on + * the spout's declared number of attributes. + * + * @param spout + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. + * @param numberOfInvocations + * The number of calls to {@link IRichSpout#nextTuple()}. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public StormFiniteSpoutWrapper(final IRichSpout spout, final Collection rawOutputs, + final int numberOfInvocations) throws IllegalArgumentException { + super(spout, rawOutputs); this.numberOfInvocations = numberOfInvocations; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java index 231cab68adafb..f33d4d3447fb1 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java @@ -17,18 +17,21 @@ package org.apache.flink.stormcompatibility.wrappers; +import java.util.HashMap; + import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; /** - * {@link StormOutputFieldsDeclarer} is used by {@link StormWrapperSetupHelper} to determine the - * number of attributes declared by the wrapped spout's or bolt's {@code declare(...)} method. + * {@link StormOutputFieldsDeclarer} is used by {@link StormWrapperSetupHelper} to determine the output streams and + * number of attributes declared by the wrapped spout's or bolt's {@code declare(...)}/{@code declareStream(...)} + * method. */ class StormOutputFieldsDeclarer implements OutputFieldsDeclarer { - /** The output schema declared by the wrapped bolt. */ - private Fields outputSchema = null; + /** The number of attributes for each declared stream by the wrapped operator. */ + HashMap outputSchemas = new HashMap(); @Override public void declare(final Fields fields) { @@ -47,28 +50,14 @@ public void declareStream(final String streamId, final Fields fields) { @Override public void declareStream(final String streamId, final boolean direct, final Fields fields) { - if (!Utils.DEFAULT_STREAM_ID.equals(streamId)) { - throw new UnsupportedOperationException("Currently, only the default output stream is supported by Flink"); + if (streamId == null) { + throw new IllegalArgumentException("Stream ID cannot be null."); } if (direct) { throw new UnsupportedOperationException("Direct emit is not supported by Flink"); } - this.outputSchema = fields; - } - - /** - * Returns the number of attributes of the output schema declare by the wrapped bolt. If no output schema is - * declared (eg, for sink bolts), {@code -1} is returned. - * - * @return the number of attributes of the output schema declare by the wrapped bolt - */ - public int getNumberOfAttributes() { - if (this.outputSchema != null) { - return this.outputSchema.size(); - } - - return -1; + this.outputSchemas.put(streamId, fields.size()); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java index 09a7ac7a18788..5a20056ae8474 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java @@ -19,9 +19,11 @@ import backtype.storm.spout.ISpoutOutputCollector; -import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; + +import java.util.HashMap; import java.util.List; /** @@ -31,23 +33,23 @@ */ class StormSpoutCollector extends AbstractStormCollector implements ISpoutOutputCollector { - // The Flink source context object + /** The Flink source context object */ private final SourceContext flinkContext; /** - * Instantiates a new {@link StormSpoutCollector} that emits Flink tuples to the given Flink - * source context. If the number of attributes is specified as zero, any output type is - * supported. If the number of attributes is between 1 to 25, the output type is {@link Tuple1} - * to {@link Tuple25}. + * Instantiates a new {@link StormSpoutCollector} that emits Flink tuples to the given Flink source context. If the + * number of attributes is specified as zero, any output type is supported. If the number of attributes is between 0 + * to 25, the output type is {@link Tuple0} to {@link Tuple25}, respectively. * * @param numberOfAttributes - * The number of attributes of the emitted tuples. + * The number of attributes of the emitted tuples. * @param flinkContext - * The Flink source context to be used. + * The Flink source context to be used. * @throws UnsupportedOperationException - * if the specified number of attributes is not in the valid range of [0,25] + * if the specified number of attributes is greater than 25 */ - public StormSpoutCollector(final int numberOfAttributes, final SourceContext flinkContext) throws UnsupportedOperationException { + public StormSpoutCollector(final HashMap numberOfAttributes, + final SourceContext flinkContext) throws UnsupportedOperationException { super(numberOfAttributes); assert (flinkContext != null); this.flinkContext = flinkContext; @@ -68,7 +70,7 @@ public void reportError(final Throwable error) { @Override public List emit(final String streamId, final List tuple, final Object messageId) { - return this.transformAndEmit(tuple); + return this.tansformAndEmit(streamId, tuple); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java index ab9a890bb390a..300b241312c55 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java @@ -17,10 +17,16 @@ package org.apache.flink.stormcompatibility.wrappers; +import java.util.Collection; + import backtype.storm.topology.IRichSpout; + +import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; +import com.google.common.collect.Sets; + /** * A {@link StormSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls the wrapped spout's * {@link IRichSpout#nextTuple() nextTuple()} method in in infinite loop. @@ -29,39 +35,61 @@ public class StormSpoutWrapper extends AbstractStormSpoutWrapper { private static final long serialVersionUID = -218340336648247605L; /** - * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it - * can - * be used within a Flink streaming program. The output type will be one of {@link Tuple1} to {@link Tuple25} + * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it can + * be used within a Flink streaming program. The output type will be one of {@link Tuple0} to {@link Tuple25} * depending on the spout's declared number of attributes. - * + * * @param spout - * The Storm {@link IRichSpout spout} to be used. + * The Storm {@link IRichSpout spout} to be used. * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [1;25]. + * If the number of declared output attributes is not with range [0;25]. */ public StormSpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { - super(spout, false); + super(spout, null); + } + + /** + * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it can + * be used within a Flink streaming program. The output type can be any type if parameter {@code rawOutput} is + * {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the + * output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of + * attributes. + * + * @param spout + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. (Can be {@code null}.) + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public StormSpoutWrapper(final IRichSpout spout, final String[] rawOutputs) + throws IllegalArgumentException { + super(spout, Sets.newHashSet(rawOutputs)); } /** - * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it - * can be used within a Flink streaming program. The output type can be any type if parameter {@code rawOutput} is + * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it can + * be used within a Flink streaming program. The output type can be any type if parameter {@code rawOutput} is * {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the - * output type will be one of {@link Tuple1} to {@link Tuple25} depending on the spout's declared number of + * output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of * attributes. - * + * * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. + * The Storm {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. (Can be {@code null}.) * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public StormSpoutWrapper(final IRichSpout spout, final boolean rawOutput) throws IllegalArgumentException { - super(spout, rawOutput); + public StormSpoutWrapper(final IRichSpout spout, final Collection rawOutputs) + throws IllegalArgumentException { + super(spout, rawOutputs); } /** diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java index e2e303a0ba222..75ab8e0d8c07f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java @@ -25,11 +25,14 @@ import backtype.storm.topology.IComponent; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.IRichSpout; + import org.apache.flink.stormcompatibility.api.FlinkTopologyContext; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; +import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; /** * {@link StormWrapperSetupHelper} is an helper class used by {@link AbstractStormSpoutWrapper} or @@ -38,52 +41,47 @@ class StormWrapperSetupHelper { /** - * Computes the number of output attributes used by a {@link AbstractStormSpoutWrapper} or - * {@link StormBoltWrapper}. Returns zero for raw output type or a value within range [1;25] for - * output type {@link org.apache.flink.api.java.tuple.Tuple1 Tuple1} to - * {@link org.apache.flink.api.java.tuple.Tuple25 Tuple25} . In case of a data sink, {@code -1} - * is returned. + * Computes the number of output attributes used by a {@link AbstractStormSpoutWrapper} or {@link StormBoltWrapper} + * per declared output stream. The number is {@code -1} for raw output type or a value within range [0;25] for + * output type {@link org.apache.flink.api.java.tuple.Tuple0 Tuple0} to + * {@link org.apache.flink.api.java.tuple.Tuple25 Tuple25}. * * @param spoutOrBolt - * The Storm {@link IRichSpout spout} or {@link IRichBolt bolt} to be used. - * @param rawOutput - * Set to {@code true} if a single attribute output stream, should not be of type - * {@link org.apache.flink.api.java.tuple.Tuple1 Tuple1} but be of a raw type. - * @return The number of attributes to be used. + * The Storm {@link IRichSpout spout} or {@link IRichBolt bolt} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type + * {@link org.apache.flink.api.java.tuple.Tuple1 Tuple1} but be of a raw type. (Can be {@code null}.) + * @return The number of attributes to be used for each stream. * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output - * attributes is not 1 or if {@code rawOuput} is {@code false} and the number - * of declared output attributes is not with range [1;25]. + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. */ - public static int getNumberOfAttributes(final IComponent spoutOrBolt, final boolean rawOutput) - throws IllegalArgumentException { + public static HashMap getNumberOfAttributes(final IComponent spoutOrBolt, + final Collection rawOutputs) + throws IllegalArgumentException { final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); spoutOrBolt.declareOutputFields(declarer); - final int declaredNumberOfAttributes = declarer.getNumberOfAttributes(); - - if (declaredNumberOfAttributes == -1) { - return -1; - } - - if ((declaredNumberOfAttributes < 1) || (declaredNumberOfAttributes > 25)) { - throw new IllegalArgumentException( - "Provided bolt declares non supported number of output attributes. Must be in range [1;25] but " + - "was " - + declaredNumberOfAttributes); - } - - if (rawOutput) { - if (declaredNumberOfAttributes > 1) { + for (Entry schema : declarer.outputSchemas.entrySet()) { + int declaredNumberOfAttributes = schema.getValue(); + if ((declaredNumberOfAttributes < 0) || (declaredNumberOfAttributes > 25)) { throw new IllegalArgumentException( - "Ouput type is requested to be raw type, but provided bolt declares more then one output " + - "attribute."); + "Provided bolt declares non supported number of output attributes. Must be in range [0;25] but " + + "was " + declaredNumberOfAttributes); + } + if (rawOutputs != null && rawOutputs.contains(schema.getKey())) { + if (declaredNumberOfAttributes != 1) { + throw new IllegalArgumentException( + "Ouput type is requested to be raw type, but provided bolt declares more then one output " + + "attribute."); + } + schema.setValue(-1); } - return 0; } - return declaredNumberOfAttributes; + return declarer.outputSchemas; } // TODO diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java index 58d81f907e311..08ac60bf69315 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.stormcompatibility.api; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; + import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.stormcompatibility.util.AbstractTest; import org.junit.Assert; import org.junit.Test; @@ -30,85 +31,112 @@ public class FlinkOutputFieldsDeclarerTest extends AbstractTest { + @Test + public void testNull() { + Assert.assertNull(new FlinkOutputFieldsDeclarer().getOutputType(null)); + } + @Test public void testDeclare() { - for (int i = 0; i < 4; ++i) { - for (int j = 0; j <= 25; ++j) { - this.runDeclareTest(i, j); + for (int i = 0; i < 2; ++i) { // test case: simple / non-direct + for (int j = 1; j < 2; ++j) { // number of streams + for (int k = 0; k <= 25; ++k) { // number of attributes + this.runDeclareTest(i, j, k); + } } } } @Test(expected = IllegalArgumentException.class) public void testDeclareSimpleToManyAttributes() { - this.runDeclareTest(0, 26); + this.runDeclareTest(0, this.r.nextBoolean() ? 1 : 2, 26); } @Test(expected = IllegalArgumentException.class) public void testDeclareNonDirectToManyAttributes() { - this.runDeclareTest(1, 26); + this.runDeclareTest(1, this.r.nextBoolean() ? 1 : 2, 26); } @Test(expected = IllegalArgumentException.class) public void testDeclareDefaultStreamToManyAttributes() { - this.runDeclareTest(2, 26); + this.runDeclareTest(2, this.r.nextBoolean() ? 1 : 2, 26); } @Test(expected = IllegalArgumentException.class) public void testDeclareFullToManyAttributes() { - this.runDeclareTest(3, 26); + this.runDeclareTest(3, this.r.nextBoolean() ? 1 : 2, 26); } - private void runDeclareTest(final int testCase, final int numberOfAttributes) { + private void runDeclareTest(final int testCase, final int numberOfStreams, + final int numberOfAttributes) { final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); + String[] streams = null; + if (numberOfStreams > 1 || r.nextBoolean()) { + streams = new String[numberOfStreams]; + for (int i = 0; i < numberOfStreams; ++i) { + streams[i] = "stream" + i; + } + } + final String[] attributes = new String[numberOfAttributes]; - for (int i = 0; i < numberOfAttributes; ++i) { + for (int i = 0; i < attributes.length; ++i) { attributes[i] = "a" + i; } switch (testCase) { - case 0: - this.declareSimple(declarer, attributes); - break; - case 1: - this.declareNonDirect(declarer, attributes); - break; - case 2: - this.declareDefaultStream(declarer, attributes); - break; - default: - this.declareFull(declarer, attributes); + case 0: + this.declareSimple(declarer, streams, attributes); + break; + default: + this.declareNonDirect(declarer, streams, attributes); + } + + if (streams == null) { + streams = new String[] { Utils.DEFAULT_STREAM_ID }; } - final TypeInformation type = declarer.getOutputType(); + for (String stream : streams) { + final TypeInformation type = declarer.getOutputType(stream); - if (numberOfAttributes == 0) { - Assert.assertNull(type); - } else { - Assert.assertEquals(numberOfAttributes, type.getArity()); if (numberOfAttributes == 1) { - Assert.assertFalse(type.isTupleType()); + Assert.assertEquals(type.getClass(), GenericTypeInfo.class); + Assert.assertEquals(type.getTypeClass(), Object.class); } else { + Assert.assertEquals(numberOfAttributes, type.getArity()); Assert.assertTrue(type.isTupleType()); } } } - private void declareSimple(final FlinkOutputFieldsDeclarer declarer, final String[] attributes) { - declarer.declare(new Fields(attributes)); - } + private void declareSimple(final FlinkOutputFieldsDeclarer declarer, final String[] streams, + final String[] attributes) { - private void declareNonDirect(final FlinkOutputFieldsDeclarer declarer, final String[] attributes) { - declarer.declare(false, new Fields(attributes)); + if (streams != null) { + for (String stream : streams) { + declarer.declareStream(stream, new Fields(attributes)); + } + } else { + declarer.declare(new Fields(attributes)); + } } - private void declareDefaultStream(final FlinkOutputFieldsDeclarer declarer, final String[] attributes) { - declarer.declareStream(Utils.DEFAULT_STREAM_ID, new Fields(attributes)); + private void declareNonDirect(final FlinkOutputFieldsDeclarer declarer, final String[] streams, + final String[] attributes) { + + if (streams != null) { + for (String stream : streams) { + declarer.declareStream(stream, false, new Fields(attributes)); + } + } else { + declarer.declare(false, new Fields(attributes)); + } } - private void declareFull(final FlinkOutputFieldsDeclarer declarer, final String[] attributes) { - declarer.declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields(attributes)); + @Test(expected = IllegalArgumentException.class) + public void testUndeclared() { + final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); + declarer.getOutputType("unknownStreamId"); } @Test(expected = UnsupportedOperationException.class) @@ -116,21 +144,11 @@ public void testDeclareDirect() { new FlinkOutputFieldsDeclarer().declare(true, null); } - @Test(expected = UnsupportedOperationException.class) - public void testDeclareNonDefaultStrem() { - new FlinkOutputFieldsDeclarer().declareStream("dummy", null); - } - @Test(expected = UnsupportedOperationException.class) public void testDeclareDirect2() { new FlinkOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, null); } - @Test(expected = UnsupportedOperationException.class) - public void testDeclareNonDefaultStrem2() { - new FlinkOutputFieldsDeclarer().declareStream("dummy", this.r.nextBoolean(), null); - } - @Test public void testGetGroupingFieldIndexes() { final int numberOfAttributes = 5 + this.r.nextInt(21); @@ -163,7 +181,8 @@ public void testGetGroupingFieldIndexes() { } } - final int[] result = declarer.getGroupingFieldIndexes(groupingFields); + final int[] result = declarer.getGroupingFieldIndexes(Utils.DEFAULT_STREAM_ID, + groupingFields); Assert.assertEquals(expectedResult.length, result.length); for (int i = 0; i < expectedResult.length; ++i) { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java new file mode 100644 index 0000000000000..0187020f496fa --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java @@ -0,0 +1,48 @@ +/* + * 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.stormcompatibility.api; + +import org.junit.Test; + +public class FlinkTopologyBuilderTest { + + @Test(expected = RuntimeException.class) + public void testUnknowSpout() { + FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); + builder.setSpout("spout", new TestSpout()); + builder.setBolt("bolt", new TestBolt()).shuffleGrouping("unknown"); + builder.createTopology(); + } + + @Test(expected = RuntimeException.class) + public void testUnknowBolt() { + FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); + builder.setSpout("spout", new TestSpout()); + builder.setBolt("bolt1", new TestBolt()).shuffleGrouping("spout"); + builder.setBolt("bolt2", new TestBolt()).shuffleGrouping("unknown"); + builder.createTopology(); + } + + @Test(expected = RuntimeException.class) + public void testUndeclaredStream() { + FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); + builder.setSpout("spout", new TestSpout()); + builder.setBolt("bolt", new TestBolt()).shuffleGrouping("spout"); + builder.createTopology(); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java new file mode 100644 index 0000000000000..2e4a53417704c --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java @@ -0,0 +1,48 @@ +/* + * 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.stormcompatibility.api; + +import java.util.Map; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; + +public class TestBolt implements IRichBolt { + private static final long serialVersionUID = -667148827441397683L; + + @SuppressWarnings("rawtypes") + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {} + + @Override + public void execute(Tuple input) {} + + @Override + public void cleanup() {} + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) {} + + @Override + public Map getComponentConfiguration() { + return null; + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java new file mode 100644 index 0000000000000..146218fcc38ab --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java @@ -0,0 +1,59 @@ +/* + * 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.stormcompatibility.api; + +import java.util.Map; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; + +public class TestSpout implements IRichSpout { + private static final long serialVersionUID = -4884029383198924007L; + + @SuppressWarnings("rawtypes") + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {} + + @Override + public void close() {} + + @Override + public void activate() {} + + @Override + public void deactivate() {} + + @Override + public void nextTuple() {} + + @Override + public void ack(Object msgId) {} + + @Override + public void fail(Object msgId) {} + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) {} + + @Override + public Map getComponentConfiguration() { + return null; + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java new file mode 100644 index 0000000000000..c0a6ed341cd33 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java @@ -0,0 +1,51 @@ +/* + * 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.stormcompatibility.wrappers; + +import java.util.Iterator; + +import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; +import org.apache.flink.stormcompatibility.util.SplitStreamType; +import org.junit.Assert; +import org.junit.Test; + +public class FlinkStormStreamSelectorTest { + + @Test + public void testSelector() { + FlinkStormStreamSelector selector = new FlinkStormStreamSelector(); + SplitStreamType tuple = new SplitStreamType(); + Iterator result; + + tuple.streamId = "stream1"; + result = selector.select(tuple).iterator(); + Assert.assertEquals("stream1", result.next()); + Assert.assertFalse(result.hasNext()); + + tuple.streamId = "stream2"; + result = selector.select(tuple).iterator(); + Assert.assertEquals("stream2", result.next()); + Assert.assertFalse(result.hasNext()); + + tuple.streamId = "stream1"; + result = selector.select(tuple).iterator(); + Assert.assertEquals("stream1", result.next()); + Assert.assertFalse(result.hasNext()); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java index 39975053475a4..d01c3e0d21ecc 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java @@ -26,6 +26,7 @@ import org.junit.Test; import java.util.Collection; +import java.util.HashMap; import java.util.List; import static org.mockito.Mockito.mock; @@ -36,19 +37,23 @@ public class StormBoltCollectorTest extends AbstractTest { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testBoltStormCollector() throws InstantiationException, IllegalAccessException { - for (int numberOfAttributes = 0; numberOfAttributes < 26; ++numberOfAttributes) { + for (int numberOfAttributes = -1; numberOfAttributes < 26; ++numberOfAttributes) { final Output flinkCollector = mock(Output.class); Tuple flinkTuple = null; final Values tuple = new Values(); StormBoltCollector collector; - if (numberOfAttributes == 0) { - collector = new StormBoltCollector(numberOfAttributes, flinkCollector); + final String streamId = "streamId"; + HashMap attributes = new HashMap(); + attributes.put(streamId, numberOfAttributes); + + if (numberOfAttributes == -1) { + collector = new StormBoltCollector(attributes, flinkCollector); tuple.add(new Integer(this.r.nextInt())); } else { - collector = new StormBoltCollector(numberOfAttributes, flinkCollector); + collector = new StormBoltCollector(attributes, flinkCollector); flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance(); for (int i = 0; i < numberOfAttributes; ++i) { @@ -57,14 +62,13 @@ public void testBoltStormCollector() throws InstantiationException, IllegalAcces } } - final String streamId = "streamId"; final Collection anchors = mock(Collection.class); final List taskIds; taskIds = collector.emit(streamId, anchors, tuple); Assert.assertNull(taskIds); - if (numberOfAttributes == 0) { + if (numberOfAttributes == -1) { verify(flinkCollector).collect(tuple.get(0)); } else { verify(flinkCollector).collect(flinkTuple); @@ -76,26 +80,26 @@ public void testBoltStormCollector() throws InstantiationException, IllegalAcces @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testReportError() { - new StormBoltCollector(1, mock(Output.class)).reportError(null); + new StormBoltCollector(mock(HashMap.class), mock(Output.class)).reportError(null); } - @SuppressWarnings({"rawtypes", "unchecked"}) + @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testEmitDirect() { - new StormBoltCollector(1, mock(Output.class)).emitDirect(0, null, + new StormBoltCollector(mock(HashMap.class), mock(Output.class)).emitDirect(0, null, null, null); } @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testAck() { - new StormBoltCollector(1, mock(Output.class)).ack(null); + new StormBoltCollector(mock(HashMap.class), mock(Output.class)).ack(null); } @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testFail() { - new StormBoltCollector(1, mock(Output.class)).fail(null); + new StormBoltCollector(mock(HashMap.class), mock(Output.class)).fail(null); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java index 3e55d232bee0e..2491486aaf579 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java @@ -20,10 +20,16 @@ import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; +import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.stormcompatibility.util.SplitStreamType; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; @@ -34,18 +40,17 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.HashSet; import java.util.Map; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.isNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; @RunWith(PowerMockRunner.class) @PrepareForTest({StreamRecordSerializer.class, StormWrapperSetupHelper.class}) -public class StormBoltWrapperTest { +public class StormBoltWrapperTest extends AbstractTest { @Test(expected = IllegalArgumentException.class) public void testWrapperRawType() throws Exception { @@ -53,7 +58,8 @@ public void testWrapperRawType() throws Exception { declarer.declare(new Fields("dummy1", "dummy2")); PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - new StormBoltWrapper(mock(IRichBolt.class), true); + new StormBoltWrapper(mock(IRichBolt.class), + new String[] { Utils.DEFAULT_STREAM_ID }); } @Test(expected = IllegalArgumentException.class) @@ -79,38 +85,40 @@ public void testWrapperToManyAttributes2() throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - new StormBoltWrapper(mock(IRichBolt.class), false); + new StormBoltWrapper(mock(IRichBolt.class), new String[] {}); } @Test public void testWrapper() throws Exception { - for (int i = 0; i < 26; ++i) { + for (int i = -1; i < 26; ++i) { this.testWrapper(i); } } @SuppressWarnings({"rawtypes", "unchecked"}) private void testWrapper(final int numberOfAttributes) throws Exception { - assert ((0 <= numberOfAttributes) && (numberOfAttributes <= 25)); + assert ((-1 <= numberOfAttributes) && (numberOfAttributes <= 25)); Tuple flinkTuple = null; String rawTuple = null; - if (numberOfAttributes == 0) { + if (numberOfAttributes == -1) { rawTuple = "test"; } else { flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance(); } - String[] schema = new String[numberOfAttributes]; - if (numberOfAttributes == 0) { + String[] schema; + if (numberOfAttributes == -1) { schema = new String[1]; + } else { + schema = new String[numberOfAttributes]; } for (int i = 0; i < schema.length; ++i) { schema[i] = "a" + i; } final StreamRecord record = mock(StreamRecord.class); - if (numberOfAttributes == 0) { + if (numberOfAttributes == -1) { when(record.getValue()).thenReturn(rawTuple); } else { when(record.getValue()).thenReturn(flinkTuple); @@ -124,17 +132,63 @@ private void testWrapper(final int numberOfAttributes) throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, null); + final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null); wrapper.setup(mock(Output.class), taskContext); wrapper.open(new Configuration()); wrapper.processElement(record); - if (numberOfAttributes == 0) { + if (numberOfAttributes == -1) { verify(bolt).execute(eq(new StormTuple(rawTuple, null))); } else { verify(bolt).execute(eq(new StormTuple(flinkTuple, null))); } + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + @Test + public void testMultipleOutputStreams() throws Exception { + final boolean rawOutType1 = super.r.nextBoolean(); + final boolean rawOutType2 = super.r.nextBoolean(); + + final StreamRecord record = mock(StreamRecord.class); + when(record.getValue()).thenReturn(2).thenReturn(3); + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + Output output = mock(Output.class); + + TestBolt bolt = new TestBolt(); + HashSet raw = new HashSet(); + if (rawOutType1) { + raw.add("stream1"); + } + if (rawOutType2) { + raw.add("stream2"); + } + + final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null, raw); + wrapper.setup(output, taskContext); + wrapper.open(new Configuration()); + + SplitStreamType splitRecord = new SplitStreamType(); + if (rawOutType1) { + splitRecord.streamId = "stream1"; + splitRecord.value = 2; + } else { + splitRecord.streamId = "stream1"; + splitRecord.value = new Tuple1(2); + } + wrapper.processElement(record); + verify(output).collect(new StreamRecord(splitRecord, 0)); + if (rawOutType2) { + splitRecord.streamId = "stream2"; + splitRecord.value = 3; + } else { + splitRecord.streamId = "stream2"; + splitRecord.value = new Tuple1(3); + } + wrapper.processElement(record); + verify(output, times(2)).collect(new StreamRecord(splitRecord, 0)); } @SuppressWarnings("unchecked") @@ -185,4 +239,40 @@ public void testClose() throws Exception { verify(bolt).cleanup(); } + private static final class TestBolt implements IRichBolt { + private static final long serialVersionUID = 7278692872260138758L; + private OutputCollector collector; + + @SuppressWarnings("rawtypes") + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.collector = collector; + } + + int counter = 0; + @Override + public void execute(backtype.storm.tuple.Tuple input) { + if (++counter % 2 == 1) { + this.collector.emit("stream1", new Values(input.getInteger(0))); + } else { + this.collector.emit("stream2", new Values(input.getInteger(0))); + } + } + + @Override + public void cleanup() {} + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declareStream("stream1", new Fields("a1")); + declarer.declareStream("stream2", new Fields("a2")); + } + + @Override + public Map getComponentConfiguration() { + return null; + } + } + + } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java index cfde7700db38a..a28b6e51a00b2 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java @@ -19,6 +19,7 @@ import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; + import org.apache.flink.stormcompatibility.util.AbstractTest; import org.junit.Assert; import org.junit.Test; @@ -31,50 +32,60 @@ public class StormOutputFieldsDeclarerTest extends AbstractTest { public void testDeclare() { final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); - Assert.assertEquals(-1, declarer.getNumberOfAttributes()); + int numberOfAttributes = this.r.nextInt(26); + declarer.declare(createSchema(numberOfAttributes)); + Assert.assertEquals(1, declarer.outputSchemas.size()); + Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(Utils.DEFAULT_STREAM_ID) + .intValue()); + + final String sid = "streamId"; + numberOfAttributes = 0 + this.r.nextInt(26); + declarer.declareStream(sid, createSchema(numberOfAttributes)); + Assert.assertEquals(2, declarer.outputSchemas.size()); + Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(sid).intValue()); + } - final int numberOfAttributes = 1 + this.r.nextInt(25); + private Fields createSchema(final int numberOfAttributes) { final ArrayList schema = new ArrayList(numberOfAttributes); for (int i = 0; i < numberOfAttributes; ++i) { schema.add("a" + i); } - declarer.declare(new Fields(schema)); - Assert.assertEquals(numberOfAttributes, declarer.getNumberOfAttributes()); + return new Fields(schema); } @Test public void testDeclareDirect() { - new StormOutputFieldsDeclarer().declare(false, null); + new StormOutputFieldsDeclarer().declare(false, new Fields()); } @Test(expected = UnsupportedOperationException.class) public void testDeclareDirectFail() { - new StormOutputFieldsDeclarer().declare(true, null); + new StormOutputFieldsDeclarer().declare(true, new Fields()); } @Test public void testDeclareStream() { - new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, null); + new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields()); } - @Test(expected = UnsupportedOperationException.class) + @Test(expected = IllegalArgumentException.class) public void testDeclareStreamFail() { - new StormOutputFieldsDeclarer().declareStream(null, null); + new StormOutputFieldsDeclarer().declareStream(null, new Fields()); } @Test public void testDeclareFullStream() { - new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, null); + new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields()); } - @Test(expected = UnsupportedOperationException.class) + @Test(expected = IllegalArgumentException.class) public void testDeclareFullStreamFailNonDefaultStream() { - new StormOutputFieldsDeclarer().declareStream(null, false, null); + new StormOutputFieldsDeclarer().declareStream(null, false, new Fields()); } @Test(expected = UnsupportedOperationException.class) public void testDeclareFullStreamFailDirect() { - new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, null); + new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields()); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java index e4826bbfe8591..36ed58aeb6365 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java @@ -25,6 +25,7 @@ import org.junit.Assert; import org.junit.Test; +import java.util.HashMap; import java.util.List; import static org.mockito.Mockito.mock; @@ -35,19 +36,23 @@ public class StormSpoutCollectorTest extends AbstractTest { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testSpoutStormCollector() throws InstantiationException, IllegalAccessException { - for (int numberOfAttributes = 0; numberOfAttributes < 26; ++numberOfAttributes) { + for (int numberOfAttributes = -1; numberOfAttributes < 26; ++numberOfAttributes) { final SourceContext flinkCollector = mock(SourceContext.class); Tuple flinkTuple = null; final Values tuple = new Values(); StormSpoutCollector collector; - if (numberOfAttributes == 0) { - collector = new StormSpoutCollector(numberOfAttributes, flinkCollector); + final String streamId = "streamId"; + HashMap attributes = new HashMap(); + attributes.put(streamId, numberOfAttributes); + + if (numberOfAttributes == -1) { + collector = new StormSpoutCollector(attributes, flinkCollector); tuple.add(new Integer(this.r.nextInt())); } else { - collector = new StormSpoutCollector(numberOfAttributes, flinkCollector); + collector = new StormSpoutCollector(attributes, flinkCollector); flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance(); for (int i = 0; i < numberOfAttributes; ++i) { @@ -56,7 +61,6 @@ public void testSpoutStormCollector() throws InstantiationException, IllegalAcce } } - final String streamId = "streamId"; final List taskIds; final Object messageId = new Integer(this.r.nextInt()); @@ -64,7 +68,7 @@ public void testSpoutStormCollector() throws InstantiationException, IllegalAcce Assert.assertNull(taskIds); - if (numberOfAttributes == 0) { + if (numberOfAttributes == -1) { verify(flinkCollector).collect(tuple.get(0)); } else { verify(flinkCollector).collect(flinkTuple); @@ -75,13 +79,15 @@ public void testSpoutStormCollector() throws InstantiationException, IllegalAcce @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testReportError() { - new StormSpoutCollector(1, mock(SourceContext.class)).reportError(null); + new StormSpoutCollector(mock(HashMap.class), mock(SourceContext.class)) + .reportError(null); } @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testEmitDirect() { - new StormSpoutCollector(1, mock(SourceContext.class)).emitDirect(0, null, null, + new StormSpoutCollector(mock(HashMap.class), mock(SourceContext.class)).emitDirect( + 0, null, null, (Object) null); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java index 96e7b35663ef2..06d539978875c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java @@ -60,7 +60,7 @@ public void nonTupleTest() { @Test public void tupleTest() throws InstantiationException, IllegalAccessException { - final int numberOfAttributes = 1 + this.r.nextInt(25); + final int numberOfAttributes = this.r.nextInt(26); final Object[] data = new Object[numberOfAttributes]; final Tuple flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance(); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java index 15129cea513f3..7497ffc2650bf 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java @@ -17,10 +17,14 @@ package org.apache.flink.stormcompatibility.wrappers; +import java.util.HashMap; + import backtype.storm.topology.IComponent; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.IRichSpout; import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; + import org.apache.flink.stormcompatibility.util.AbstractTest; import org.junit.Assert; import org.junit.Test; @@ -29,29 +33,14 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import com.google.common.collect.Sets; + import static org.mockito.Mockito.mock; @RunWith(PowerMockRunner.class) @PrepareForTest(StormWrapperSetupHelper.class) public class StormWrapperSetupHelperTest extends AbstractTest { - @Test(expected = IllegalArgumentException.class) - public void testZeroAttributesDeclarerBolt() throws Exception { - IComponent boltOrSpout; - - if (this.r.nextBoolean()) { - boltOrSpout = mock(IRichSpout.class); - } else { - boltOrSpout = mock(IRichBolt.class); - } - - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); - declarer.declare(new Fields()); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, this.r.nextBoolean()); - } - @Test public void testEmptyDeclarerBolt() { IComponent boltOrSpout; @@ -62,7 +51,8 @@ public void testEmptyDeclarerBolt() { boltOrSpout = mock(IRichBolt.class); } - Assert.assertEquals(-1, StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, this.r.nextBoolean())); + Assert.assertEquals(new HashMap(), + StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null)); } @Test(expected = IllegalArgumentException.class) @@ -79,7 +69,8 @@ public void testRawType() throws Exception { declarer.declare(new Fields("dummy1", "dummy2")); PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, true); + StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, + Sets.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID })); } @Test(expected = IllegalArgumentException.class) @@ -100,20 +91,22 @@ public void testToManyAttributes() throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, false); + StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null); } @Test public void testTupleTypes() throws Exception { - for (int i = 0; i < 26; ++i) { + for (int i = -1; i < 26; ++i) { this.testTupleTypes(i); } } private void testTupleTypes(final int numberOfAttributes) throws Exception { - String[] schema = new String[numberOfAttributes]; - if (numberOfAttributes == 0) { + String[] schema; + if (numberOfAttributes == -1) { schema = new String[1]; + } else { + schema = new String[numberOfAttributes]; } for (int i = 0; i < schema.length; ++i) { schema[i] = "a" + i; @@ -130,7 +123,13 @@ private void testTupleTypes(final int numberOfAttributes) throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, numberOfAttributes == 0); + HashMap attributes = new HashMap(); + attributes.put(Utils.DEFAULT_STREAM_ID, numberOfAttributes); + + Assert.assertEquals(attributes, StormWrapperSetupHelper.getNumberOfAttributes( + boltOrSpout, + numberOfAttributes == -1 ? Sets + .newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }) : null)); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml index 4621650c2e94e..430972b9a558a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml @@ -135,6 +135,7 @@ under the License. + WordCount-SpoutSource package @@ -176,6 +177,7 @@ under the License. + WordCount-BoltTokenizer package @@ -222,6 +224,7 @@ under the License. + maven-assembly-plugin diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java index 7bcb7f9816b98..ee5d9f997cdcb 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java @@ -26,6 +26,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import backtype.storm.utils.Utils; + /** * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. @@ -65,8 +67,9 @@ public static void main(final String[] args) throws Exception { final DataStream exclaimed = text .transform("StormBoltTokenizer", TypeExtractor.getForObject(""), - new StormBoltWrapper(new ExclamationBolt(), true)) - .map(new ExclamationMap()); + new StormBoltWrapper(new ExclamationBolt(), + new String[] { Utils.DEFAULT_STREAM_ID })) + .map(new ExclamationMap()); // emit result if (fileOutput) { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java index f027eae165ce2..962a3189da6b0 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java @@ -27,6 +27,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import backtype.storm.utils.Utils; + /** * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. @@ -84,6 +86,7 @@ public static void main(final String[] args) throws Exception { // ************************************************************************* private static class ExclamationMap implements MapFunction { + private static final long serialVersionUID = -684993133807698042L; @Override public String map(String value) throws Exception { @@ -126,13 +129,14 @@ private static DataStream getTextDataStream(final StreamExecutionEnviron final String[] tokens = textPath.split(":"); final String localFile = tokens[tokens.length - 1]; return env.addSource( - new FiniteStormSpoutWrapper(new FiniteStormFileSpout(localFile), true), - TypeExtractor.getForClass(String.class)).setParallelism(1); + new FiniteStormSpoutWrapper(new FiniteStormFileSpout(localFile), + new String[] { Utils.DEFAULT_STREAM_ID }), + TypeExtractor.getForClass(String.class)).setParallelism(1); } return env.addSource( - new FiniteStormSpoutWrapper( - new FiniteStormInMemorySpout(WordCountData.WORDS), true), + new FiniteStormSpoutWrapper(new FiniteStormInMemorySpout( + WordCountData.WORDS), new String[] { Utils.DEFAULT_STREAM_ID }), TypeExtractor.getForClass(String.class)).setParallelism(1); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java new file mode 100644 index 0000000000000..4116f3cca134e --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java @@ -0,0 +1,102 @@ +/* + * 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.stormcompatibility.split; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.stormcompatibility.split.stormoperators.RandomSpout; +import org.apache.flink.stormcompatibility.split.stormoperators.VerifyAndEnrichBolt; +import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; +import org.apache.flink.stormcompatibility.util.SplitStreamMapper; +import org.apache.flink.stormcompatibility.util.SplitStreamType; +import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; +import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SplitDataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +/** + * Implements a simple example with two declared output streams for the embedded Spout. + *

    + * This example shows how to: + *

      + *
    • handle multiple output stream of a spout
    • + *
    • accessing each stream by .split(...) and .select(...)
    • + *
    • strip wrapper data type SplitStreamType for furhter processing in Flink
    • + *
    + *

    + * This example would work the same way for multiple bolt output streams. + */ +public class SpoutSplitExample { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(final String[] args) throws Exception { + + // set up the execution environment + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + String[] rawOutputs = new String[] { RandomSpout.EVEN_STREAM, RandomSpout.ODD_STREAM }; + + final DataStream> numbers = env.addSource( + new StormSpoutWrapper>(new RandomSpout(true, 0), + rawOutputs), TypeExtractor.getForObject(new SplitStreamType())); + + SplitDataStream> splitStream = numbers + .split(new FlinkStormStreamSelector()); + + DataStream> evenStream = splitStream.select(RandomSpout.EVEN_STREAM); + DataStream> oddStream = splitStream.select(RandomSpout.ODD_STREAM); + + evenStream.map(new SplitStreamMapper()).returns(Integer.class).map(new Enrich("even")).print(); + oddStream.transform("oddBolt", + TypeExtractor.getForObject(new Tuple2("", 0)), + new StormBoltWrapper, Tuple2>( + new VerifyAndEnrichBolt(false))) + .print(); + + // execute program + env.execute("Spout split stream example"); + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** + * Same as {@link VerifyAndEnrichBolt}. + */ + private final static class Enrich implements MapFunction> { + private static final long serialVersionUID = 5213888269197438892L; + private final Tuple2 out; + + public Enrich(String token) { + this.out = new Tuple2(token, 0); + } + + @Override + public Tuple2 map(Integer value) throws Exception { + this.out.setField(value, 1); + return this.out; + } + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java new file mode 100644 index 0000000000000..75d710e37561f --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java @@ -0,0 +1,76 @@ +/* + * 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.stormcompatibility.split.stormoperators; + +import java.util.Map; +import java.util.Random; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +public class RandomSpout extends BaseRichSpout { + private static final long serialVersionUID = -3978554318742509334L; + + public static final String EVEN_STREAM = "even"; + public static final String ODD_STREAM = "odd"; + + private final boolean split; + private Random r = new Random(); + private SpoutOutputCollector collector; + + public RandomSpout(boolean split, long seed) { + this.split = split; + this.r = new Random(seed); + } + + @SuppressWarnings("rawtypes") + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.collector = collector; + } + + @Override + public void nextTuple() { + int i = r.nextInt(); + if (split) { + if (i % 2 == 0) { + this.collector.emit(EVEN_STREAM, new Values(i)); + } else { + this.collector.emit(ODD_STREAM, new Values(i)); + } + } else { + this.collector.emit(new Values(i)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + Fields schema = new Fields("number"); + if (split) { + declarer.declareStream(EVEN_STREAM, schema); + declarer.declareStream(ODD_STREAM, schema); + } else { + declarer.declare(schema); + } + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java new file mode 100644 index 0000000000000..585370522810f --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java @@ -0,0 +1,61 @@ +/* + * 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.stormcompatibility.split.stormoperators; + +import java.util.Map; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +public class VerifyAndEnrichBolt extends BaseRichBolt { + private static final long serialVersionUID = -7277395570966328721L; + + private final boolean evenOrOdd; // true: even -- false: odd + private final String token; + private OutputCollector collector; + + public VerifyAndEnrichBolt(boolean evenOrOdd) { + this.evenOrOdd = evenOrOdd; + this.token = evenOrOdd ? "even" : "odd"; + } + + @SuppressWarnings("rawtypes") + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.collector = collector; + } + + @Override + public void execute(Tuple input) { + if ((input.getInteger(0) % 2 == 0) != this.evenOrOdd) { + throw new RuntimeException("Invalid number detected."); + } + this.collector.emit(new Values(this.token, input.getInteger(0))); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("evenOrOdd", "number")); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java index 4c012d80e683a..0f04fea451f7b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java @@ -18,6 +18,8 @@ package org.apache.flink.stormcompatibility.wordcount; import backtype.storm.topology.IRichSpout; +import backtype.storm.utils.Utils; + import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -140,11 +142,14 @@ private static DataStream getTextDataStream(final StreamExecutionEnviron final String[] tokens = textPath.split(":"); final String localFile = tokens[tokens.length - 1]; return env.addSource( - new StormFiniteSpoutWrapper(new StormFileSpout(localFile), true), + new StormFiniteSpoutWrapper(new StormFileSpout(localFile), + new String[] { Utils.DEFAULT_STREAM_ID }), TypeExtractor.getForClass(String.class)).setParallelism(1); } - return env.addSource(new StormFiniteSpoutWrapper(new StormInMemorySpout(WordCountData.WORDS), true), + return env.addSource( + new StormFiniteSpoutWrapper(new StormInMemorySpout(WordCountData.WORDS), + new String[] { Utils.DEFAULT_STREAM_ID }), TypeExtractor.getForClass(String.class)).setParallelism(1); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java new file mode 100644 index 0000000000000..305245b14999a --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java @@ -0,0 +1,28 @@ +/* + * 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.stormcompatibility.split; + +import org.junit.Test; + +public class BoltSplitITCase { + + @Test + public void testTopology() throws Exception { + StormSplitStreamBoltLocal.main(new String[] { "0", "/dev/null" }); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java new file mode 100644 index 0000000000000..c40e0548cd6f9 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java @@ -0,0 +1,61 @@ +/* + * 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.stormcompatibility.split; + +import java.util.Map; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +public class SplitBolt extends BaseRichBolt { + private static final long serialVersionUID = -6627606934204267173L; + + public static final String EVEN_STREAM = "even"; + public static final String ODD_STREAM = "odd"; + + private OutputCollector collector; + + @SuppressWarnings("rawtypes") + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.collector = collector; + + } + + @Override + public void execute(Tuple input) { + if (input.getInteger(0) % 2 == 0) { + this.collector.emit(EVEN_STREAM, new Values(input.getInteger(0))); + } else { + this.collector.emit(ODD_STREAM, new Values(input.getInteger(0))); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + Fields schema = new Fields("number"); + declarer.declareStream(EVEN_STREAM, schema); + declarer.declareStream(ODD_STREAM, schema); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java new file mode 100644 index 0000000000000..c992b6b4ebecc --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java @@ -0,0 +1,87 @@ +/* + * 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.stormcompatibility.split; + +import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.stormcompatibility.split.stormoperators.RandomSpout; +import org.apache.flink.stormcompatibility.split.stormoperators.VerifyAndEnrichBolt; +import org.apache.flink.stormcompatibility.util.OutputFormatter; +import org.apache.flink.stormcompatibility.util.StormBoltFileSink; +import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; +import org.apache.flink.stormcompatibility.util.TupleOutputFormatter; + +public class SplitBoltTopology { + public final static String spoutId = "randomSource"; + public final static String boltId = "splitBolt"; + public final static String evenVerifierId = "evenVerifier"; + public final static String oddVerifierId = "oddVerifier"; + public final static String sinkId = "sink"; + private final static OutputFormatter formatter = new TupleOutputFormatter(); + + public static FlinkTopologyBuilder buildTopology() { + final FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); + + builder.setSpout(spoutId, new RandomSpout(false, seed)); + builder.setBolt(boltId, new SplitBolt()).shuffleGrouping(spoutId); + builder.setBolt(evenVerifierId, new VerifyAndEnrichBolt(true)).shuffleGrouping(boltId, + SplitBolt.EVEN_STREAM); + builder.setBolt(oddVerifierId, new VerifyAndEnrichBolt(false)).shuffleGrouping(boltId, + SplitBolt.ODD_STREAM); + + // emit result + if (outputPath != null) { + // read the text file from given input path + final String[] tokens = outputPath.split(":"); + final String outputFile = tokens[tokens.length - 1]; + builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + } else { + builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + } + + return builder; + } + + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static long seed = System.currentTimeMillis(); + private static String outputPath = null; + + static boolean parseParameters(final String[] args) { + + if (args.length > 0) { + // parse input arguments + if (args.length == 2) { + seed = Long.parseLong(args[0]); + outputPath = args[1]; + } else { + System.err.println("Usage: SplitBoltTopology "); + return false; + } + } else { + System.out.println("Executing SplitBoltTopology example with random data"); + System.out.println(" Usage: SplitBoltTopology "); + } + + return true; + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java new file mode 100644 index 0000000000000..613fd10b1f0bd --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java @@ -0,0 +1,85 @@ +/* + * 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.stormcompatibility.split; + +import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.stormcompatibility.split.stormoperators.RandomSpout; +import org.apache.flink.stormcompatibility.split.stormoperators.VerifyAndEnrichBolt; +import org.apache.flink.stormcompatibility.util.OutputFormatter; +import org.apache.flink.stormcompatibility.util.StormBoltFileSink; +import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; +import org.apache.flink.stormcompatibility.util.TupleOutputFormatter; + +public class SplitSpoutTopology { + public final static String spoutId = "randomSplitSource"; + public final static String evenVerifierId = "evenVerifier"; + public final static String oddVerifierId = "oddVerifier"; + public final static String sinkId = "sink"; + private final static OutputFormatter formatter = new TupleOutputFormatter(); + + public static FlinkTopologyBuilder buildTopology() { + final FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); + + builder.setSpout(spoutId, new RandomSpout(true, seed)); + builder.setBolt(evenVerifierId, new VerifyAndEnrichBolt(true)).shuffleGrouping(spoutId, + RandomSpout.EVEN_STREAM); + builder.setBolt(oddVerifierId, new VerifyAndEnrichBolt(false)).shuffleGrouping(spoutId, + RandomSpout.ODD_STREAM); + + // emit result + if (outputPath != null) { + // read the text file from given input path + final String[] tokens = outputPath.split(":"); + final String outputFile = tokens[tokens.length - 1]; + builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + } else { + builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + } + + return builder; + } + + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static long seed = System.currentTimeMillis(); + private static String outputPath = null; + + static boolean parseParameters(final String[] args) { + + if (args.length > 0) { + // parse input arguments + if (args.length == 2) { + seed = Long.parseLong(args[0]); + outputPath = args[1]; + } else { + System.err.println("Usage: SplitSpoutTopology "); + return false; + } + } else { + System.out.println("Executing SplitSpoutTopology example with random data"); + System.out.println(" Usage: SplitSpoutTopology "); + } + + return true; + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java new file mode 100644 index 0000000000000..f30e160c51f49 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java @@ -0,0 +1,28 @@ +/* + * 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.stormcompatibility.split; + +import org.junit.Test; + +public class SpoutSplitITCase { + + @Test + public void testTopology() throws Exception { + StormSplitStreamSpoutLocal.main(new String[] { "0", "/dev/null" }); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java new file mode 100644 index 0000000000000..028f6d113dff1 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java @@ -0,0 +1,51 @@ +/* + * 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.stormcompatibility.split; + +import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; +import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; + +import backtype.storm.utils.Utils; + +public class StormSplitStreamBoltLocal { + public final static String topologyId = "Bolt split stream example"; + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(final String[] args) throws Exception { + + if (!SplitBoltTopology.parseParameters(args)) { + return; + } + + // build Topology the Storm way + final FlinkTopologyBuilder builder = SplitBoltTopology.buildTopology(); + + // execute program locally + final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster(); + cluster.submitTopology(topologyId, null, builder.createTopology()); + + Utils.sleep(5 * 1000); + + // TODO kill does no do anything so far + cluster.killTopology(topologyId); + cluster.shutdown(); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java new file mode 100644 index 0000000000000..cc5acd969e448 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java @@ -0,0 +1,51 @@ +/* + * 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.stormcompatibility.split; + +import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; +import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; + +import backtype.storm.utils.Utils; + +public class StormSplitStreamSpoutLocal { + public final static String topologyId = "Spout split stream example"; + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(final String[] args) throws Exception { + + if (!SplitSpoutTopology.parseParameters(args)) { + return; + } + + // build Topology the Storm way + final FlinkTopologyBuilder builder = SplitSpoutTopology.buildTopology(); + + // execute program locally + final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster(); + cluster.submitTopology(topologyId, null, builder.createTopology()); + + Utils.sleep(5 * 1000); + + // TODO kill does no do anything so far + cluster.killTopology(topologyId); + cluster.shutdown(); + } + +} From 970ab35ec8f0268ed38234f35fa50706c5ff3071 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 3 Aug 2015 14:37:19 +0200 Subject: [PATCH 158/175] [FLINK-2451] [gelly] removed redundant examples; added comments describing which gelly method each example illustrates. --- .../graph/example/CommunityDetection.java | 142 -------------- .../graph/example/ConnectedComponents.java | 11 +- .../graph/example/EuclideanGraphWeighing.java | 3 + .../example/GSASingleSourceShortestPaths.java | 27 ++- .../flink/graph/example/GraphMetrics.java | 3 +- .../flink/graph/example/IncrementalSSSP.java | 6 +- .../example/JaccardSimilarityMeasure.java | 10 +- .../flink/graph/example/LabelPropagation.java | 171 ----------------- .../flink/graph/example/MusicProfiles.java | 46 ++--- .../apache/flink/graph/example/PageRank.java | 153 --------------- .../example/SingleSourceShortestPaths.java | 106 +++++++++-- .../GSAConnectedComponents.java | 0 .../{example => library}/GSAPageRank.java | 0 .../library/GSASingleSourceShortestPaths.java | 180 ++++++++++++++++++ 14 files changed, 328 insertions(+), 530 deletions(-) delete mode 100644 flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/CommunityDetection.java delete mode 100644 flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/LabelPropagation.java delete mode 100644 flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/PageRank.java rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/{example => library}/GSAConnectedComponents.java (100%) rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/{example => library}/GSAPageRank.java (100%) create mode 100755 flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/CommunityDetection.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/CommunityDetection.java deleted file mode 100644 index e44e5bd7920dc..0000000000000 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/CommunityDetection.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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.graph.example; - -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.graph.Edge; -import org.apache.flink.graph.Graph; -import org.apache.flink.graph.Vertex; -import org.apache.flink.graph.example.utils.CommunityDetectionData; -import org.apache.flink.graph.library.CommunityDetectionAlgorithm; -import org.apache.flink.graph.utils.Tuple3ToEdgeMap; - -/** - * This example shows how to use the {@link org.apache.flink.graph.library.CommunityDetectionAlgorithm} - * library method: - *

      - *
    • with the edge data set given as a parameter - *
    • with default data - *
    - * - * The input file is a plain text file and must be formatted as follows: - * Edges are represented by tuples of srcVertexId, trgVertexId, weight which are - * separated by tabs. Edges themselves are separated by newlines. - * For example: 1\t2\t1.0\n1\t3\t2.0\n defines two edges, - * 1-2 with weight 1.0 and 1-3 with weight 2.0. - * - * Usage CommunityDetection <edge path> <result path> - * <number of iterations> <delta>
    - * If no parameters are provided, the program is run with default data from - * {@link org.apache.flink.graph.example.utils.CommunityDetectionData} - */ -public class CommunityDetection implements ProgramDescription { - - @SuppressWarnings("serial") - public static void main(String [] args) throws Exception { - - if(!parseParameters(args)) { - return; - } - - // set up the execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // set up the graph - DataSet> edges = getEdgesDataSet(env); - Graph graph = Graph.fromDataSet(edges, - new MapFunction() { - - public Long map(Long label) { - return label; - } - }, env); - - // the result is in the form of , where the communityId is the label - // which the vertex converged to - DataSet> communityVertices = - graph.run(new CommunityDetectionAlgorithm(maxIterations, delta)).getVertices(); - - // emit result - if (fileOutput) { - communityVertices.writeAsCsv(outputPath, "\n", ","); - - // since file sinks are lazy, we trigger the execution explicitly - env.execute("Executing Community Detection Example"); - } else { - communityVertices.print(); - } - - } - - @Override - public String getDescription() { - return "Community Detection"; - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileOutput = false; - private static String edgeInputPath = null; - private static String outputPath = null; - private static Integer maxIterations = CommunityDetectionData.MAX_ITERATIONS; - private static Double delta = CommunityDetectionData.DELTA; - - private static boolean parseParameters(String [] args) { - if(args.length > 0) { - if(args.length != 4) { - System.err.println("Usage CommunityDetection " + - " "); - return false; - } - - fileOutput = true; - edgeInputPath = args[0]; - outputPath = args[1]; - maxIterations = Integer.parseInt(args[2]); - delta = Double.parseDouble(args[3]); - - } else { - System.out.println("Executing SimpleCommunityDetection example with default parameters and built-in default data."); - System.out.println("Provide parameters to read input data from files."); - System.out.println("Usage CommunityDetection " + - " "); - } - - return true; - } - - private static DataSet> getEdgesDataSet(ExecutionEnvironment env) { - - if(fileOutput) { - return env.readCsvFile(edgeInputPath) - .ignoreComments("#") - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class, Double.class) - .map(new Tuple3ToEdgeMap()); - } else { - return CommunityDetectionData.getDefaultEdgeDataSet(env); - } - } -} diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java index 3443a553a5e0d..b841ced9abcd0 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java @@ -31,12 +31,11 @@ import org.apache.flink.types.NullValue; /** - * This example shows how to use the {@link org.apache.flink.graph.library.ConnectedComponentsAlgorithm} - * library method: - *
      - *
    • with the edge data set given as a parameter - *
    • with default data - *
    + * This example shows how to use Gelly's library methods. + * You can find all available library methods in {@link org.apache.flink.graph.library}. + * + * In particular, this example uses the {@link org.apache.flink.graph.library.ConnectedComponentsAlgorithm} + * library method to compute the connected components of the input graph. * * The input file is a plain text file and must be formatted as follows: * Edges are represented by tuples of srcVertexId, trgVertexId which are diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java index 7e2c057f106a0..b7e3385c2fa9f 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java @@ -33,6 +33,9 @@ import java.io.Serializable; /** + * This example shows how to use Gelly's {@link Graph#getTriplets()} and + * {@link Graph#joinWithEdges(DataSet, MapFunction)} methods. + * * Given a directed, unweighted graph, with vertex values representing points in a plan, * return a weighted graph where the edge weights are equal to the Euclidean distance between the * src and the trg vertex values. diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java index b01aa234d9135..23a3a82f25757 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java @@ -33,7 +33,19 @@ import org.apache.flink.graph.utils.Tuple3ToEdgeMap; /** - * This is an implementation of the Single Source Shortest Paths algorithm, using a gather-sum-apply iteration + * This example shows how to use Gelly's Gather-Sum-Apply iterations. + * + * It is an implementation of the Single-Source-Shortest-Paths algorithm. + * For a vertex-centric implementation of the same algorithm, please refer to {@link SingleSourceShortestPaths}. + * + * The input file is a plain text file and must be formatted as follows: + * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are + * separated by tabs. Edges themselves are separated by newlines. + * For example: 1\t2\t0.1\n1\t3\t1.4\n defines two edges, + * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4. + * + * If no parameters are provided, the program is run with default data from + * {@link org.apache.flink.graph.example.utils.SingleSourceShortestPathsData} */ public class GSASingleSourceShortestPaths implements ProgramDescription { @@ -54,9 +66,8 @@ public static void main(String[] args) throws Exception { Graph graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env); // Execute the GSA iteration - Graph result = graph - .runGatherSumApplyIteration(new CalculateDistances(), new ChooseMinDistance(), - new UpdateDistance(), maxIterations); + Graph result = graph.runGatherSumApplyIteration( + new CalculateDistances(), new ChooseMinDistance(), new UpdateDistance(), maxIterations); // Extract the vertices as the result DataSet> singleSourceShortestPaths = result.getVertices(); @@ -73,6 +84,10 @@ public static void main(String[] args) throws Exception { } + // -------------------------------------------------------------------------------------------- + // Single Source Shortest Path UDFs + // -------------------------------------------------------------------------------------------- + @SuppressWarnings("serial") private static final class InitVertices implements MapFunction{ @@ -92,10 +107,6 @@ public Double map(Long id) { } } - // -------------------------------------------------------------------------------------------- - // Single Source Shortest Path UDFs - // -------------------------------------------------------------------------------------------- - @SuppressWarnings("serial") private static final class CalculateDistances extends GatherFunction { diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java index c6a776dfbdfaf..6c4d0c256d78e 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java @@ -30,8 +30,9 @@ import org.apache.flink.types.NullValue; /** + * This example illustrate how to use Gelly metrics methods and get simple statistics + * from the input graph. * - * A simple example to illustrate the basic functionality of the graph-api. * The program creates a random graph and computes and prints * the following metrics: * - number of vertices diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java index 8d94cbc165065..cc672b27a962d 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java @@ -35,11 +35,11 @@ import org.apache.flink.graph.utils.Tuple3ToEdgeMap; /** - * Incremental Single Sink Shortest Paths Example. Shortest Paths are incrementally updated - * upon edge removal. - * * This example illustrates the usage of vertex-centric iteration's * messaging direction configuration option. + * + * Incremental Single Sink Shortest Paths Example. Shortest Paths are incrementally updated + * upon edge removal. * * The program takes as input the resulted graph after a SSSP computation, * an edge to be removed and the initial graph(i.e. before SSSP was computed). diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java index 79de407b5f847..0f84dbbc787e8 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java @@ -34,6 +34,13 @@ import java.util.HashSet; /** + * This example shows how to use + *
      + *
    • neighborhood methods + *
    • join with vertices + *
    • triplets + *
    + * * Given a directed, unweighted graph, return a weighted graph where the edge values are equal * to the Jaccard similarity coefficient - the number of common neighbors divided by the the size * of the union of neighbor sets - for the src and target vertices. @@ -117,8 +124,7 @@ public String getDescription() { private static final class GatherNeighbors implements ReduceNeighborsFunction> { @Override - public HashSet reduceNeighbors(HashSet first, - HashSet second) { + public HashSet reduceNeighbors(HashSet first, HashSet second) { first.addAll(second); return new HashSet(first); } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/LabelPropagation.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/LabelPropagation.java deleted file mode 100644 index bee5af378f474..0000000000000 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/LabelPropagation.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.graph.example; - -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.graph.Edge; -import org.apache.flink.graph.Graph; -import org.apache.flink.graph.Vertex; -import org.apache.flink.graph.library.LabelPropagationAlgorithm; -import org.apache.flink.graph.utils.Tuple2ToVertexMap; -import org.apache.flink.types.NullValue; -import org.apache.flink.util.Collector; - -/** - * This example uses the label propagation algorithm to detect communities by - * propagating labels. Initially, each vertex is assigned its id as its label. - * The vertices iteratively propagate their labels to their neighbors and adopt - * the most frequent label among their neighbors. The algorithm converges when - * no vertex changes value or the maximum number of iterations have been - * reached. - * - * The edges input file is expected to contain one edge per line, with long IDs - * in the following format:"\t". - * - * The vertices input file is expected to contain one vertex per line, with long IDs - * and long vertex values, in the following format:"\t". - * - * If no arguments are provided, the example runs with a random graph of 100 vertices. - */ -public class LabelPropagation implements ProgramDescription { - - public static void main(String[] args) throws Exception { - - if(!parseParameters(args)) { - return; - } - - // Set up the execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // Set up the graph - DataSet> vertices = getVertexDataSet(env); - DataSet> edges = getEdgeDataSet(env); - - Graph graph = Graph.fromDataSet(vertices, edges, env); - - // Set up the program - DataSet> verticesWithCommunity = graph.run( - new LabelPropagationAlgorithm(maxIterations)).getVertices(); - - // Emit results - if(fileOutput) { - verticesWithCommunity.writeAsCsv(outputPath, "\n", ","); - - // Execute the program - env.execute("Label Propagation Example"); - } else { - verticesWithCommunity.print(); - } - - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileOutput = false; - private static String vertexInputPath = null; - private static String edgeInputPath = null; - private static String outputPath = null; - private static long numVertices = 100; - private static int maxIterations = 10; - - private static boolean parseParameters(String[] args) { - - if(args.length > 0) { - if(args.length != 4) { - System.err.println("Usage: LabelPropagation "); - return false; - } - - fileOutput = true; - vertexInputPath = args[0]; - edgeInputPath = args[1]; - outputPath = args[2]; - maxIterations = Integer.parseInt(args[3]); - } else { - System.out.println("Executing LabelPropagation example with default parameters and built-in default data."); - System.out.println(" Provide parameters to read input data from files."); - System.out.println(" See the documentation for the correct format of input files."); - System.out.println(" Usage: LabelPropagation "); - } - return true; - } - - @SuppressWarnings("serial") - private static DataSet> getVertexDataSet(ExecutionEnvironment env) { - - if (fileOutput) { - return env.readCsvFile(vertexInputPath) - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class) - .map(new Tuple2ToVertexMap()); - } - - return env.generateSequence(1, numVertices).map( - new MapFunction>() { - public Vertex map(Long l) throws Exception { - return new Vertex(l, l); - } - }); - } - - @SuppressWarnings("serial") - private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { - - if (fileOutput) { - return env.readCsvFile(edgeInputPath) - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class) - .map(new MapFunction, Edge>() { - @Override - public Edge map(Tuple2 value) throws Exception { - return new Edge(value.f0, value.f1, NullValue.getInstance()); - } - }); - } - - return env.generateSequence(1, numVertices).flatMap( - new FlatMapFunction>() { - @Override - public void flatMap(Long key, - Collector> out) { - int numOutEdges = (int) (Math.random() * (numVertices / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numVertices) + 1; - out.collect(new Edge(key, target, - NullValue.getInstance())); - } - } - }); - } - - @Override - public String getDescription() { - return "Label Propagation Example"; - } -} \ No newline at end of file diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java index 0fc45bdc9120b..764397677cde0 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java @@ -42,32 +42,32 @@ import org.apache.flink.types.NullValue; import org.apache.flink.util.Collector; +/** + * This example demonstrates how to mix the DataSet Flink API with the Gelly API. + * The input is a set triplets and + * a set of bad records, i.e. song ids that should not be trusted. + * Initially, we use the DataSet API to filter out the bad records. + * Then, we use Gelly to create a user -> song weighted bipartite graph and compute + * the top song (most listened) per user. + * Then, we use the DataSet API again, to create a user-user similarity graph, + * based on common songs, where users that are listeners of the same song + * are connected. A user-defined threshold on the playcount value + * defines when a user is considered to be a listener of a song. + * Finally, we use the graph API to run the label propagation community detection algorithm on + * the similarity graph. + * + * The triplets input is expected to be given as one triplet per line, + * in the following format: "\t\t". + * + * The mismatches input file is expected to contain one mismatch record per line, + * in the following format: + * "ERROR: song_title" + * + * If no arguments are provided, the example runs with default data from {@link MusicProfilesData}. + */ @SuppressWarnings("serial") public class MusicProfiles implements ProgramDescription { - /** - * This example demonstrates how to mix the DataSet Flink API with the Gelly API. - * The input is a set triplets and - * a set of bad records, i.e. song ids that should not be trusted. - * Initially, we use the DataSet API to filter out the bad records. - * Then, we use Gelly to create a user -> song weighted bipartite graph and compute - * the top song (most listened) per user. - * Then, we use the DataSet API again, to create a user-user similarity graph, - * based on common songs, where users that are listeners of the same song - * are connected. A user-defined threshold on the playcount value - * defines when a user is considered to be a listener of a song. - * Finally, we use the graph API to run the label propagation community detection algorithm on - * the similarity graph. - * - * The triplets input is expected to be given as one triplet per line, - * in the following format: "\t\t". - * - * The mismatches input file is expected to contain one mismatch record per line, - * in the following format: - * "ERROR: song_title" - * - * If no arguments are provided, the example runs with default data from {@link MusicProfilesData}. - */ public static void main(String[] args) throws Exception { if (!parseParameters(args)) { diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/PageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/PageRank.java deleted file mode 100644 index 10b4be4cbb041..0000000000000 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/PageRank.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.graph.example; - -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.graph.Edge; -import org.apache.flink.graph.Graph; -import org.apache.flink.graph.Vertex; -import org.apache.flink.graph.library.PageRankAlgorithm; -import org.apache.flink.util.Collector; - -/** - * This example implements a simple PageRank algorithm, using a vertex-centric iteration. - * - * The edges input file is expected to contain one edge per line, with long IDs and no - * values, in the following format:"\t". - * - * If no arguments are provided, the example runs with a random graph of 10 vertices. - * - */ -public class PageRank implements ProgramDescription { - - @SuppressWarnings("serial") - public static void main(String[] args) throws Exception { - - if(!parseParameters(args)) { - return; - } - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> links = getLinksDataSet(env); - - Graph network = Graph.fromDataSet(links, new MapFunction() { - - public Double map(Long value) throws Exception { - return 1.0; - } - }, env); - - DataSet> vertexOutDegrees = network.outDegrees(); - - // assign the transition probabilities as the edge weights - Graph networkWithWeights = network - .joinWithEdgesOnSource(vertexOutDegrees, - new MapFunction, Double>() { - public Double map(Tuple2 value) { - return value.f0 / value.f1; - } - }); - - DataSet> pageRanks = networkWithWeights.run( - new PageRankAlgorithm(DAMPENING_FACTOR, maxIterations)) - .getVertices(); - - if (fileOutput) { - pageRanks.writeAsCsv(outputPath, "\n", "\t"); - - // since file sinks are lazy, we trigger the execution explicitly - env.execute(); - } else { - pageRanks.print(); - } - - } - - @Override - public String getDescription() { - return "PageRank example"; - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileOutput = false; - private static final double DAMPENING_FACTOR = 0.85; - private static long numPages = 10; - private static String edgeInputPath = null; - private static String outputPath = null; - private static int maxIterations = 10; - - private static boolean parseParameters(String[] args) { - - if(args.length > 0) { - if(args.length != 3) { - System.err.println("Usage: PageRank "); - return false; - } - - fileOutput = true; - edgeInputPath = args[0]; - outputPath = args[1]; - maxIterations = Integer.parseInt(args[2]); - } else { - System.out.println("Executing PageRank example with default parameters and built-in default data."); - System.out.println(" Provide parameters to read input data from files."); - System.out.println(" See the documentation for the correct format of input files."); - System.out.println(" Usage: PageRank "); - } - return true; - } - - @SuppressWarnings("serial") - private static DataSet> getLinksDataSet(ExecutionEnvironment env) { - - if (fileOutput) { - return env.readCsvFile(edgeInputPath) - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class) - .map(new MapFunction, Edge>() { - public Edge map(Tuple2 input) { - return new Edge(input.f0, input.f1, 1.0); - } - }).withForwardedFields("f0; f1"); - } - - return env.generateSequence(1, numPages).flatMap( - new FlatMapFunction>() { - @Override - public void flatMap(Long key, - Collector> out) throws Exception { - int numOutEdges = (int) (Math.random() * (numPages / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numPages) + 1; - out.collect(new Edge(key, target, 1.0)); - } - } - }); - } -} diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java index 9d7d2c20974e6..391ebaf3b56ee 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java @@ -26,23 +26,28 @@ import org.apache.flink.graph.Graph; import org.apache.flink.graph.Vertex; import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData; -import org.apache.flink.graph.library.SingleSourceShortestPathsAlgorithm; +import org.apache.flink.graph.spargel.MessageIterator; +import org.apache.flink.graph.spargel.MessagingFunction; +import org.apache.flink.graph.spargel.VertexUpdateFunction; import org.apache.flink.graph.utils.Tuple3ToEdgeMap; /** - * This example implements the Single Source Shortest Paths algorithm, - * using a vertex-centric iteration. + * This example shows how to use Gelly's vertex-centric iterations. + * + * It is an implementation of the Single-Source-Shortest-Paths algorithm. + * For a gather-sum-apply implementation of the same algorithm, please refer to {@link GSASingleSourceShortestPaths}. * - * The input file is expected to contain one edge per line, with long IDs - * and double weights, separated by tabs, in the following format: - * "\t\t". - * - * If no arguments are provided, the example runs with default data from {@link SingleSourceShortestPathsData}. + * The input file is a plain text file and must be formatted as follows: + * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are + * separated by tabs. Edges themselves are separated by newlines. + * For example: 1\t2\t0.1\n1\t3\t1.4\n defines two edges, + * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4. * + * If no parameters are provided, the program is run with default data from + * {@link org.apache.flink.graph.example.utils.SingleSourceShortestPathsData} */ public class SingleSourceShortestPaths implements ProgramDescription { - @SuppressWarnings("serial") public static void main(String[] args) throws Exception { if (!parseParameters(args)) { @@ -53,17 +58,14 @@ public static void main(String[] args) throws Exception { DataSet> edges = getEdgesDataSet(env); - Graph graph = Graph.fromDataSet(edges, - new MapFunction() { + Graph graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env); - public Double map(Long value) { - return Double.MAX_VALUE; - } - }, env); + // Execute the vertex-centric iteration + Graph result = graph.runVertexCentricIteration( + new VertexDistanceUpdater(), new MinDistanceMessenger(), maxIterations); - DataSet> singleSourceShortestPaths = graph - .run(new SingleSourceShortestPathsAlgorithm(srcVertexId, maxIterations)) - .getVertices(); + // Extract the vertices as the result + DataSet> singleSourceShortestPaths = result.getVertices(); // emit result if (fileOutput) { @@ -77,9 +79,66 @@ public Double map(Long value) { } - @Override - public String getDescription() { - return "Single Source Shortest Paths"; + // -------------------------------------------------------------------------------------------- + // Single Source Shortest Path UDFs + // -------------------------------------------------------------------------------------------- + + @SuppressWarnings("serial") + private static final class InitVertices implements MapFunction{ + + private long srcId; + + public InitVertices(long srcId) { + this.srcId = srcId; + } + + public Double map(Long id) { + if (id.equals(srcId)) { + return 0.0; + } + else { + return Double.POSITIVE_INFINITY; + } + } + } + + /** + * Function that updates the value of a vertex by picking the minimum + * distance from all incoming messages. + */ + @SuppressWarnings("serial") + public static final class VertexDistanceUpdater extends VertexUpdateFunction { + + @Override + public void updateVertex(Vertex vertex, MessageIterator inMessages) { + + Double minDistance = Double.MAX_VALUE; + + for (double msg : inMessages) { + if (msg < minDistance) { + minDistance = msg; + } + } + + if (vertex.getValue() > minDistance) { + setNewVertexValue(minDistance); + } + } + } + + /** + * Distributes the minimum distance associated with a given vertex among all + * the target vertices summed up with the edge's value. + */ + @SuppressWarnings("serial") + public static final class MinDistanceMessenger extends MessagingFunction { + + @Override + public void sendMessages(Vertex vertex) { + for (Edge edge : getEdges()) { + sendMessageTo(edge.getTarget(), vertex.getValue() + edge.getValue()); + } + } } // ****************************************************************************************************************** @@ -132,4 +191,9 @@ private static DataSet> getEdgesDataSet(ExecutionEnvironment return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env); } } + + @Override + public String getDescription() { + return "Vertex-centric Single Source Shortest Paths"; + } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSAConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java similarity index 100% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSAConnectedComponents.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSAPageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java similarity index 100% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSAPageRank.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java new file mode 100755 index 0000000000000..b01aa234d9135 --- /dev/null +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java @@ -0,0 +1,180 @@ +/* + * 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.graph.example; + +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData; +import org.apache.flink.graph.gsa.ApplyFunction; +import org.apache.flink.graph.gsa.GatherFunction; +import org.apache.flink.graph.gsa.SumFunction; +import org.apache.flink.graph.gsa.Neighbor; +import org.apache.flink.graph.utils.Tuple3ToEdgeMap; + +/** + * This is an implementation of the Single Source Shortest Paths algorithm, using a gather-sum-apply iteration + */ +public class GSASingleSourceShortestPaths implements ProgramDescription { + + // -------------------------------------------------------------------------------------------- + // Program + // -------------------------------------------------------------------------------------------- + + public static void main(String[] args) throws Exception { + + if(!parseParameters(args)) { + return; + } + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> edges = getEdgeDataSet(env); + + Graph graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env); + + // Execute the GSA iteration + Graph result = graph + .runGatherSumApplyIteration(new CalculateDistances(), new ChooseMinDistance(), + new UpdateDistance(), maxIterations); + + // Extract the vertices as the result + DataSet> singleSourceShortestPaths = result.getVertices(); + + // emit result + if(fileOutput) { + singleSourceShortestPaths.writeAsCsv(outputPath, "\n", " "); + + // since file sinks are lazy, we trigger the execution explicitly + env.execute("GSA Single Source Shortest Paths"); + } else { + singleSourceShortestPaths.print(); + } + + } + + @SuppressWarnings("serial") + private static final class InitVertices implements MapFunction{ + + private long srcId; + + public InitVertices(long srcId) { + this.srcId = srcId; + } + + public Double map(Long id) { + if (id.equals(srcId)) { + return 0.0; + } + else { + return Double.POSITIVE_INFINITY; + } + } + } + + // -------------------------------------------------------------------------------------------- + // Single Source Shortest Path UDFs + // -------------------------------------------------------------------------------------------- + + @SuppressWarnings("serial") + private static final class CalculateDistances extends GatherFunction { + + public Double gather(Neighbor neighbor) { + return neighbor.getNeighborValue() + neighbor.getEdgeValue(); + } + }; + + @SuppressWarnings("serial") + private static final class ChooseMinDistance extends SumFunction { + + public Double sum(Double newValue, Double currentValue) { + return Math.min(newValue, currentValue); + } + }; + + @SuppressWarnings("serial") + private static final class UpdateDistance extends ApplyFunction { + + public void apply(Double newDistance, Double oldDistance) { + if (newDistance < oldDistance) { + setResult(newDistance); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Util methods + // -------------------------------------------------------------------------------------------- + + private static boolean fileOutput = false; + + private static Long srcVertexId = 1l; + + private static String edgesInputPath = null; + + private static String outputPath = null; + + private static int maxIterations = 5; + + private static boolean parseParameters(String[] args) { + + if (args.length > 0) { + if(args.length != 4) { + System.err.println("Usage: GSASingleSourceShortestPaths " + + " "); + return false; + } + + fileOutput = true; + srcVertexId = Long.parseLong(args[0]); + edgesInputPath = args[1]; + outputPath = args[2]; + maxIterations = Integer.parseInt(args[3]); + } else { + System.out.println("Executing GSASingle Source Shortest Paths example " + + "with default parameters and built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); + System.out.println("Usage: GSASingleSourceShortestPaths " + + " "); + } + return true; + } + + private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { + if (fileOutput) { + return env.readCsvFile(edgesInputPath) + .fieldDelimiter("\t") + .lineDelimiter("\n") + .types(Long.class, Long.class, Double.class) + .map(new Tuple3ToEdgeMap()); + } else { + return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env); + } + } + + @Override + public String getDescription() { + return "GSA Single Source Shortest Paths"; + } +} From d2d061c3004dcdc31a1df6b422a6aba6d9c294f8 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 3 Aug 2015 15:20:44 +0200 Subject: [PATCH 159/175] [FLINK-2451] [gelly] library methods cleanup --- .../graph/example/ConnectedComponents.java | 4 +- .../flink/graph/example/MusicProfiles.java | 4 +- ...Algorithm.java => CommunityDetection.java} | 4 +- ...lgorithm.java => ConnectedComponents.java} | 9 +- .../graph/library/GSAConnectedComponents.java | 127 ++------------ .../flink/graph/library/GSAPageRank.java | 161 +++--------------- .../library/GSASingleSourceShortestPaths.java | 126 +++----------- ...onAlgorithm.java => LabelPropagation.java} | 6 +- .../{PageRankAlgorithm.java => PageRank.java} | 16 +- ...hm.java => SingleSourceShortestPaths.java} | 10 +- 10 files changed, 84 insertions(+), 383 deletions(-) rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/{CommunityDetectionAlgorithm.java => CommunityDetection.java} (97%) rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/{ConnectedComponentsAlgorithm.java => ConnectedComponents.java} (92%) rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/{LabelPropagationAlgorithm.java => LabelPropagation.java} (94%) rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/{PageRankAlgorithm.java => PageRank.java} (90%) rename flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/{SingleSourceShortestPathsAlgorithm.java => SingleSourceShortestPaths.java} (91%) diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java index b841ced9abcd0..a4a6708cdf917 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java @@ -27,7 +27,7 @@ import org.apache.flink.graph.Graph; import org.apache.flink.graph.Vertex; import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData; -import org.apache.flink.graph.library.ConnectedComponentsAlgorithm; +import org.apache.flink.graph.library.GSAConnectedComponents; import org.apache.flink.types.NullValue; /** @@ -69,7 +69,7 @@ public Long map(Long value) throws Exception { }, env); DataSet> verticesWithMinIds = graph - .run(new ConnectedComponentsAlgorithm(maxIterations)).getVertices(); + .run(new GSAConnectedComponents(maxIterations)).getVertices(); // emit result if (fileOutput) { diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java index 764397677cde0..a56224d3031f9 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java @@ -38,7 +38,7 @@ import org.apache.flink.graph.Graph; import org.apache.flink.graph.Vertex; import org.apache.flink.graph.example.utils.MusicProfilesData; -import org.apache.flink.graph.library.LabelPropagationAlgorithm; +import org.apache.flink.graph.library.LabelPropagation; import org.apache.flink.types.NullValue; import org.apache.flink.util.Collector; @@ -153,7 +153,7 @@ public Tuple2 map(Tuple2 tuple2) throws Exception { public Long map(Tuple2 value) { return value.f1; } - }).run(new LabelPropagationAlgorithm(maxIterations)) + }).run(new LabelPropagation(maxIterations)) .getVertices(); if (fileOutput) { diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetectionAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java similarity index 97% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetectionAlgorithm.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java index 6f72debd928b4..21bef531a5175 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetectionAlgorithm.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java @@ -44,13 +44,13 @@ * * @see article explaining the algorithm in detail */ -public class CommunityDetectionAlgorithm implements GraphAlgorithm { +public class CommunityDetection implements GraphAlgorithm { private Integer maxIterations; private Double delta; - public CommunityDetectionAlgorithm(Integer maxIterations, Double delta) { + public CommunityDetection(Integer maxIterations, Double delta) { this.maxIterations = maxIterations; this.delta = delta; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponentsAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java similarity index 92% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponentsAlgorithm.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java index 7b536e5e27757..c2cec18fddc85 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponentsAlgorithm.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java @@ -27,7 +27,7 @@ import org.apache.flink.types.NullValue; /** - * Connected components algorithm. + * A vertex-centric implementation of the Connected components algorithm. * * Initially, each vertex will have its own ID as a value(is its own component). The vertices propagate their * current component ID in iterations, each time adopting a new value from the received neighbor IDs, @@ -37,11 +37,11 @@ * is reached. */ @SuppressWarnings("serial") -public class ConnectedComponentsAlgorithm implements GraphAlgorithm{ +public class ConnectedComponents implements GraphAlgorithm{ private Integer maxIterations; - public ConnectedComponentsAlgorithm(Integer maxIterations) { + public ConnectedComponents(Integer maxIterations) { this.maxIterations = maxIterations; } @@ -51,8 +51,7 @@ public Graph run(Graph graph) thro Graph undirectedGraph = graph.getUndirected(); // initialize vertex values and run the Vertex Centric Iteration - return undirectedGraph.runVertexCentricIteration(new CCUpdater(), - new CCMessenger(), maxIterations); + return undirectedGraph.runVertexCentricIteration(new CCUpdater(), new CCMessenger(), maxIterations); } /** diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java index 9b75c92ce9bcf..b77ca079a0978 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java @@ -16,71 +16,35 @@ * limitations under the License. */ -package org.apache.flink.graph.example; +package org.apache.flink.graph.library; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.graph.Edge; import org.apache.flink.graph.Graph; -import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.GraphAlgorithm; import org.apache.flink.graph.gsa.ApplyFunction; import org.apache.flink.graph.gsa.GatherFunction; import org.apache.flink.graph.gsa.SumFunction; import org.apache.flink.graph.gsa.Neighbor; import org.apache.flink.types.NullValue; -import org.apache.flink.util.Collector; /** - * This is an implementation of the Connected Components algorithm, using a gather-sum-apply iteration + * This is an implementation of the Connected Components algorithm, using a gather-sum-apply iteration. */ -public class GSAConnectedComponents implements ProgramDescription { +public class GSAConnectedComponents implements GraphAlgorithm { - // -------------------------------------------------------------------------------------------- - // Program - // -------------------------------------------------------------------------------------------- - - public static void main(String[] args) throws Exception { - - if (!parseParameters(args)) { - return; - } - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> edges = getEdgeDataSet(env); - - Graph graph = Graph.fromDataSet(edges, new InitVertices(), env); - - // Execute the GSA iteration - Graph result = - graph.runGatherSumApplyIteration(new GatherNeighborIds(), new SelectMinId(), - new UpdateComponentId(), maxIterations); - - // Extract the vertices as the result - DataSet> connectedComponents = result.getVertices(); - - // emit result - if (fileOutput) { - connectedComponents.writeAsCsv(outputPath, "\n", " "); - - // since file sinks are lazy, we trigger the execution explicitly - env.execute("GSA Connected Components"); - } else { - connectedComponents.print(); - } + private Integer maxIterations; + public GSAConnectedComponents(Integer maxIterations) { + this.maxIterations = maxIterations; } - @SuppressWarnings("serial") - private static final class InitVertices implements MapFunction { + @Override + public Graph run(Graph graph) throws Exception { - public Long map(Long vertexId) { - return vertexId; - } + Graph undirectedGraph = graph.getUndirected(); + + // initialize vertex values and run the Vertex Centric Iteration + return undirectedGraph.runGatherSumApplyIteration(new GatherNeighborIds(), new SelectMinId(), new UpdateComponentId(), + maxIterations); } // -------------------------------------------------------------------------------------------- @@ -112,67 +76,4 @@ public void apply(Long summedValue, Long origValue) { } } } - - // -------------------------------------------------------------------------------------------- - // Util methods - // -------------------------------------------------------------------------------------------- - - private static boolean fileOutput = false; - private static String edgeInputPath = null; - private static String outputPath = null; - - private static int maxIterations = 16; - - private static boolean parseParameters(String[] args) { - - if (args.length > 0) { - // parse input arguments - fileOutput = true; - - if (args.length != 3) { - System.err.println("Usage: GSAConnectedComponents " + - " "); - return false; - } - - edgeInputPath = args[0]; - outputPath = args[1]; - maxIterations = Integer.parseInt(args[2]); - } else { - System.out.println("Executing GSA Connected Components example with built-in default data."); - System.out.println(" Provide parameters to read input data from files."); - System.out.println(" See the documentation for the correct format of input files."); - System.out.println(" Usage: GSAConnectedComponents "); - } - return true; - } - - @SuppressWarnings("serial") - private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { - if (fileOutput) { - return env.readCsvFile(edgeInputPath) - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class) - .map(new MapFunction, Edge>() { - - public Edge map(Tuple2 value) throws Exception { - return new Edge(value.f0, value.f1, NullValue.getInstance()); - } - }); - } - - // Generates 3 components of size 2 - return env.generateSequence(0, 2).flatMap(new FlatMapFunction>() { - @Override - public void flatMap(Long value, Collector> out) throws Exception { - out.collect(new Edge(value, value + 3, NullValue.getInstance())); - } - }); - } - - @Override - public String getDescription() { - return "GSA Connected Components"; - } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java index 45d4555794d48..4299381019b6c 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java @@ -16,86 +16,35 @@ * limitations under the License. */ -package org.apache.flink.graph.example; - -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.graph.Edge; +package org.apache.flink.graph.library; + import org.apache.flink.graph.Graph; -import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.GraphAlgorithm; import org.apache.flink.graph.gsa.ApplyFunction; import org.apache.flink.graph.gsa.GatherFunction; import org.apache.flink.graph.gsa.Neighbor; import org.apache.flink.graph.gsa.SumFunction; -import org.apache.flink.util.Collector; /** - * This example implements a simple PageRank algorithm, using a gather-sum-apply iteration. - * - * The edges input file is expected to contain one edge per line, with long IDs and no - * values, in the following format:"\t". - * - * If no arguments are provided, the example runs with a random graph of 10 vertices - * and random edge weights. + * This is an implementation of a simple PageRank algorithm, using a gather-sum-apply iteration. */ -public class GSAPageRank implements ProgramDescription { - - @SuppressWarnings("serial") - public static void main(String[] args) throws Exception { +public class GSAPageRank implements GraphAlgorithm { - if(!parseParameters(args)) { - return; - } - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> links = getLinksDataSet(env); + private double beta; + private int maxIterations; - Graph network = Graph.fromDataSet(links, new MapFunction() { - - @Override - public Double map(Long value) throws Exception { - return 1.0; - } - }, env); - - DataSet> vertexOutDegrees = network.outDegrees(); - - // Assign the transition probabilities as the edge weights - Graph networkWithWeights = network - .joinWithEdgesOnSource(vertexOutDegrees, - new MapFunction, Double>() { - - @Override - public Double map(Tuple2 value) { - return value.f0 / value.f1; - } - }); - - long numberOfVertices = networkWithWeights.numberOfVertices(); - - // Execute the GSA iteration - Graph result = networkWithWeights - .runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(), - new UpdateRanks(numberOfVertices), maxIterations); - - // Extract the vertices as the result - DataSet> pageRanks = result.getVertices(); + public GSAPageRank(double beta, int maxIterations) { + this.beta = beta; + this.maxIterations = maxIterations; + } - // emit result - if (fileOutput) { - pageRanks.writeAsCsv(outputPath, "\n", "\t"); + @Override + public Graph run(Graph network) throws Exception { - // since file sinks are lazy, we trigger the execution explicitly - env.execute("GSA Page Ranks"); - } else { - pageRanks.print(); - } + final long numberOfVertices = network.numberOfVertices(); + return network.runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(), + new UpdateRanks(beta, numberOfVertices), maxIterations); } // -------------------------------------------------------------------------------------------- @@ -133,83 +82,19 @@ public Double sum(Double newValue, Double currentValue) { } @SuppressWarnings("serial") - private static final class UpdateRanks extends ApplyFunction { + private static final class UpdateRanks extends ApplyFunction { - long numberOfVertices; + private final double beta; + private final long numVertices; - public UpdateRanks(long numberOfVertices) { - this.numberOfVertices = numberOfVertices; + public UpdateRanks(double beta, long numberOfVertices) { + this.beta = beta; + this.numVertices = numberOfVertices; } @Override public void apply(Double rankSum, Double currentValue) { - setResult((1-DAMPENING_FACTOR)/numberOfVertices + DAMPENING_FACTOR * rankSum); - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileOutput = false; - private static final double DAMPENING_FACTOR = 0.85; - private static long numPages = 10; - private static String edgeInputPath = null; - private static String outputPath = null; - private static int maxIterations = 10; - - private static boolean parseParameters(String[] args) { - - if(args.length > 0) { - if(args.length != 3) { - System.err.println("Usage: GSAPageRank "); - return false; - } - - fileOutput = true; - edgeInputPath = args[0]; - outputPath = args[1]; - maxIterations = Integer.parseInt(args[2]); - } else { - System.out.println("Executing GSAPageRank example with default parameters and built-in default data."); - System.out.println(" Provide parameters to read input data from files."); - System.out.println(" See the documentation for the correct format of input files."); - System.out.println(" Usage: GSAPageRank "); - } - return true; - } - - @SuppressWarnings("serial") - private static DataSet> getLinksDataSet(ExecutionEnvironment env) { - - if (fileOutput) { - return env.readCsvFile(edgeInputPath) - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class) - .map(new MapFunction, Edge>() { - public Edge map(Tuple2 input) { - return new Edge(input.f0, input.f1, 1.0); - } - }).withForwardedFields("f0; f1"); + setResult((1-beta)/numVertices + beta * rankSum); } - - return env.generateSequence(1, numPages).flatMap( - new FlatMapFunction>() { - @Override - public void flatMap(Long key, - Collector> out) throws Exception { - int numOutEdges = (int) (Math.random() * (numPages / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numPages) + 1; - out.collect(new Edge(key, target, 1.0)); - } - } - }); - } - - @Override - public String getDescription() { - return "GSA Page Rank"; } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java index b01aa234d9135..78c535baeef6d 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java @@ -16,78 +16,52 @@ * limitations under the License. */ -package org.apache.flink.graph.example; +package org.apache.flink.graph.library; -import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.graph.Edge; import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; import org.apache.flink.graph.Vertex; -import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData; import org.apache.flink.graph.gsa.ApplyFunction; import org.apache.flink.graph.gsa.GatherFunction; import org.apache.flink.graph.gsa.SumFunction; import org.apache.flink.graph.gsa.Neighbor; -import org.apache.flink.graph.utils.Tuple3ToEdgeMap; /** * This is an implementation of the Single Source Shortest Paths algorithm, using a gather-sum-apply iteration */ -public class GSASingleSourceShortestPaths implements ProgramDescription { +public class GSASingleSourceShortestPaths implements GraphAlgorithm { - // -------------------------------------------------------------------------------------------- - // Program - // -------------------------------------------------------------------------------------------- - - public static void main(String[] args) throws Exception { - - if(!parseParameters(args)) { - return; - } + private final K srcVertexId; + private final Integer maxIterations; - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> edges = getEdgeDataSet(env); + public GSASingleSourceShortestPaths(K srcVertexId, Integer maxIterations) { + this.srcVertexId = srcVertexId; + this.maxIterations = maxIterations; + } - Graph graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env); + @Override + public Graph run(Graph input) { - // Execute the GSA iteration - Graph result = graph + return input.mapVertices(new InitVerticesMapper(srcVertexId)) .runGatherSumApplyIteration(new CalculateDistances(), new ChooseMinDistance(), - new UpdateDistance(), maxIterations); - - // Extract the vertices as the result - DataSet> singleSourceShortestPaths = result.getVertices(); - - // emit result - if(fileOutput) { - singleSourceShortestPaths.writeAsCsv(outputPath, "\n", " "); - - // since file sinks are lazy, we trigger the execution explicitly - env.execute("GSA Single Source Shortest Paths"); - } else { - singleSourceShortestPaths.print(); - } - + new UpdateDistance(), maxIterations); } @SuppressWarnings("serial") - private static final class InitVertices implements MapFunction{ + public static final class InitVerticesMapper implements MapFunction, Double> { - private long srcId; + private K srcVertexId; - public InitVertices(long srcId) { - this.srcId = srcId; + public InitVerticesMapper(K srcId) { + this.srcVertexId = srcId; } - public Double map(Long id) { - if (id.equals(srcId)) { + public Double map(Vertex value) { + if (value.f0.equals(srcVertexId)) { return 0.0; - } - else { - return Double.POSITIVE_INFINITY; + } else { + return Double.MAX_VALUE; } } } @@ -113,7 +87,7 @@ public Double sum(Double newValue, Double currentValue) { }; @SuppressWarnings("serial") - private static final class UpdateDistance extends ApplyFunction { + private static final class UpdateDistance extends ApplyFunction { public void apply(Double newDistance, Double oldDistance) { if (newDistance < oldDistance) { @@ -121,60 +95,4 @@ public void apply(Double newDistance, Double oldDistance) { } } } - - // -------------------------------------------------------------------------------------------- - // Util methods - // -------------------------------------------------------------------------------------------- - - private static boolean fileOutput = false; - - private static Long srcVertexId = 1l; - - private static String edgesInputPath = null; - - private static String outputPath = null; - - private static int maxIterations = 5; - - private static boolean parseParameters(String[] args) { - - if (args.length > 0) { - if(args.length != 4) { - System.err.println("Usage: GSASingleSourceShortestPaths " + - " "); - return false; - } - - fileOutput = true; - srcVertexId = Long.parseLong(args[0]); - edgesInputPath = args[1]; - outputPath = args[2]; - maxIterations = Integer.parseInt(args[3]); - } else { - System.out.println("Executing GSASingle Source Shortest Paths example " - + "with default parameters and built-in default data."); - System.out.println(" Provide parameters to read input data from files."); - System.out.println(" See the documentation for the correct format of input files."); - System.out.println("Usage: GSASingleSourceShortestPaths " + - " "); - } - return true; - } - - private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { - if (fileOutput) { - return env.readCsvFile(edgesInputPath) - .fieldDelimiter("\t") - .lineDelimiter("\n") - .types(Long.class, Long.class, Double.class) - .map(new Tuple3ToEdgeMap()); - } else { - return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env); - } - } - - @Override - public String getDescription() { - return "GSA Single Source Shortest Paths"; - } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagationAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java similarity index 94% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagationAlgorithm.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java index 0b0f4fc81fda4..16489228497db 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagationAlgorithm.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java @@ -26,7 +26,6 @@ import org.apache.flink.graph.spargel.VertexUpdateFunction; import org.apache.flink.types.NullValue; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; @@ -43,12 +42,11 @@ */ @SuppressWarnings("serial") -public class LabelPropagationAlgorithm & Serializable> - implements GraphAlgorithm { +public class LabelPropagation> implements GraphAlgorithm { private final int maxIterations; - public LabelPropagationAlgorithm(int maxIterations) { + public LabelPropagation(int maxIterations) { this.maxIterations = maxIterations; } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRankAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java similarity index 90% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRankAlgorithm.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java index f63fb0dd2c8b8..03cb740ae5c85 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRankAlgorithm.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java @@ -26,16 +26,15 @@ import org.apache.flink.graph.spargel.MessagingFunction; import org.apache.flink.graph.spargel.VertexUpdateFunction; -import java.io.Serializable; - - -public class PageRankAlgorithm & Serializable> implements - GraphAlgorithm { +/** + * This is an implementation of a simple PageRank algorithm, using a vertex-centric iteration. + */ +public class PageRank implements GraphAlgorithm { private double beta; private int maxIterations; - public PageRankAlgorithm(double beta, int maxIterations) { + public PageRank(double beta, int maxIterations) { this.beta = beta; this.maxIterations = maxIterations; } @@ -44,8 +43,9 @@ public PageRankAlgorithm(double beta, int maxIterations) { public Graph run(Graph network) throws Exception { final long numberOfVertices = network.numberOfVertices(); - return network.runVertexCentricIteration(new VertexRankUpdater(beta, numberOfVertices), new RankMessenger(numberOfVertices), - maxIterations); + + return network.runVertexCentricIteration(new VertexRankUpdater(beta, numberOfVertices), + new RankMessenger(numberOfVertices), maxIterations); } /** diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPathsAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java similarity index 91% rename from flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPathsAlgorithm.java rename to flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java index e78ae3ec63fde..97ea0009d923a 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPathsAlgorithm.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java @@ -27,16 +27,16 @@ import org.apache.flink.graph.spargel.MessagingFunction; import org.apache.flink.graph.spargel.VertexUpdateFunction; -import java.io.Serializable; - +/** + * This is an implementation of the Single-Source-Shortest Paths algorithm, using a vertex-centric iteration. + */ @SuppressWarnings("serial") -public class SingleSourceShortestPathsAlgorithm & Serializable> - implements GraphAlgorithm { +public class SingleSourceShortestPaths implements GraphAlgorithm { private final K srcVertexId; private final Integer maxIterations; - public SingleSourceShortestPathsAlgorithm(K srcVertexId, Integer maxIterations) { + public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) { this.srcVertexId = srcVertexId; this.maxIterations = maxIterations; } From 8f35988fc3edfeda8044a6675d04f111596b8e31 Mon Sep 17 00:00:00 2001 From: vasia Date: Fri, 7 Aug 2015 11:28:20 +0200 Subject: [PATCH 160/175] [FLINK-2451] [gelly] re-organized tests; compare with collect() instead of temp files where possible This closes #1000 --- .../example/GSASingleSourceShortestPaths.java | 2 +- .../example/utils/CommunityDetectionData.java | 30 ++++ .../example/utils/LabelPropagationData.java | 110 ++++++++++++++ .../graph/example/utils/PageRankData.java | 33 +++- .../graph/test/GatherSumApplyITCase.java | 111 +++++++------- .../example/CommunityDetectionITCase.java | 100 ------------ .../test/example/LabelPropagationITCase.java | 143 ------------------ .../graph/test/example/PageRankITCase.java | 78 ---------- .../SingleSourceShortestPathsITCase.java | 9 ++ .../library/CommunityDetectionITCase.java | 82 ++++++++++ ...edComponentsWithRandomisedEdgesITCase.java | 6 +- .../test/library/LabelPropagationITCase.java | 78 ++++++++++ .../graph/test/library/PageRankITCase.java | 125 +++++++++++++++ 13 files changed, 519 insertions(+), 388 deletions(-) create mode 100644 flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java delete mode 100644 flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/CommunityDetectionITCase.java delete mode 100644 flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationITCase.java delete mode 100644 flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java create mode 100644 flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java rename flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/{example => library}/ConnectedComponentsWithRandomisedEdgesITCase.java (94%) create mode 100644 flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java create mode 100644 flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java index 23a3a82f25757..9ea8fe2fe28cb 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java @@ -74,7 +74,7 @@ public static void main(String[] args) throws Exception { // emit result if(fileOutput) { - singleSourceShortestPaths.writeAsCsv(outputPath, "\n", " "); + singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ","); // since file sinks are lazy, we trigger the execution explicitly env.execute("GSA Single Source Shortest Paths"); diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java index 629f5ef890801..196de3af2a569 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java @@ -36,6 +36,11 @@ public class CommunityDetectionData { public static final double DELTA = 0.5f; + public static final String COMMUNITIES_SINGLE_ITERATION = "1,5\n" + "2,6\n" + + "3,1\n" + "4,1\n" + "5,1\n" + "6,8\n" + "7,8\n" + "8,7"; + + public static final String COMMUNITIES_WITH_TIE = "1,2\n" + "2,1\n" + "3,1\n" + "4,1\n" + "5,1"; + public static DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { List> edges = new ArrayList>(); @@ -61,5 +66,30 @@ public static DataSet> getDefaultEdgeDataSet(ExecutionEnviron return env.fromCollection(edges); } + public static DataSet> getSimpleEdgeDataSet(ExecutionEnvironment env) { + + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 1.0)); + edges.add(new Edge(1L, 3L, 2.0)); + edges.add(new Edge(1L, 4L, 3.0)); + edges.add(new Edge(1L, 5L, 4.0)); + edges.add(new Edge(2L, 6L, 5.0)); + edges.add(new Edge(6L, 7L, 6.0)); + edges.add(new Edge(6L, 8L, 7.0)); + edges.add(new Edge(7L, 8L, 8.0)); + + return env.fromCollection(edges); + } + private CommunityDetectionData() {} + + public static DataSet> getTieEdgeDataSet(ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 1.0)); + edges.add(new Edge(1L, 3L, 1.0)); + edges.add(new Edge(1L, 4L, 1.0)); + edges.add(new Edge(1L, 5L, 1.0)); + + return env.fromCollection(edges); + } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java new file mode 100644 index 0000000000000..b70a9c4e0e569 --- /dev/null +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java @@ -0,0 +1,110 @@ +/* + * 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.graph.example.utils; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Vertex; +import org.apache.flink.types.NullValue; + +public class LabelPropagationData { + + public static final String LABELS_AFTER_1_ITERATION = "1,10\n" + + "2,10\n" + + "3,10\n" + + "4,40\n" + + "5,40\n" + + "6,40\n" + + "7,40\n"; + + public static final String LABELS_WITH_TIE ="1,10\n" + + "2,10\n" + + "3,10\n" + + "4,10\n" + + "5,20\n" + + "6,20\n" + + "7,20\n" + + "8,20\n" + + "9,20\n"; + + private LabelPropagationData() {} + + public static final DataSet> getDefaultVertexSet(ExecutionEnvironment env) { + + List> vertices = new ArrayList>(); + vertices.add(new Vertex(1l, 10l)); + vertices.add(new Vertex(2l, 10l)); + vertices.add(new Vertex(3l, 30l)); + vertices.add(new Vertex(4l, 40l)); + vertices.add(new Vertex(5l, 40l)); + vertices.add(new Vertex(6l, 40l)); + vertices.add(new Vertex(7l, 40l)); + + return env.fromCollection(vertices); + } + + public static final DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { + + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 3L, NullValue.getInstance())); + edges.add(new Edge(2L, 3L, NullValue.getInstance())); + edges.add(new Edge(4L, 7L, NullValue.getInstance())); + edges.add(new Edge(5L, 7L, NullValue.getInstance())); + edges.add(new Edge(6L, 7L, NullValue.getInstance())); + edges.add(new Edge(7L, 3L, NullValue.getInstance())); + + return env.fromCollection(edges); + } + + public static final DataSet> getTieVertexSet(ExecutionEnvironment env) { + + List> vertices = new ArrayList>(); + vertices.add(new Vertex(1l, 10l)); + vertices.add(new Vertex(2l, 10l)); + vertices.add(new Vertex(3l, 10l)); + vertices.add(new Vertex(4l, 10l)); + vertices.add(new Vertex(5l, 0l)); + vertices.add(new Vertex(6l, 20l)); + vertices.add(new Vertex(7l, 20l)); + vertices.add(new Vertex(8l, 20l)); + vertices.add(new Vertex(9l, 20l)); + + return env.fromCollection(vertices); + } + + public static final DataSet> getTieEdgeDataSet(ExecutionEnvironment env) { + + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 5L, NullValue.getInstance())); + edges.add(new Edge(2L, 5L, NullValue.getInstance())); + edges.add(new Edge(4L, 5L, NullValue.getInstance())); + edges.add(new Edge(5L, 5L, NullValue.getInstance())); + edges.add(new Edge(6L, 5L, NullValue.getInstance())); + edges.add(new Edge(7L, 5L, NullValue.getInstance())); + edges.add(new Edge(8L, 5L, NullValue.getInstance())); + edges.add(new Edge(9L, 5L, NullValue.getInstance())); + + return env.fromCollection(edges); + } +} + diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java index 077572e612916..c84808afa0508 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java @@ -18,6 +18,13 @@ package org.apache.flink.graph.example.utils; +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Edge; + public class PageRankData { public static final String EDGES = "2 1\n" + @@ -31,12 +38,28 @@ public class PageRankData { "3 5\n"; - public static final String RANKS_AFTER_3_ITERATIONS = "1 0.237\n" + - "2 0.248\n" + - "3 0.173\n" + - "4 0.175\n" + - "5 0.165\n"; + public static final String RANKS_AFTER_3_ITERATIONS = "1,0.237\n" + + "2,0.248\n" + + "3,0.173\n" + + "4,0.175\n" + + "5,0.165\n"; private PageRankData() {} + + public static final DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { + + List> edges = new ArrayList>(); + edges.add(new Edge(2L, 1L, 1.0)); + edges.add(new Edge(5L, 2L, 1.0)); + edges.add(new Edge(5L, 4L, 1.0)); + edges.add(new Edge(4L, 3L, 1.0)); + edges.add(new Edge(4L, 2L, 1.0)); + edges.add(new Edge(1L, 4L, 1.0)); + edges.add(new Edge(1L, 2L, 1.0)); + edges.add(new Edge(1L, 3L, 1.0)); + edges.add(new Edge(3L, 5L, 1.0)); + + return env.fromCollection(edges); + } } diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java index a883fa0e1d2ec..4b381b6a0f679 100755 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java @@ -18,20 +18,21 @@ package org.apache.flink.graph.test; -import com.google.common.base.Charsets; -import com.google.common.io.Files; -import org.apache.flink.graph.example.GSAConnectedComponents; -import org.apache.flink.graph.example.GSASingleSourceShortestPaths; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData; +import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData; +import org.apache.flink.graph.library.GSAConnectedComponents; +import org.apache.flink.graph.library.GSASingleSourceShortestPaths; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; +import org.apache.flink.types.NullValue; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; +import java.util.List; @RunWith(Parameterized.class) public class GatherSumApplyITCase extends MultipleProgramsTestBase { @@ -40,44 +41,29 @@ public GatherSumApplyITCase(TestExecutionMode mode){ super(mode); } - private String edgesPath; - private String resultPath; private String expectedResult; - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - - File edgesFile = tempFolder.newFile(); - Files.write(GatherSumApplyITCase.EDGES, edgesFile, Charsets.UTF_8); - - edgesPath = edgesFile.toURI().toString(); - - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expectedResult, resultPath); - } - // -------------------------------------------------------------------------------------------- // Connected Components Test // -------------------------------------------------------------------------------------------- @Test public void testConnectedComponents() throws Exception { - GSAConnectedComponents.main(new String[]{edgesPath, resultPath, "16"}); - expectedResult = "1 1\n" + - "2 1\n" + - "3 1\n" + - "4 1\n" + - "5 1\n" + - "6 6\n" + - "7 6\n"; + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env), + new InitMapperCC(), env); + + List> result = inputGraph.run(new GSAConnectedComponents(16)) + .getVertices().collect(); + expectedResult = "1,1\n" + + "2,1\n" + + "3,1\n" + + "4,1\n"; + + compareResultAsTuples(result, expectedResult); } // -------------------------------------------------------------------------------------------- @@ -86,26 +72,35 @@ public void testConnectedComponents() throws Exception { @Test public void testSingleSourceShortestPaths() throws Exception { - GSASingleSourceShortestPaths.main(new String[]{"1", edgesPath, resultPath, "16"}); - expectedResult = "1 0.0\n" + - "2 12.0\n" + - "3 13.0\n" + - "4 47.0\n" + - "5 48.0\n" + - "6 Infinity\n" + - "7 Infinity\n"; + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + SingleSourceShortestPathsData.getDefaultEdgeDataSet(env), + new InitMapperSSSP(), env); + + List> result = inputGraph.run(new GSASingleSourceShortestPaths(1l, 16)) + .getVertices().collect(); + + expectedResult = "1,0.0\n" + + "2,12.0\n" + + "3,13.0\n" + + "4,47.0\n" + + "5,48.0\n"; + + compareResultAsTuples(result, expectedResult); } - // -------------------------------------------------------------------------------------------- - // Sample data - // -------------------------------------------------------------------------------------------- + @SuppressWarnings("serial") + private static final class InitMapperCC implements MapFunction { + public Long map(Long value) { + return value; + } + } - private static final String EDGES = "1 2 12.0\n" + - "1 3 13.0\n" + - "2 3 23.0\n" + - "3 4 34.0\n" + - "3 5 35.0\n" + - "4 5 45.0\n" + - "5 1 51.0\n" + - "6 7 67.0\n"; -} + @SuppressWarnings("serial") + private static final class InitMapperSSSP implements MapFunction { + public Double map(Long value) { + return 0.0; + } + } +} \ No newline at end of file diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/CommunityDetectionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/CommunityDetectionITCase.java deleted file mode 100644 index 1302424b595b7..0000000000000 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/CommunityDetectionITCase.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.graph.test.example; - -import com.google.common.base.Charsets; -import com.google.common.io.Files; -import org.apache.flink.graph.example.CommunityDetection; -import org.apache.flink.graph.example.utils.CommunityDetectionData; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; - -@RunWith(Parameterized.class) -public class CommunityDetectionITCase extends MultipleProgramsTestBase { - - private String edgesPath; - - private String resultPath; - - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - public CommunityDetectionITCase(TestExecutionMode mode) { - super(mode); - } - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - - @Test - public void testSingleIteration() throws Exception { - /* - * Test one iteration of the Simple Community Detection Example - */ - final String edges = "1 2 1.0\n" + "1 3 2.0\n" + "1 4 3.0\n" + "1 5 4.0\n" + "2 6 5.0\n" + - "6 7 6.0\n" + "6 8 7.0\n" + "7 8 8.0"; - edgesPath = createTempFile(edges); - - CommunityDetection.main(new String[]{edgesPath, resultPath, "1", - CommunityDetectionData.DELTA + ""}); - - expected = "1,5\n" + "2,6\n" + "3,1\n" + "4,1\n" + "5,1\n" + "6,8\n" + "7,8\n" + "8,7"; - } - - @Test - public void testTieBreaker() throws Exception { - /* - * Test one iteration of the Simple Community Detection Example where a tie must be broken - */ - - final String edges = "1 2 1.0\n" + "1 3 1.0\n" + "1 4 1.0\n" + "1 5 1.0"; - edgesPath = createTempFile(edges); - - CommunityDetection.main(new String[]{edgesPath, resultPath, "1", - CommunityDetectionData.DELTA + ""}); - - expected = "1,2\n" + "2,1\n" + "3,1\n" + "4,1\n" + "5,1"; - } - - - // ------------------------------------------------------------------------- - // Util methods - // ------------------------------------------------------------------------- - private String createTempFile(final String rows) throws Exception { - File tempFile = tempFolder.newFile(); - Files.write(rows, tempFile, Charsets.UTF_8); - return tempFile.toURI().toString(); - } -} diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationITCase.java deleted file mode 100644 index 858d06c37cb47..0000000000000 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationITCase.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.graph.test.example; - -import com.google.common.base.Charsets; -import com.google.common.io.Files; -import org.apache.flink.graph.example.LabelPropagation; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; - -@RunWith(Parameterized.class) -public class LabelPropagationITCase extends MultipleProgramsTestBase { - - public LabelPropagationITCase(TestExecutionMode mode){ - super(mode); - } - - private String resultPath; - private String expectedResult; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expectedResult, resultPath); - } - - @Test - public void testSingleIteration() throws Exception { - /* - * Test one iteration of label propagation example with a simple graph - */ - - final String vertices = "1 10\n" + - "2 10\n" + - "3 30\n" + - "4 40\n" + - "5 40\n" + - "6 40\n" + - "7 70\n"; - - final String edges = "1 3\n" + - "2 3\n" + - "4 7\n" + - "5 7\n" + - "6 7\n" + - "7 3\n"; - - String verticesPath = createTempFile(vertices); - String edgesPath = createTempFile(edges); - - LabelPropagation.main(new String[]{verticesPath, edgesPath, resultPath, "1"}); - - expectedResult = "1,10\n" + - "2,10\n" + - "3,10\n" + - "4,40\n" + - "5,40\n" + - "6,40\n" + - "7,40\n"; - } - - @Test - public void testTieBreaker() throws Exception { - /* - * Test the label propagation example where a tie must be broken - */ - - final String vertices = "1 10\n" + - "2 10\n" + - "3 10\n" + - "4 10\n" + - "5 0\n" + - "6 20\n" + - "7 20\n" + - "8 20\n" + - "9 20\n"; - - final String edges = "1 5\n" + - "2 5\n" + - "3 5\n" + - "4 5\n" + - "6 5\n" + - "7 5\n" + - "8 5\n" + - "9 5\n"; - - String verticesPath = createTempFile(vertices); - String edgesPath = createTempFile(edges); - - LabelPropagation.main(new String[]{verticesPath, edgesPath, resultPath, "1"}); - - expectedResult = "1,10\n" + - "2,10\n" + - "3,10\n" + - "4,10\n" + - "5,20\n" + - "6,20\n" + - "7,20\n" + - "8,20\n" + - "9,20\n"; - } - - // ------------------------------------------------------------------------- - // Util methods - // ------------------------------------------------------------------------- - - private String createTempFile(final String rows) throws Exception { - File tempFile = tempFolder.newFile(); - Files.write(rows, tempFile, Charsets.UTF_8); - return tempFile.toURI().toString(); - } -} diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java deleted file mode 100644 index cde959f7f694d..0000000000000 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/PageRankITCase.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.graph.test.example; - -import java.io.File; - -import com.google.common.base.Charsets; -import com.google.common.io.Files; - -import org.apache.flink.graph.example.GSAPageRank; -import org.apache.flink.graph.example.PageRank; -import org.apache.flink.graph.example.utils.PageRankData; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class PageRankITCase extends MultipleProgramsTestBase { - - public PageRankITCase(TestExecutionMode mode){ - super(mode); - } - - private String edgesPath; - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - - File edgesFile = tempFolder.newFile(); - Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8); - - edgesPath = edgesFile.toURI().toString(); - } - - @After - public void after() throws Exception{ - compareKeyValuePairsWithDelta(expected, resultPath, "\t", 0.01); - } - - @Test - public void testPageRankWithThreeIterations() throws Exception { - PageRank.main(new String[] {edgesPath, resultPath, "3"}); - expected = PageRankData.RANKS_AFTER_3_ITERATIONS; - } - - @Test - public void testGSAPageRankWithThreeIterations() throws Exception { - GSAPageRank.main(new String[] {edgesPath, resultPath, "3"}); - expected = PageRankData.RANKS_AFTER_3_ITERATIONS; - } -} diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java index 2e68b0a489a90..d8f8c8ff67d0c 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java @@ -20,6 +20,8 @@ import com.google.common.base.Charsets; import com.google.common.io.Files; + +import org.apache.flink.graph.example.GSASingleSourceShortestPaths; import org.apache.flink.graph.example.SingleSourceShortestPaths; import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData; import org.apache.flink.test.util.MultipleProgramsTestBase; @@ -65,6 +67,13 @@ public void testSSSPExample() throws Exception { expected = SingleSourceShortestPathsData.RESULTED_SINGLE_SOURCE_SHORTEST_PATHS; } + @Test + public void testGSASSSPExample() throws Exception { + GSASingleSourceShortestPaths.main(new String[]{SingleSourceShortestPathsData.SRC_VERTEX_ID + "", + edgesPath, resultPath, 10 + ""}); + expected = SingleSourceShortestPathsData.RESULTED_SINGLE_SOURCE_SHORTEST_PATHS; + } + @After public void after() throws Exception { compareResultsByLinesInMemory(expected, resultPath); diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java new file mode 100644 index 0000000000000..104996ef17ff9 --- /dev/null +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java @@ -0,0 +1,82 @@ +/* + * 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.graph.test.library; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.example.utils.CommunityDetectionData; +import org.apache.flink.graph.library.CommunityDetection; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; + +@RunWith(Parameterized.class) +public class CommunityDetectionITCase extends MultipleProgramsTestBase { + + public CommunityDetectionITCase(TestExecutionMode mode) { + super(mode); + } + + private String expected; + + @Test + public void testSingleIteration() throws Exception { + /* + * Test one iteration of the Simple Community Detection Example + */ + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph inputGraph = Graph.fromDataSet( + CommunityDetectionData.getSimpleEdgeDataSet(env), new InitLabels(), env); + + List> result = inputGraph.run(new CommunityDetection(1, CommunityDetectionData.DELTA)) + .getVertices().collect(); + + expected = CommunityDetectionData.COMMUNITIES_SINGLE_ITERATION; + compareResultAsTuples(result, expected); + } + + @Test + public void testTieBreaker() throws Exception { + /* + * Test one iteration of the Simple Community Detection Example where a tie must be broken + */ + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph inputGraph = Graph.fromDataSet( + CommunityDetectionData.getTieEdgeDataSet(env), new InitLabels(), env); + + List> result = inputGraph.run(new CommunityDetection(1, CommunityDetectionData.DELTA)) + .getVertices().collect(); + expected = CommunityDetectionData.COMMUNITIES_WITH_TIE; + compareResultAsTuples(result, expected); + } + + @SuppressWarnings("serial") + private static final class InitLabels implements MapFunction{ + + public Long map(Long id) { + return id; + } + } +} diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsWithRandomisedEdgesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java similarity index 94% rename from flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsWithRandomisedEdgesITCase.java rename to flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java index f2f3d8c68edc9..ef4b4670c0a59 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsWithRandomisedEdgesITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.graph.test.example; +package org.apache.flink.graph.test.library; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; @@ -25,7 +25,7 @@ import org.apache.flink.graph.Edge; import org.apache.flink.graph.Graph; import org.apache.flink.graph.Vertex; -import org.apache.flink.graph.library.ConnectedComponentsAlgorithm; +import org.apache.flink.graph.library.ConnectedComponents; import org.apache.flink.test.testdata.ConnectedComponentsData; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.types.NullValue; @@ -61,7 +61,7 @@ protected void testProgram() throws Exception { Graph graph = Graph.fromDataSet(initialVertices, edges, env); DataSet> result = graph - .run(new ConnectedComponentsAlgorithm(100)).getVertices(); + .run(new ConnectedComponents(100)).getVertices(); result.writeAsCsv(resultPath, "\n", " "); env.execute(); diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java new file mode 100644 index 0000000000000..da36ef6c0b9ab --- /dev/null +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java @@ -0,0 +1,78 @@ +/* + * 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.graph.test.library; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.example.utils.LabelPropagationData; +import org.apache.flink.graph.library.LabelPropagation; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.apache.flink.types.NullValue; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; + +@RunWith(Parameterized.class) +public class LabelPropagationITCase extends MultipleProgramsTestBase { + + public LabelPropagationITCase(TestExecutionMode mode){ + super(mode); + } + + private String expectedResult; + + @Test + public void testSingleIteration() throws Exception { + /* + * Test one iteration of label propagation example with a simple graph + */ + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + LabelPropagationData.getDefaultVertexSet(env), + LabelPropagationData.getDefaultEdgeDataSet(env), env); + + List> result = inputGraph.run(new LabelPropagation(1)) + .getVertices().collect(); + + expectedResult = LabelPropagationData.LABELS_AFTER_1_ITERATION; + compareResultAsTuples(result, expectedResult); + } + + @Test + public void testTieBreaker() throws Exception { + /* + * Test the label propagation example where a tie must be broken + */ + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + LabelPropagationData.getTieVertexSet(env), + LabelPropagationData.getTieEdgeDataSet(env), env); + + List> result = inputGraph.run(new LabelPropagation(1)) + .getVertices().collect(); + + expectedResult = LabelPropagationData.LABELS_WITH_TIE; + compareResultAsTuples(result, expectedResult); + } +} diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java new file mode 100644 index 0000000000000..cc1132d436ff7 --- /dev/null +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java @@ -0,0 +1,125 @@ +/* + * 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.graph.test.library; + +import java.util.Arrays; +import java.util.List; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.example.utils.PageRankData; +import org.apache.flink.graph.library.GSAPageRank; +import org.apache.flink.graph.library.PageRank; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class PageRankITCase extends MultipleProgramsTestBase { + + public PageRankITCase(TestExecutionMode mode){ + super(mode); + } + + private String expectedResult; + + @Test + public void testPageRankWithThreeIterations() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env); + + DataSet> vertexOutDegrees = inputGraph.outDegrees(); + + Graph networkWithWeights = inputGraph + .joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper()); + + List> result = networkWithWeights.run(new PageRank(0.85, 3)) + .getVertices().collect(); + + compareWithDelta(result, expectedResult, 0.01); + } + + @Test + public void testGSAPageRankWithThreeIterations() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env); + + DataSet> vertexOutDegrees = inputGraph.outDegrees(); + + Graph networkWithWeights = inputGraph + .joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper()); + + List> result = networkWithWeights.run(new GSAPageRank(0.85, 3)) + .getVertices().collect(); + + compareWithDelta(result, expectedResult, 0.01); + } + + private void compareWithDelta(List> result, + String expectedResult, double delta) { + + String resultString = ""; + for (Vertex v : result) { + resultString += v.f0.toString() + "," + v.f1.toString() +"\n"; + } + + expectedResult = PageRankData.RANKS_AFTER_3_ITERATIONS; + String[] expected = expectedResult.isEmpty() ? new String[0] : expectedResult.split("\n"); + + String[] resultArray = resultString.isEmpty() ? new String[0] : resultString.split("\n"); + + Arrays.sort(expected); + Arrays.sort(resultArray); + + for (int i = 0; i < expected.length; i++) { + String[] expectedFields = expected[i].split(","); + String[] resultFields = resultArray[i].split(","); + + double expectedPayLoad = Double.parseDouble(expectedFields[1]); + double resultPayLoad = Double.parseDouble(resultFields[1]); + + Assert.assertTrue("Values differ by more than the permissible delta", + Math.abs(expectedPayLoad - resultPayLoad) < delta); + } + } + + @SuppressWarnings("serial") + private static final class InitMapper implements MapFunction { + public Double map(Long value) { + return 1.0; + } + } + + @SuppressWarnings("serial") + private static final class InitWeightsMapper implements MapFunction, Double> { + public Double map(Tuple2 value) { + return value.f0 / value.f1; + } + } +} From 5ae84273ce60899ba118b8d21b40587a71515f9b Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 8 Aug 2015 13:36:47 +0200 Subject: [PATCH 161/175] [gelly] made the number of vertices an optional parameter of PageRank; added the edge weight initialization to the library methods --- .../java/org/apache/flink/graph/Graph.java | 1 - .../flink/graph/library/GSAPageRank.java | 37 ++++++++++++++- .../apache/flink/graph/library/PageRank.java | 44 ++++++++++++++++- .../graph/test/library/PageRankITCase.java | 47 +++++++++++-------- 4 files changed, 104 insertions(+), 25 deletions(-) diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java index 8552c01a31b54..ff0ec245c5f21 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java @@ -1145,7 +1145,6 @@ public Graph addVertices(List> verticesToAdd) { * @return the new graph containing the existing vertices and edges plus the * newly added edge */ - @SuppressWarnings("unchecked") public Graph addEdge(Vertex source, Vertex target, EV edgeValue) { Graph partialGraph = fromCollection(Arrays.asList(source, target), Arrays.asList(new Edge(source.f0, target.f0, edgeValue)), diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java index 4299381019b6c..4adaaa9604bd9 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java @@ -18,6 +18,9 @@ package org.apache.flink.graph.library; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.graph.Graph; import org.apache.flink.graph.GraphAlgorithm; import org.apache.flink.graph.gsa.ApplyFunction; @@ -27,23 +30,46 @@ /** * This is an implementation of a simple PageRank algorithm, using a gather-sum-apply iteration. + * The user can define the damping factor and the maximum number of iterations. + * If the number of vertices of the input graph is known, it should be provided as a parameter + * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices. + * + * The implementation assumes that each page has at least one incoming and one outgoing link. */ public class GSAPageRank implements GraphAlgorithm { private double beta; private int maxIterations; + private long numberOfVertices; + /** + * @param beta the damping factor + * @param maxIterations the maximum number of iterations + */ public GSAPageRank(double beta, int maxIterations) { this.beta = beta; this.maxIterations = maxIterations; } + public GSAPageRank(double beta, long numVertices, int maxIterations) { + this.beta = beta; + this.numberOfVertices = numVertices; + this.maxIterations = maxIterations; + } + @Override public Graph run(Graph network) throws Exception { - final long numberOfVertices = network.numberOfVertices(); + if (numberOfVertices == 0) { + numberOfVertices = network.numberOfVertices(); + } + + DataSet> vertexOutDegrees = network.outDegrees(); - return network.runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(), + Graph networkWithWeights = network + .joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper()); + + return networkWithWeights.runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(), new UpdateRanks(beta, numberOfVertices), maxIterations); } @@ -97,4 +123,11 @@ public void apply(Double rankSum, Double currentValue) { setResult((1-beta)/numVertices + beta * rankSum); } } + + @SuppressWarnings("serial") + private static final class InitWeightsMapper implements MapFunction, Double> { + public Double map(Tuple2 value) { + return value.f0 / value.f1; + } + } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java index 03cb740ae5c85..93b10eb67e909 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java @@ -18,6 +18,9 @@ package org.apache.flink.graph.library; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.graph.Edge; import org.apache.flink.graph.Graph; import org.apache.flink.graph.GraphAlgorithm; @@ -28,23 +31,52 @@ /** * This is an implementation of a simple PageRank algorithm, using a vertex-centric iteration. + * The user can define the damping factor and the maximum number of iterations. + * If the number of vertices of the input graph is known, it should be provided as a parameter + * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices. + * + * The implementation assumes that each page has at least one incoming and one outgoing link. */ public class PageRank implements GraphAlgorithm { private double beta; private int maxIterations; + private long numberOfVertices; + /** + * @param beta the damping factor + * @param maxIterations the maximum number of iterations + */ public PageRank(double beta, int maxIterations) { this.beta = beta; this.maxIterations = maxIterations; + this.numberOfVertices = 0; + } + + /** + * @param beta the damping factor + * @param maxIterations the maximum number of iterations + * @param numVertices the number of vertices in the input + */ + public PageRank(double beta, long numVertices, int maxIterations) { + this.beta = beta; + this.maxIterations = maxIterations; + this.numberOfVertices = numVertices; } @Override public Graph run(Graph network) throws Exception { - final long numberOfVertices = network.numberOfVertices(); + if (numberOfVertices == 0) { + numberOfVertices = network.numberOfVertices(); + } + + DataSet> vertexOutDegrees = network.outDegrees(); - return network.runVertexCentricIteration(new VertexRankUpdater(beta, numberOfVertices), + Graph networkWithWeights = network + .joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper()); + + return networkWithWeights.runVertexCentricIteration(new VertexRankUpdater(beta, numberOfVertices), new RankMessenger(numberOfVertices), maxIterations); } @@ -102,4 +134,12 @@ public void sendMessages(Vertex vertex) { } } } + + @SuppressWarnings("serial") + private static final class InitWeightsMapper implements MapFunction, Double> { + public Double map(Tuple2 value) { + return value.f0 / value.f1; + } + } + } diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java index cc1132d436ff7..cc0327f38c502 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java @@ -22,9 +22,7 @@ import java.util.List; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.graph.Graph; import org.apache.flink.graph.Vertex; import org.apache.flink.graph.example.utils.PageRankData; @@ -51,13 +49,8 @@ public void testPageRankWithThreeIterations() throws Exception { Graph inputGraph = Graph.fromDataSet( PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env); - - DataSet> vertexOutDegrees = inputGraph.outDegrees(); - Graph networkWithWeights = inputGraph - .joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper()); - - List> result = networkWithWeights.run(new PageRank(0.85, 3)) + List> result = inputGraph.run(new PageRank(0.85, 3)) .getVertices().collect(); compareWithDelta(result, expectedResult, 0.01); @@ -69,13 +62,34 @@ public void testGSAPageRankWithThreeIterations() throws Exception { Graph inputGraph = Graph.fromDataSet( PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env); - - DataSet> vertexOutDegrees = inputGraph.outDegrees(); - Graph networkWithWeights = inputGraph - .joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper()); + List> result = inputGraph.run(new GSAPageRank(0.85, 3)) + .getVertices().collect(); + + compareWithDelta(result, expectedResult, 0.01); + } + + @Test + public void testPageRankWithThreeIterationsAndNumOfVertices() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph inputGraph = Graph.fromDataSet( + PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env); + + List> result = inputGraph.run(new PageRank(0.85, 5, 3)) + .getVertices().collect(); + + compareWithDelta(result, expectedResult, 0.01); + } + + @Test + public void testGSAPageRankWithThreeIterationsAndNumOfVertices() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - List> result = networkWithWeights.run(new GSAPageRank(0.85, 3)) + Graph inputGraph = Graph.fromDataSet( + PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env); + + List> result = inputGraph.run(new GSAPageRank(0.85, 5, 3)) .getVertices().collect(); compareWithDelta(result, expectedResult, 0.01); @@ -115,11 +129,4 @@ public Double map(Long value) { return 1.0; } } - - @SuppressWarnings("serial") - private static final class InitWeightsMapper implements MapFunction, Double> { - public Double map(Tuple2 value) { - return value.f0 / value.f1; - } - } } From 9c7f769388d90c3a79d8c08995d4eae892b23a6e Mon Sep 17 00:00:00 2001 From: mjsax Date: Sat, 22 Aug 2015 01:55:40 +0200 Subject: [PATCH 162/175] [FLINK-2560] Flink-Avro Plugin cannot be handled by Eclipse - disabled avro-maven-plugin within Eclipse via ... This closes #1041 --- flink-staging/flink-avro/pom.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/flink-staging/flink-avro/pom.xml b/flink-staging/flink-avro/pom.xml index 71a99d717d1c1..6b7e330aa3db4 100644 --- a/flink-staging/flink-avro/pom.xml +++ b/flink-staging/flink-avro/pom.xml @@ -181,6 +181,19 @@ under the License. + + + org.apache.avro + avro-maven-plugin + [1.7.7,) + + schema + + + + + + From 5aa70f70846f021b384c005b3dd3fd5ee3424d00 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 24 Aug 2015 18:04:22 +0200 Subject: [PATCH 163/175] [FLINK-2468] [tests] Make sure StateCheckpoinedITCase tolerates errors before first checkpoint --- .../flink/test/checkpointing/StateCheckpoinedITCase.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java index 2c2f2b48166f9..10d4d66536f15 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java @@ -95,8 +95,9 @@ public void testProgram(StreamExecutionEnvironment env) { @Override public void postSubmit() { - assertTrue("Test inconclusive: failure occurred before first checkpoint", - OnceFailingAggregator.wasCheckpointedBeforeFailure); + if (!OnceFailingAggregator.wasCheckpointedBeforeFailure) { + System.err.println("Test inconclusive: failure occurred before first checkpoint"); + } long filterSum = 0; for (long l : StringRichFilterFunction.counts) { From c6391dbfd1eff0ba57da412485d89755e591a601 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 21 Aug 2015 14:06:04 +0200 Subject: [PATCH 164/175] [FLINK-2558] Add Streaming Connector for Elasticsearch --- docs/apis/streaming_guide.md | 178 +++++++++- .../flink-connector-elasticsearch/pom.xml | 81 +++++ .../elasticsearch/ElasticsearchSink.java | 315 ++++++++++++++++++ .../elasticsearch/IndexRequestBuilder.java | 66 ++++ .../examples/ElasticsearchExample.java | 81 +++++ .../ElasticsearchSinkITCase.java | 205 ++++++++++++ .../src/test/resources/log4j-test.properties | 27 ++ .../src/test/resources/logback-test.xml | 30 ++ .../flink-streaming-connectors/pom.xml | 1 + 9 files changed, 981 insertions(+), 3 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties create mode 100644 flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index cd5acc31d179c..5ffe432544f56 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -1517,11 +1517,24 @@ Stream connectors ---------------- -Connectors provide an interface for accessing data from various third party sources (message queues). Currently three connectors are natively supported, namely [Apache Kafka](https://kafka.apache.org/), [RabbitMQ](http://www.rabbitmq.com/) and the [Twitter Streaming API](https://dev.twitter.com/docs/streaming-apis). +Connectors provide code for interfacing with various third-party systems. +Typically the connector packages consist of a source and sink class +(with the exception of Twitter where only a source is provided and Elasticsearch +where only a sink is provided). -Typically the connector packages consist of a source and sink class (with the exception of Twitter where only a source is provided). To use these sources the user needs to pass Serialization/Deserialization schemas for the connectors for the desired types. (Or use some predefined ones) +Currently these systems are supported: -To run an application using one of these connectors, additional third party components are usually required to be installed and launched, e.g. the servers for the message queues. Further instructions for these can be found in the corresponding subsections. [Docker containers](#docker-containers-for-connectors) are also provided encapsulating these services to aid users getting started with connectors. + * [Apache Kafka](https://kafka.apache.org/) (sink/source) + * [Elasticsearch](https://elastic.co/) (sink) + * [RabbitMQ](http://www.rabbitmq.com/) (sink/source) + * [Twitter Streaming API](https://dev.twitter.com/docs/streaming-apis) (source) + +To run an application using one of these connectors, additional third party +components are usually required to be installed and launched, e.g. the servers +for the message queues. Further instructions for these can be found in the +corresponding subsections. [Docker containers](#docker-containers-for-connectors) +are also provided encapsulating these services to aid users getting started +with connectors. ### Apache Kafka @@ -1661,6 +1674,165 @@ More about Kafka can be found [here](https://kafka.apache.org/documentation.html [Back to top](#top) +### Elasticsearch + +This connector provides a Sink that can write to an +[Elasticsearch](https://elastic.co/) Index. To use this connector, add the +following dependency to your project: + +{% highlight xml %} + + org.apache.flink + flink-connector-elasticsearch + {{site.version }} + +{% endhighlight %} + +Note that the streaming connectors are currently not part of the binary +distribution. See +[here](cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution) +for information about how to package the program with the libraries for +cluster execution. + +#### Installing Elasticsearch + +Instructions for setting up an Elasticsearch cluster can be found +[here](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup.html). +Make sure to set and remember a cluster name. This must be set when +creating a Sink for writing to your cluster + +#### Elasticsearch Sink +The connector provides a Sink that can send data to an Elasticsearch Index. + +The sink can use two different methods for communicating with Elasticsearch: + +1. An embedded Node +2. The TransportClient + +See [here](https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/client.html) +for information about the differences between the two modes. + +This code shows how to create a sink that uses an embedded Node for +communication: + +
    +
    +{% highlight java %} +DataStream input = ...; + +Map config = Maps.newHashMap(); +// This instructs the sink to emit after every element, otherwise they would be buffered +config.put("bulk.flush.max.actions", "1"); +config.put("cluster.name", "my-cluster-name"); + +input.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder() { + @Override + public IndexRequest createIndexRequest(String element, RuntimeContext ctx) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json); + } +})); +{% endhighlight %} +
    +
    +{% highlight scala %} +val input: DataStream[String] = ... + +val config = new util.HashMap[String, String] +config.put("bulk.flush.max.actions", "1") +config.put("cluster.name", "my-cluster-name") + +text.addSink(new ElasticsearchSink(config, new IndexRequestBuilder[String] { + override def createIndexRequest(element: String, ctx: RuntimeContext): IndexRequest = { + val json = new util.HashMap[String, AnyRef] + json.put("data", element) + println("SENDING: " + element) + Requests.indexRequest.index("my-index").`type`("my-type").source(json) + } +})) +{% endhighlight %} +
    +
    + +Not how a Map of Strings is used to configure the Sink. The configuration keys +are documented in the Elasticsearch documentation +[here](https://www.elastic.co/guide/en/elasticsearch/reference/current/index.html). +Especially important is the `cluster.name` parameter that must correspond to +the name of your cluster. + +Internally, the sink uses a `BulkProcessor` to send index requests to the cluster. +This will buffer elements before sending a request to the cluster. The behaviour of the +`BulkProcessor` can be configured using these config keys: + * **bulk.flush.max.actions**: Maximum amount of elements to buffer + * **bulk.flush.max.size.mb**: Maximum amount of data (in megabytes) to buffer + * **bulk.flush.interval.ms**: Interval at which to flush data regardless of the other two + settings in milliseconds + +This example code does the same, but with a `TransportClient`: +
    +
    +{% highlight java %} +DataStream input = ...; + +Map config = Maps.newHashMap(); +// This instructs the sink to emit after every element, otherwise they would be buffered +config.put("bulk.flush.max.actions", "1"); +config.put("cluster.name", "my-cluster-name"); + +List transports = new ArrayList(); +transports.add(new InetSocketTransportAddress("node-1", 9300)); +transports.add(new InetSocketTransportAddress("node-2", 9300)); + +input.addSink(new ElasticsearchSink<>(config, transports, new IndexRequestBuilder() { + @Override + public IndexRequest createIndexRequest(String element, RuntimeContext ctx) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json); + } +})); +{% endhighlight %} +
    +
    +{% highlight scala %} +val input: DataStream[String] = ... + +val config = new util.HashMap[String, String] +config.put("bulk.flush.max.actions", "1") +config.put("cluster.name", "my-cluster-name") + +val transports = new ArrayList[String] +transports.add(new InetSocketTransportAddress("node-1", 9300)) +transports.add(new InetSocketTransportAddress("node-2", 9300)) + +text.addSink(new ElasticsearchSink(config, transports, new IndexRequestBuilder[String] { + override def createIndexRequest(element: String, ctx: RuntimeContext): IndexRequest = { + val json = new util.HashMap[String, AnyRef] + json.put("data", element) + println("SENDING: " + element) + Requests.indexRequest.index("my-index").`type`("my-type").source(json) + } +})) +{% endhighlight %} +
    +
    + +The difference is that we now need to provide a list of Elasticsearch Nodes +to which the sink should connect using a `TransportClient`. + +More about information about Elasticsearch can be found [here](https://elastic.co). + +[Back to top](#top) + ### RabbitMQ This connector provides access to data streams from [RabbitMQ](http://www.rabbitmq.com/). To use this connector, add the following dependency to your project: diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml new file mode 100644 index 0000000000000..ccce7466d9824 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml @@ -0,0 +1,81 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-streaming-connectors-parent + 0.10-SNAPSHOT + .. + + + flink-connector-elasticsearch + flink-connector-elasticsearch + + jar + + + + 1.7.1 + + + + + + org.apache.flink + flink-streaming-core + ${project.version} + + + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + + + + org.apache.flink + flink-streaming-core + ${project.version} + test + test-jar + + + + org.apache.flink + flink-tests + ${project.version} + test + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + + diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java new file mode 100644 index 0000000000000..546ec8d69e4be --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java @@ -0,0 +1,315 @@ +/* + * 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.streaming.connectors.elasticsearch; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.transport.TransportClient; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.collect.ImmutableList; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.node.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.elasticsearch.node.NodeBuilder.nodeBuilder; + + +/** + * Sink that emits its input elements to an Elasticsearch cluster. + * + *

    + * When using the first constructor {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)} + * the sink will create a local {@link Node} for communicating with the + * Elasticsearch cluster. When using the second constructor + * {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)} a {@link TransportClient} will + * be used instead. + * + *

    + * Attention: When using the {@code TransportClient} the sink will fail if no cluster + * can be connected to. With the {@code Node Client} the sink will block and wait for a cluster + * to come online. + * + *

    + * The {@link Map} passed to the constructor is forwarded to Elasticsearch when creating + * the {@link Node} or {@link TransportClient}. The config keys can be found in the Elasticsearch + * documentation. An important setting is {@code cluster.name}, this should be set to the name + * of the cluster that the sink should emit to. + * + *

    + * Internally, the sink will use a {@link BulkProcessor} to send {@link IndexRequest IndexRequests}. + * This will buffer elements before sending a request to the cluster. The behaviour of the + * {@code BulkProcessor} can be configured using these config keys: + *

      + *
    • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
    • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
    • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds + *
    + * + *

    + * You also have to provide an {@link IndexRequestBuilder}. This is used to create an + * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See + * {@link org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder} for an example. + * + * @param Type of the elements emitted by this sink + */ +public class ElasticsearchSink extends RichSinkFunction { + + public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; + public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; + public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class); + + /** + * The user specified config map that we forward to Elasticsearch when we create the Client. + */ + private final Map userConfig; + + /** + * The list of nodes that the TransportClient should connect to. This is null if we are using + * an embedded Node to get a Client. + */ + private final List transportNodes; + + /** + * The builder that is used to construct an {@link IndexRequest} from the incoming element. + */ + private final IndexRequestBuilder indexRequestBuilder; + + /** + * The embedded Node that is used to communicate with the Elasticsearch cluster. This is null + * if we are using a TransportClient. + */ + private transient Node node; + + /** + * The Client that was either retrieved from a Node or is a TransportClient. + */ + private transient Client client; + + /** + * Bulk processor that was created using the client + */ + private transient BulkProcessor bulkProcessor; + + /** + * This is set from inside the BulkProcessor listener if there where failures in processing. + */ + private final AtomicBoolean hasFailure = new AtomicBoolean(false); + + /** + * This is set from inside the BulkProcessor listener if a Throwable was thrown during processing. + */ + private final AtomicReference failureThrowable = new AtomicReference<>(); + + /** + * Creates a new ElasticsearchSink that connects to the cluster using an embedded Node. + * + * @param userConfig The map of user settings that are passed when constructing the Node and BulkProcessor + * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element + */ + public ElasticsearchSink(Map userConfig, IndexRequestBuilder indexRequestBuilder) { + this.userConfig = userConfig; + this.indexRequestBuilder = indexRequestBuilder; + transportNodes = null; + } + + /** + * Creates a new ElasticsearchSink that connects to the cluster using a TransportClient. + * + * @param userConfig The map of user settings that are passed when constructing the TransportClient and BulkProcessor + * @param transportNodes The Elasticsearch Nodes to which to connect using a {@code TransportClient} + * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element + * + */ + public ElasticsearchSink(Map userConfig, List transportNodes, IndexRequestBuilder indexRequestBuilder) { + this.userConfig = userConfig; + this.indexRequestBuilder = indexRequestBuilder; + this.transportNodes = transportNodes; + } + + /** + * Initializes the connection to Elasticsearch by either creating an embedded + * {@link org.elasticsearch.node.Node} and retrieving the + * {@link org.elasticsearch.client.Client} from it or by creating a + * {@link org.elasticsearch.client.transport.TransportClient}. + */ + @Override + public void open(Configuration configuration) { + if (transportNodes == null) { + // Make sure that we disable http access to our embedded node + Settings settings = + ImmutableSettings.settingsBuilder() + .put(userConfig) + .put("http.enabled", false) + .build(); + + node = + nodeBuilder() + .settings(settings) + .client(true) + .data(false) + .node(); + + client = node.client(); + + if (LOG.isInfoEnabled()) { + LOG.info("Created Elasticsearch Client {} from embedded Node", client); + } + + } else { + Settings settings = ImmutableSettings.settingsBuilder() + .put(userConfig) + .build(); + + TransportClient transportClient = new TransportClient(settings); + for (TransportAddress transport: transportNodes) { + transportClient.addTransportAddress(transport); + } + + // verify that we actually are connected to a cluster + ImmutableList nodes = transportClient.connectedNodes(); + if (nodes.isEmpty()) { + throw new RuntimeException("Client is not connected to any Elasticsearch nodes!"); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Connected to nodes: " + nodes.toString()); + } + } + + client = transportClient; + + if (LOG.isInfoEnabled()) { + LOG.info("Created Elasticsearch TransportClient {}", client); + } + } + + BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder( + client, + new BulkProcessor.Listener() { + @Override + public void beforeBulk(long executionId, + BulkRequest request) { + + } + + @Override + public void afterBulk(long executionId, + BulkRequest request, + BulkResponse response) { + if (response.hasFailures()) { + for (BulkItemResponse itemResp : response.getItems()) { + if (itemResp.isFailed()) { + LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage()); + failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage())); + } + } + hasFailure.set(true); + } + } + + @Override + public void afterBulk(long executionId, + BulkRequest request, + Throwable failure) { + LOG.error(failure.getMessage()); + failureThrowable.compareAndSet(null, failure); + hasFailure.set(true); + } + }); + + // This makes flush() blocking + bulkProcessorBuilder.setConcurrentRequests(0); + + ParameterTool params = ParameterTool.fromMap(userConfig); + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { + bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) { + bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt( + CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB)); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { + bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS))); + } + + bulkProcessor = bulkProcessorBuilder.build(); + } + + @Override + public void invoke(T element) { + IndexRequest indexRequest = indexRequestBuilder.createIndexRequest(element, getRuntimeContext()); + + if (LOG.isDebugEnabled()) { + LOG.debug("Emitting IndexRequest: {}", indexRequest); + } + + bulkProcessor.add(indexRequest); + } + + @Override + public void close() { + if (bulkProcessor != null) { + bulkProcessor.close(); + bulkProcessor = null; + } + + if (client != null) { + client.close(); + } + + if (node != null) { + node.close(); + } + + if (hasFailure.get()) { + Throwable cause = failureThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occured in ElasticsearchSink.", cause); + } else { + throw new RuntimeException("An error occured in ElasticsearchSink."); + + } + } + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java new file mode 100644 index 0000000000000..04ae40a2cadee --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java @@ -0,0 +1,66 @@ +/** + * 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.streaming.connectors.elasticsearch; + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.elasticsearch.action.index.IndexRequest; + +import java.io.Serializable; + +/** + * Function that creates an {@link IndexRequest} from an element in a Stream. + * + *

    + * This is used by {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink} + * to prepare elements for sending them to Elasticsearch. See + * Index API + * for information about how to format data for adding it to an Elasticsearch index. + * + *

    + * Example: + * + *

    {@code
    + *     private static class MyIndexRequestBuilder implements IndexRequestBuilder {
    + *
    + *         public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
    + *             Map json = new HashMap<>();
    + *             json.put("data", element);
    + *
    + *             return Requests.indexRequest()
    + *                 .index("my-index")
    + *                 .type("my-type")
    + *                 .source(json);
    + *         }
    + *     }
    + * }
    + * + * @param The type of the element handled by this {@code IndexRequestBuilder} + */ +public interface IndexRequestBuilder extends Function, Serializable { + + /** + * Creates an {@link org.elasticsearch.action.index.IndexRequest} from an element. + * + * @param element The element that needs to be turned in to an {@code IndexRequest} + * @param ctx The Flink {@link RuntimeContext} of the {@link ElasticsearchSink} + * + * @return The constructed {@code IndexRequest} + */ + IndexRequest createIndexRequest(T element, RuntimeContext ctx); +} diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java new file mode 100644 index 0000000000000..298eb64222436 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java @@ -0,0 +1,81 @@ +/* + * 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.streaming.connectors.elasticsearch.examples; + +import com.google.common.collect.Maps; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; + +import java.util.HashMap; +import java.util.Map; + +/** + * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that + * you have a cluster names "elasticsearch" running or change the cluster name in the config map. + */ +public class ElasticsearchExample { + + public static void main(String[] args) throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource source = env.addSource(new SourceFunction() { + private static final long serialVersionUID = 1L; + + private volatile boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + for (int i = 0; i < 20 && running; i++) { + ctx.collect("message #" + i); + } + } + + @Override + public void cancel() { + running = false; + } + }); + + Map config = Maps.newHashMap(); + // This instructs the sink to emit after every element, otherwise they would be buffered + config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + source.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder() { + @Override + public IndexRequest createIndexRequest(String element, RuntimeContext ctx) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json); + } + })); + + + env.execute("Elasticsearch Example"); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java new file mode 100644 index 0000000000000..33a2e476e9ae7 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java @@ -0,0 +1,205 @@ +/** + * 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.streaming.connectors.elasticsearch; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.Requests; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.node.Node; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.node.NodeBuilder.nodeBuilder; + +public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase { + + private static final int NUM_ELEMENTS = 20; + + @ClassRule + public static TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testNodeClient() throws Exception{ + + File dataDir = tempFolder.newFolder(); + + Node node = nodeBuilder() + .settings(ImmutableSettings.settingsBuilder() + .put("http.enabled", false) + .put("path.data", dataDir.getAbsolutePath())) + // set a custom cluster name to verify that user config works correctly + .clusterName("my-node-client-cluster") + .local(true) + .node(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new TestSourceFunction()); + + Map config = Maps.newHashMap(); + // This instructs the sink to emit after every element, otherwise they would be buffered + config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + config.put("cluster.name", "my-node-client-cluster"); + + // connect to our local node + config.put("node.local", "true"); + + source.addSink(new ElasticsearchSink<>(config, new TestIndexRequestBuilder())); + + env.execute("Elasticsearch Node Client Test"); + + + // verify the results + Client client = node.client(); + for (int i = 0; i < NUM_ELEMENTS; i++) { + GetResponse response = client.get(new GetRequest("my-index", + "my-type", + Integer.toString(i))).actionGet(); + Assert.assertEquals("message #" + i, response.getSource().get("data")); + } + + node.close(); + } + + @Test + public void testTransportClient() throws Exception { + + File dataDir = tempFolder.newFolder(); + + Node node = nodeBuilder() + .settings(ImmutableSettings.settingsBuilder() + .put("http.enabled", false) + .put("path.data", dataDir.getAbsolutePath())) + // set a custom cluster name to verify that user config works correctly + .clusterName("my-node-client-cluster") + .local(true) + .node(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new TestSourceFunction()); + + Map config = Maps.newHashMap(); + // This instructs the sink to emit after every element, otherwise they would be buffered + config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + config.put("cluster.name", "my-node-client-cluster"); + + // connect to our local node + config.put("node.local", "true"); + + List transports = Lists.newArrayList(); + transports.add(new LocalTransportAddress("1")); + + source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder())); + + env.execute("Elasticsearch TransportClient Test"); + + + // verify the results + Client client = node.client(); + for (int i = 0; i < NUM_ELEMENTS; i++) { + GetResponse response = client.get(new GetRequest("my-index", + "my-type", + Integer.toString(i))).actionGet(); + Assert.assertEquals("message #" + i, response.getSource().get("data")); + } + + node.close(); + } + + @Test(expected = JobExecutionException.class) + public void testTransportClientFails() throws Exception{ + // this checks whether the TransportClient fails early when there is no cluster to + // connect to. We don't hava such as test for the Node Client version since that + // one will block and wait for a cluster to come online + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new TestSourceFunction()); + + Map config = Maps.newHashMap(); + // This instructs the sink to emit after every element, otherwise they would be buffered + config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + config.put("cluster.name", "my-node-client-cluster"); + + // connect to our local node + config.put("node.local", "true"); + + List transports = Lists.newArrayList(); + transports.add(new LocalTransportAddress("1")); + + source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder())); + + env.execute("Elasticsearch Node Client Test"); + } + + private static class TestSourceFunction implements SourceFunction> { + private static final long serialVersionUID = 1L; + + private volatile boolean running = true; + + @Override + public void run(SourceContext> ctx) throws Exception { + for (int i = 0; i < NUM_ELEMENTS && running; i++) { + ctx.collect(Tuple2.of(i, "message #" + i)); + } + } + + @Override + public void cancel() { + running = false; + } + } + + private static class TestIndexRequestBuilder implements IndexRequestBuilder> { + private static final long serialVersionUID = 1L; + + @Override + public IndexRequest createIndexRequest(Tuple2 element, RuntimeContext ctx) { + Map json = new HashMap<>(); + json.put("data", element.f1); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .id(element.f0.toString()) + .source(json); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..dc207262c0760 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties @@ -0,0 +1,27 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=OFF, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml new file mode 100644 index 0000000000000..45b3b92f0cf65 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml @@ -0,0 +1,30 @@ + + + + + + %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n + + + + + + + + \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml index b06b57a9064d5..4e9730f21e04d 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml +++ b/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml @@ -38,6 +38,7 @@ under the License. flink-connector-flume flink-connector-kafka + flink-connector-elasticsearch flink-connector-rabbitmq flink-connector-twitter From cf88407a5a81d11f29782d9c3c2775653715ce4a Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 23 Aug 2015 13:06:15 +0200 Subject: [PATCH 165/175] [FLINK-2563] [gelly] extended the run() method of GraphAlgorithm interface to return an arbitrary type This closes #1042 --- .../main/scala/org/apache/flink/graph/scala/Graph.scala | 5 +++-- .../src/main/java/org/apache/flink/graph/Graph.java | 9 +++++++-- .../main/java/org/apache/flink/graph/GraphAlgorithm.java | 5 +++-- .../apache/flink/graph/example/ConnectedComponents.java | 2 -- .../apache/flink/graph/library/CommunityDetection.java | 3 ++- .../apache/flink/graph/library/ConnectedComponents.java | 3 ++- .../flink/graph/library/GSAConnectedComponents.java | 3 ++- .../java/org/apache/flink/graph/library/GSAPageRank.java | 2 +- .../graph/library/GSASingleSourceShortestPaths.java | 3 ++- .../org/apache/flink/graph/library/LabelPropagation.java | 3 ++- .../java/org/apache/flink/graph/library/PageRank.java | 3 ++- .../flink/graph/library/SingleSourceShortestPaths.java | 3 ++- .../graph/test/GatherSumApplyConfigurationITCase.java | 3 +++ 13 files changed, 31 insertions(+), 16 deletions(-) diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala index 738fd903def97..73e175eb92306 100644 --- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala +++ b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala @@ -649,8 +649,9 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { jtuple.f1)) } - def run(algorithm: GraphAlgorithm[K, VV, EV]) = { - wrapGraph(jgraph.run(algorithm)) + def run[T: TypeInformation : ClassTag](algorithm: GraphAlgorithm[K, VV, EV, T]): + T = { + jgraph.run(algorithm) } /** diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java index ff0ec245c5f21..84085c90d9202 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java @@ -432,7 +432,6 @@ public Graph mapEdges(final MapFunction, NV> mapper) * @param returnType the explicit return type. * @return a new graph */ - @SuppressWarnings({ "unchecked", "rawtypes" }) public Graph mapEdges(final MapFunction, NV> mapper, TypeInformation> returnType) { DataSet> mappedEdges = edges.map( new MapFunction, Edge>() { @@ -1451,7 +1450,13 @@ public Graph runGatherSumApplyIteration( return new Graph(newVertices, this.edges, this.context); } - public Graph run(GraphAlgorithm algorithm) throws Exception { + /** + * @param algorithm the algorithm to run on the Graph + * @param the return type + * @return the result of the graph algorithm + * @throws Exception + */ + public T run(GraphAlgorithm algorithm) throws Exception { return algorithm.run(this); } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java index 04181d56e7a15..08cf011068ec5 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java @@ -22,8 +22,9 @@ * @param key type * @param vertex value type * @param edge value type + * @param the return type */ -public interface GraphAlgorithm { +public interface GraphAlgorithm { - public Graph run(Graph input) throws Exception; + public T run(Graph input) throws Exception; } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java index a4a6708cdf917..2b17a149a5134 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java @@ -80,8 +80,6 @@ public Long map(Long value) throws Exception { } else { verticesWithMinIds.print(); } - - } @Override diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java index 21bef531a5175..31488eeb86aee 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java @@ -44,7 +44,8 @@ * * @see article explaining the algorithm in detail */ -public class CommunityDetection implements GraphAlgorithm { +public class CommunityDetection implements + GraphAlgorithm> { private Integer maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java index c2cec18fddc85..871f3154dd121 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java @@ -37,7 +37,8 @@ * is reached. */ @SuppressWarnings("serial") -public class ConnectedComponents implements GraphAlgorithm{ +public class ConnectedComponents implements + GraphAlgorithm> { private Integer maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java index b77ca079a0978..f852f3e11e351 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java @@ -29,7 +29,8 @@ /** * This is an implementation of the Connected Components algorithm, using a gather-sum-apply iteration. */ -public class GSAConnectedComponents implements GraphAlgorithm { +public class GSAConnectedComponents implements + GraphAlgorithm> { private Integer maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java index 4adaaa9604bd9..6ce2ed6633d38 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java @@ -36,7 +36,7 @@ * * The implementation assumes that each page has at least one incoming and one outgoing link. */ -public class GSAPageRank implements GraphAlgorithm { +public class GSAPageRank implements GraphAlgorithm> { private double beta; private int maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java index 78c535baeef6d..18bdd1d353d2a 100755 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java @@ -30,7 +30,8 @@ /** * This is an implementation of the Single Source Shortest Paths algorithm, using a gather-sum-apply iteration */ -public class GSASingleSourceShortestPaths implements GraphAlgorithm { +public class GSASingleSourceShortestPaths implements + GraphAlgorithm> { private final K srcVertexId; private final Integer maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java index 16489228497db..2dd21800aa870 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java @@ -42,7 +42,8 @@ */ @SuppressWarnings("serial") -public class LabelPropagation> implements GraphAlgorithm { +public class LabelPropagation> implements + GraphAlgorithm> { private final int maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java index 93b10eb67e909..47f7acd035966 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java @@ -37,7 +37,8 @@ * * The implementation assumes that each page has at least one incoming and one outgoing link. */ -public class PageRank implements GraphAlgorithm { +public class PageRank implements + GraphAlgorithm> { private double beta; private int maxIterations; diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java index 97ea0009d923a..1911f73ff4d02 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java @@ -31,7 +31,8 @@ * This is an implementation of the Single-Source-Shortest Paths algorithm, using a vertex-centric iteration. */ @SuppressWarnings("serial") -public class SingleSourceShortestPaths implements GraphAlgorithm { +public class SingleSourceShortestPaths implements + GraphAlgorithm> { private final K srcVertexId; private final Integer maxIterations; diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java index 53455c5a86779..75f174c7fdc5d 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java @@ -369,6 +369,7 @@ public HashSet map(Vertex value) throws Exception { } } + @SuppressWarnings("serial") private static final class GetReachableVertices extends GatherFunction, Long, HashSet> { @Override @@ -377,6 +378,7 @@ public HashSet gather(Neighbor, Long> neighbor) { } } + @SuppressWarnings("serial") private static final class FindAllReachableVertices extends SumFunction, Long, HashSet> { @Override public HashSet sum(HashSet newSet, HashSet currentSet) { @@ -388,6 +390,7 @@ public HashSet sum(HashSet newSet, HashSet currentSet) { } } + @SuppressWarnings("serial") private static final class UpdateReachableVertices extends ApplyFunction, HashSet> { @Override From 5156a1b3f82c2362481812dfee480823ac492975 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Mon, 24 Aug 2015 17:49:44 +0800 Subject: [PATCH 166/175] [FLINK-2564] [core] Improve stability of RandomSamplerTest - Expand the verification boundary for random sampler test - Increase the number of times we sample This closes #1047 --- .../flink/api/java/sampling/RandomSamplerTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java index 83e5b419e98f6..83a95ae28863e 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java @@ -239,14 +239,14 @@ private void verifySamplerFraction(double fraction, boolean withReplacement) { sampler = new BernoulliSampler(fraction); } - // take 5 times sample, and take the average result size for next step comparison. + // take 20 times sample, and take the average result size for next step comparison. int totalSampledSize = 0; - double sampleCount = 5; + double sampleCount = 20; for (int i = 0; i < sampleCount; i++) { totalSampledSize += getSize(sampler.sample(source.iterator())); } double resultFraction = totalSampledSize / ((double) SOURCE_SIZE * sampleCount); - assertTrue(String.format("expected fraction: %f, result fraction: %f", fraction, resultFraction), Math.abs((resultFraction - fraction) / fraction) < 0.1); + assertTrue(String.format("expected fraction: %f, result fraction: %f", fraction, resultFraction), Math.abs((resultFraction - fraction) / fraction) < 0.2); } /* @@ -440,13 +440,13 @@ private double[] getWrongSampler(int fixSize) { } /* - * Calculate the D value of K-S test for p-value 0.05, m and n are the sample size + * Calculate the D value of K-S test for p-value 0.01, m and n are the sample size */ private double getDValue(int m, int n) { Preconditions.checkArgument(m > 0, "input sample size should be positive."); Preconditions.checkArgument(n > 0, "input sample size should be positive."); double first = (double) m; double second = (double) n; - return 1.36 * Math.sqrt((first + second) / (first * second)); + return 1.63 * Math.sqrt((first + second) / (first * second)); } } From 6f07c5f3af1b4fd67cb61ad140b99095f7ee6c45 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 25 Aug 2015 14:42:47 +0200 Subject: [PATCH 167/175] [FLINK-2528] [tests] Increase robustness and error reporting of MatchTaskTest --- .../runtime/operators/MatchTaskTest.java | 280 ++++++++++-------- .../operators/testutils/DriverTestBase.java | 22 +- 2 files changed, 170 insertions(+), 132 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java index 8fbf05eac8b34..15f3d0cada927 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java @@ -16,12 +16,12 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; @@ -38,9 +38,12 @@ import org.apache.flink.types.Key; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; + import org.junit.Assert; import org.junit.Test; +import static org.junit.Assert.*; + @SuppressWarnings("deprecation") public class MatchTaskTest extends DriverTestBase> { @@ -58,11 +61,11 @@ public class MatchTaskTest extends DriverTestBase>[])new Class[]{ IntValue.class }); + new int[]{0}, (Class>[])new Class[]{ IntValue.class }); @SuppressWarnings("unchecked") private final RecordComparator comparator2 = new RecordComparator( - new int[]{0}, (Class>[])new Class[]{ IntValue.class }); + new int[]{0}, (Class>[])new Class[]{ IntValue.class }); private final List outList = new ArrayList(); @@ -393,150 +396,177 @@ public void testFailingMatchTask() { @Test public void testCancelMatchTaskWhileSort1() { - int keyCnt = 20; - int valCnt = 20; - - setOutput(new NirvanaOutputList()); - addDriverComparator(this.comparator1); - addDriverComparator(this.comparator2); - getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get()); - getTaskConfig().setDriverStrategy(DriverStrategy.MERGE); - getTaskConfig().setRelativeMemoryDriver(bnljn_frac); - setNumFileHandlesForSort(4); - - final JoinDriver testTask = new JoinDriver(); + final int keyCnt = 20; + final int valCnt = 20; try { - addInputSorted(new DelayingInfinitiveInputIterator(100), this.comparator1.duplicate()); - addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("The test caused an exception."); - } - - final AtomicBoolean success = new AtomicBoolean(false); - - Thread taskRunner = new Thread() { - @Override - public void run() { - try { - testDriver(testTask, MockMatchStub.class); - success.set(true); - } catch (Exception ie) { - ie.printStackTrace(); + setOutput(new NirvanaOutputList()); + addDriverComparator(this.comparator1); + addDriverComparator(this.comparator2); + getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get()); + getTaskConfig().setDriverStrategy(DriverStrategy.MERGE); + getTaskConfig().setRelativeMemoryDriver(bnljn_frac); + setNumFileHandlesForSort(4); + + final JoinDriver testTask = new JoinDriver(); + + try { + addInputSorted(new DelayingInfinitiveInputIterator(100), this.comparator1.duplicate()); + addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("The test caused an exception."); + } + + final AtomicReference error = new AtomicReference<>(); + + Thread taskRunner = new Thread("Task runner for testCancelMatchTaskWhileSort1()") { + @Override + public void run() { + try { + testDriver(testTask, MockMatchStub.class); + } + catch (Throwable t) { + error.set(t); + } } + }; + taskRunner.start(); + + Thread.sleep(1000); + + cancel(); + taskRunner.interrupt(); + + taskRunner.join(60000); + + assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive()); + + Throwable taskError = error.get(); + if (taskError != null) { + taskError.printStackTrace(); + fail("Error in task while canceling: " + taskError.getMessage()); } - }; - taskRunner.start(); - - TaskCancelThread tct = new TaskCancelThread(1, taskRunner, this); - tct.start(); - - try { - tct.join(); - taskRunner.join(); - } catch(InterruptedException ie) { - Assert.fail("Joining threads failed"); } - - Assert.assertTrue("Test threw an exception even though it was properly canceled.", success.get()); + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } @Test public void testCancelMatchTaskWhileSort2() { - int keyCnt = 20; - int valCnt = 20; - - setOutput(new NirvanaOutputList()); - addDriverComparator(this.comparator1); - addDriverComparator(this.comparator2); - getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get()); - getTaskConfig().setDriverStrategy(DriverStrategy.MERGE); - getTaskConfig().setRelativeMemoryDriver(bnljn_frac); - setNumFileHandlesForSort(4); - - final JoinDriver testTask = new JoinDriver(); + final int keyCnt = 20; + final int valCnt = 20; try { - addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); - addInputSorted(new DelayingInfinitiveInputIterator(100), this.comparator1.duplicate()); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("The test caused an exception."); - } - - final AtomicBoolean success = new AtomicBoolean(false); - - Thread taskRunner = new Thread() { - @Override - public void run() { - try { - testDriver(testTask, MockMatchStub.class); - success.set(true); - } catch (Exception ie) { - ie.printStackTrace(); + setOutput(new NirvanaOutputList()); + addDriverComparator(this.comparator1); + addDriverComparator(this.comparator2); + getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get()); + getTaskConfig().setDriverStrategy(DriverStrategy.MERGE); + getTaskConfig().setRelativeMemoryDriver(bnljn_frac); + setNumFileHandlesForSort(4); + + final JoinDriver testTask = new JoinDriver(); + + try { + addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); + addInputSorted(new DelayingInfinitiveInputIterator(100), this.comparator1.duplicate()); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("The test caused an exception."); + } + + final AtomicReference error = new AtomicReference<>(); + + Thread taskRunner = new Thread("Task runner for testCancelMatchTaskWhileSort2()") { + @Override + public void run() { + try { + testDriver(testTask, MockMatchStub.class); + } + catch (Throwable t) { + error.set(t); + } } + }; + taskRunner.start(); + + Thread.sleep(1000); + + cancel(); + taskRunner.interrupt(); + + taskRunner.join(60000); + + assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive()); + + Throwable taskError = error.get(); + if (taskError != null) { + taskError.printStackTrace(); + fail("Error in task while canceling: " + taskError.getMessage()); } - }; - taskRunner.start(); - - TaskCancelThread tct = new TaskCancelThread(1, taskRunner, this); - tct.start(); - - try { - tct.join(); - taskRunner.join(); - } catch(InterruptedException ie) { - Assert.fail("Joining threads failed"); } - - Assert.assertTrue("Test threw an exception even though it was properly canceled.", success.get()); + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } @Test public void testCancelMatchTaskWhileMatching() { - int keyCnt = 20; - int valCnt = 20; - - setOutput(new NirvanaOutputList()); - addDriverComparator(this.comparator1); - addDriverComparator(this.comparator2); - getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get()); - getTaskConfig().setDriverStrategy(DriverStrategy.MERGE); - getTaskConfig().setRelativeMemoryDriver(bnljn_frac); - setNumFileHandlesForSort(4); - - final JoinDriver testTask = new JoinDriver(); - - addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); - addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); - - final AtomicBoolean success = new AtomicBoolean(false); + final int keyCnt = 20; + final int valCnt = 20; - Thread taskRunner = new Thread() { - @Override - public void run() { - try { - testDriver(testTask, MockDelayingMatchStub.class); - success.set(true); - } catch (Exception ie) { - ie.printStackTrace(); + try { + setOutput(new NirvanaOutputList()); + addDriverComparator(this.comparator1); + addDriverComparator(this.comparator2); + getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get()); + getTaskConfig().setDriverStrategy(DriverStrategy.MERGE); + getTaskConfig().setRelativeMemoryDriver(bnljn_frac); + setNumFileHandlesForSort(4); + + final JoinDriver testTask = new JoinDriver(); + + addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); + addInput(new UniformRecordGenerator(keyCnt, valCnt, true)); + + final AtomicReference error = new AtomicReference<>(); + + Thread taskRunner = new Thread("Task runner for testCancelMatchTaskWhileMatching()") { + @Override + public void run() { + try { + testDriver(testTask, MockDelayingMatchStub.class); + } + catch (Throwable t) { + error.set(t); + } } + }; + taskRunner.start(); + + Thread.sleep(1000); + + cancel(); + taskRunner.interrupt(); + + taskRunner.join(60000); + + assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive()); + + Throwable taskError = error.get(); + if (taskError != null) { + taskError.printStackTrace(); + fail("Error in task while canceling: " + taskError.getMessage()); } - }; - taskRunner.start(); - - TaskCancelThread tct = new TaskCancelThread(1, taskRunner, this); - tct.start(); - - try { - tct.join(); - taskRunner.join(); - } catch(InterruptedException ie) { - Assert.fail("Joining threads failed"); } - - Assert.assertTrue("Test threw an exception even though it was properly canceled.", success.get()); + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index e4aad981a4ac2..12ca909e5ac96 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.operators.testutils; -import java.io.FileNotFoundException; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.LinkedList; @@ -88,7 +86,7 @@ public class DriverTestBase extends TestLogger implements Pa private PactDriver driver; - private volatile boolean running; + private volatile boolean running = true; private ExecutionConfig executionConfig; @@ -119,7 +117,7 @@ protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNu } @Parameterized.Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { + public static Collection getConfigurations() { LinkedList configs = new LinkedList(); @@ -184,7 +182,6 @@ public void testDriverInternal(PactDriver driver, Class stubClass) throws Except this.stub = (S)stubClass.newInstance(); // regular running logic - this.running = true; boolean stubOpen = false; try { @@ -205,6 +202,10 @@ public void testDriverInternal(PactDriver driver, Class stubClass) throws Except throw new Exception("The user defined 'open()' method caused an exception: " + t.getMessage(), t); } + if (!running) { + return; + } + // run the user code driver.run(); @@ -222,10 +223,10 @@ public void testDriverInternal(PactDriver driver, Class stubClass) throws Except try { FunctionUtils.closeFunction(this.stub); } - catch (Throwable t) {} + catch (Throwable ignored) {} } - // if resettable driver invoke treardown + // if resettable driver invoke tear down if (this.driver instanceof ResettablePactDriver) { final ResettablePactDriver resDriver = (ResettablePactDriver) this.driver; try { @@ -269,6 +270,13 @@ public void testResettableDriver(ResettablePactDriver driver, Class stubClass, i public void cancel() throws Exception { this.running = false; + + // compensate for races, where cancel is called before the driver is set + // not that this is an artifact of a bad design of this test base, where the setup + // of the basic properties is not separated from the invocation of the execution logic + while (this.driver == null) { + Thread.sleep(200); + } this.driver.cancel(); } From b2f8e3070b8db16093f36680f9c8ce124aae8627 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 19 Aug 2015 16:11:13 +0200 Subject: [PATCH 168/175] [FLINK-2089] [runtime] Fix illegal state in RecordWriter after partition write failure - Address PR comments This closes #1050. --- .../io/network/api/writer/RecordWriter.java | 59 +++-- .../network/api/writer/RecordWriterTest.java | 217 ++++++++++-------- 2 files changed, 162 insertions(+), 114 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 17a6a18993da3..c534aa2188d28 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -86,8 +86,7 @@ public void emit(T record) throws IOException, InterruptedException { Buffer buffer = serializer.getCurrentBuffer(); if (buffer != null) { - writer.writeBuffer(buffer, targetChannel); - serializer.clearCurrentBuffer(); + writeBuffer(buffer, targetChannel, serializer); } buffer = writer.getBufferProvider().requestBufferBlocking(); @@ -112,8 +111,7 @@ public void broadcastEmit(T record) throws IOException, InterruptedException { Buffer buffer = serializer.getCurrentBuffer(); if (buffer != null) { - writer.writeBuffer(buffer, targetChannel); - serializer.clearCurrentBuffer(); + writeBuffer(buffer, targetChannel, serializer); } buffer = writer.getBufferProvider().requestBufferBlocking(); @@ -135,8 +133,7 @@ public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedE throw new IllegalStateException("Serializer has data but no buffer."); } - writer.writeBuffer(buffer, targetChannel); - serializer.clearCurrentBuffer(); + writeBuffer(buffer, targetChannel, serializer); writer.writeEvent(event, targetChannel); @@ -157,8 +154,7 @@ public void sendEndOfSuperstep() throws IOException, InterruptedException { synchronized (serializer) { Buffer buffer = serializer.getCurrentBuffer(); if (buffer != null) { - writer.writeBuffer(buffer, targetChannel); - serializer.clearCurrentBuffer(); + writeBuffer(buffer, targetChannel, serializer); buffer = writer.getBufferProvider().requestBufferBlocking(); serializer.setNextBuffer(buffer); @@ -174,26 +170,31 @@ public void flush() throws IOException { RecordSerializer serializer = serializers[targetChannel]; synchronized (serializer) { - Buffer buffer = serializer.getCurrentBuffer(); + try { + Buffer buffer = serializer.getCurrentBuffer(); - if (buffer != null) { - // Only clear the serializer after the buffer was written out. - writer.writeBuffer(buffer, targetChannel); + if (buffer != null) { + writeBuffer(buffer, targetChannel, serializer); + } + } finally { + serializer.clear(); } - - serializer.clear(); } } } public void clearBuffers() { - for (RecordSerializer s : serializers) { - synchronized (s) { - Buffer b = s.getCurrentBuffer(); - s.clear(); + for (RecordSerializer serializer : serializers) { + synchronized (serializer) { + try { + Buffer buffer = serializer.getCurrentBuffer(); - if (b != null) { - b.recycle(); + if (buffer != null) { + buffer.recycle(); + } + } + finally { + serializer.clear(); } } } @@ -208,4 +209,22 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { } } + /** + * Writes the buffer to the {@link ResultPartitionWriter}. + * + *

    The buffer is cleared from the serializer state after a call to this method. + */ + private void writeBuffer( + Buffer buffer, + int targetChannel, + RecordSerializer serializer) throws IOException { + + try { + writer.writeBuffer(buffer, targetChannel); + } + finally { + serializer.clearCurrentBuffer(); + } + } + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java index 7061fb83ac45f..9e1058268cea9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java @@ -19,9 +19,13 @@ package org.apache.flink.runtime.io.network.api.writer; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.util.TestBufferFactory; +import org.apache.flink.runtime.io.network.util.TestTaskEvent; import org.apache.flink.types.IntValue; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.invocation.InvocationOnMock; @@ -37,6 +41,7 @@ import java.util.concurrent.Future; import static com.google.common.base.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -59,100 +64,6 @@ public class RecordWriterTest { // Resource release tests // --------------------------------------------------------------------------------------------- - @Test - public void testClearBuffersAfterEmit() throws Exception { - final Buffer buffer = TestBufferFactory.createBuffer(32); - - BufferProvider bufferProvider = createBufferProvider(buffer); - ResultPartitionWriter partitionWriter = createResultPartitionWriter(bufferProvider); - - RecordWriter recordWriter = new RecordWriter(partitionWriter); - - // Emit single record, the buffer will not be written out. - // Therefore, it needs to be cleared explicitly. - recordWriter.emit(new IntValue(0)); - - // Verify that a buffer is requested, but not written out. - verify(bufferProvider, times(1)).requestBufferBlocking(); - verify(partitionWriter, never()).writeBuffer(any(Buffer.class), anyInt()); - - recordWriter.clearBuffers(); - - assertTrue("Buffer not recycled.", buffer.isRecycled()); - } - - @Test - public void testClearBuffersAfterExceptionInFlushWritePartition() throws Exception { - // Size of buffer ensures that a single record will fill the buffer. - final Buffer buffer = TestBufferFactory.createBuffer(4); - - BufferProvider bufferProvider = createBufferProvider(buffer); - ResultPartitionWriter partitionWriter = createResultPartitionWriter(bufferProvider); - - doThrow(new IOException("Expected test exception")) - .when(partitionWriter).writeBuffer(eq(buffer), eq(0)); - - RecordWriter recordWriter = new RecordWriter(partitionWriter); - - try { - // Emit single record, the buffer will not be written out, - // because of the Exception. Therefore, it needs to be cleared - // explicitly. - recordWriter.emit(new IntValue(0)); - - fail("Did not throw expected Exception. This means that the record " - + "writer did not request a buffer as expected."); - } - catch (IOException expected) { - } - - // Verify that a buffer is requested, but not written out due to the Exception. - verify(bufferProvider, times(1)).requestBufferBlocking(); - verify(partitionWriter, times(1)).writeBuffer(any(Buffer.class), anyInt()); - - recordWriter.clearBuffers(); - - assertTrue("Buffer not recycled.", buffer.isRecycled()); - - } - - @Test - public void testClearBuffersAfterExceptionInEmitWritePartition() throws Exception { - // Size of buffer ensures that a single record will NOT fill the buffer. - final Buffer buffer = TestBufferFactory.createBuffer(32); - - BufferProvider bufferProvider = createBufferProvider(buffer); - ResultPartitionWriter partitionWriter = createResultPartitionWriter(bufferProvider); - - doThrow(new IOException("Expected test exception")) - .when(partitionWriter).writeBuffer(eq(buffer), eq(0)); - - RecordWriter recordWriter = new RecordWriter(partitionWriter); - - try { - recordWriter.emit(new IntValue(0)); - - // Verify that a buffer is requested, but not written out. - verify(bufferProvider, times(1)).requestBufferBlocking(); - verify(partitionWriter, never()).writeBuffer(any(Buffer.class), anyInt()); - - // Now flush the record. - recordWriter.flush(); - - fail("Did not throw expected Exception. This means that the record " - + "writer did not request a buffer as expected."); - } - catch (IOException expected) { - } - - // Flushing the buffer tried to write out the buffer. - verify(partitionWriter, times(1)).writeBuffer(any(Buffer.class), anyInt()); - - recordWriter.clearBuffers(); - - assertTrue("Buffer not recycled.", buffer.isRecycled()); - } - /** * Tests a fix for FLINK-2089. * @@ -241,6 +152,124 @@ public Void call() throws Exception { } } + @Test + public void testClearBuffersAfterExceptionInPartitionWriter() throws Exception { + NetworkBufferPool buffers = null; + BufferPool bufferPool = null; + + try { + buffers = new NetworkBufferPool(1, 1024); + bufferPool = spy(buffers.createBufferPool(1, true)); + + ResultPartitionWriter partitionWriter = mock(ResultPartitionWriter.class); + when(partitionWriter.getBufferProvider()).thenReturn(checkNotNull(bufferPool)); + when(partitionWriter.getNumberOfOutputChannels()).thenReturn(1); + + // Recycle buffer and throw Exception + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Buffer buffer = (Buffer) invocation.getArguments()[0]; + buffer.recycle(); + + throw new RuntimeException("Expected test Exception"); + } + }).when(partitionWriter).writeBuffer(any(Buffer.class), anyInt()); + + RecordWriter recordWriter = new RecordWriter<>(partitionWriter); + + try { + // Verify that emit correctly clears the buffer. The infinite loop looks + // dangerous indeed, but the buffer will only be flushed after its full. Adding a + // manual flush here doesn't test this case (see next). + for (;;) { + recordWriter.emit(new IntValue(0)); + } + } + catch (Exception e) { + // Verify that the buffer is not part of the record writer state after a failure + // to flush it out. If the buffer is still part of the record writer state, this + // will fail, because the buffer has already been recycled. NOTE: The mock + // partition writer needs to recycle the buffer to correctly test this. + recordWriter.clearBuffers(); + } + + // Verify expected methods have been called + verify(partitionWriter, times(1)).writeBuffer(any(Buffer.class), anyInt()); + verify(bufferPool, times(1)).requestBufferBlocking(); + + try { + // Verify that manual flushing correctly clears the buffer. + recordWriter.emit(new IntValue(0)); + recordWriter.flush(); + + Assert.fail("Did not throw expected test Exception"); + } + catch (Exception e) { + recordWriter.clearBuffers(); + } + + // Verify expected methods have been called + verify(partitionWriter, times(2)).writeBuffer(any(Buffer.class), anyInt()); + verify(bufferPool, times(2)).requestBufferBlocking(); + + try { + // Verify that broadcast emit correctly clears the buffer. + for (;;) { + recordWriter.broadcastEmit(new IntValue(0)); + } + } + catch (Exception e) { + recordWriter.clearBuffers(); + } + + // Verify expected methods have been called + verify(partitionWriter, times(3)).writeBuffer(any(Buffer.class), anyInt()); + verify(bufferPool, times(3)).requestBufferBlocking(); + + try { + // Verify that end of super step correctly clears the buffer. + recordWriter.emit(new IntValue(0)); + recordWriter.sendEndOfSuperstep(); + + Assert.fail("Did not throw expected test Exception"); + } + catch (Exception e) { + recordWriter.clearBuffers(); + } + + // Verify expected methods have been called + verify(partitionWriter, times(4)).writeBuffer(any(Buffer.class), anyInt()); + verify(bufferPool, times(4)).requestBufferBlocking(); + + try { + // Verify that broadcasting and event correctly clears the buffer. + recordWriter.emit(new IntValue(0)); + recordWriter.broadcastEvent(new TestTaskEvent()); + + Assert.fail("Did not throw expected test Exception"); + } + catch (Exception e) { + recordWriter.clearBuffers(); + } + + // Verify expected methods have been called + verify(partitionWriter, times(5)).writeBuffer(any(Buffer.class), anyInt()); + verify(bufferPool, times(5)).requestBufferBlocking(); + } + finally { + if (bufferPool != null) { + assertEquals(1, bufferPool.getNumberOfAvailableMemorySegments()); + bufferPool.lazyDestroy(); + } + + if (buffers != null) { + assertEquals(1, buffers.getNumberOfAvailableMemorySegments()); + buffers.destroy(); + } + } + } + @Test public void testSerializerClearedAfterClearBuffers() throws Exception { From 5322e87061d11f532657f9c7a5e144fca3748201 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 26 Aug 2015 10:36:03 +0200 Subject: [PATCH 169/175] [FLINK-2538] Close created JarFile in ClassLoaderUtil --- .../org/apache/flink/runtime/util/ClassLoaderUtil.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java index fbb707ee9b99e..c0145b5988287 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ClassLoaderUtil.java @@ -72,13 +72,19 @@ else if ("file".equals(url.getProtocol())) { bld.append(" (directory)"); } else { + JarFile jar = null; try { - new JarFile(filePath); + jar = new JarFile(filePath); bld.append(" (valid JAR)"); } catch (Exception e) { bld.append(" (invalid JAR: ").append(e.getMessage()).append(')'); } + finally { + if (jar != null) { + jar.close(); + } + } } } else { From 5a9daca44d998338ab4815b5cf35c99936fcad5f Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 26 Aug 2015 10:55:20 +0200 Subject: [PATCH 170/175] [FLINK-2270] [docs] Fix typo in docs (enableMonitoring => enableCheckpointing) --- docs/apis/streaming_guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index 5ffe432544f56..a1b6ac7cd5681 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -1597,7 +1597,7 @@ To use fault tolerant Kafka Sources, monitoring of the topology needs to be enab

    {% highlight java %} final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -env.enableMonitoring(5000); +env.enableCheckpointing(5000); {% endhighlight %}
    From a17d4e823fd08e8e49fde169dc3ddd264964d85a Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Mon, 10 Aug 2015 15:15:07 +0200 Subject: [PATCH 171/175] [FLINK-2460] [runtime] Check parent state in isReleased() check of partition view - Address PR comments This closes #1051. --- .../partition/PipelinedSubpartition.java | 7 +++- .../network/partition/ResultSubpartition.java | 2 + .../partition/SpillableSubpartition.java | 7 +++- .../partition/SpillableSubpartitionView.java | 4 +- .../SpilledSubpartitionViewAsyncIO.java | 2 +- .../SpilledSubpartitionViewSyncIO.java | 2 +- .../partition/PipelinedSubpartitionTest.java | 4 ++ .../partition/SubpartitionTestBase.java | 42 +++++++++++++++++++ 8 files changed, 64 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index 931790a03eefb..3b7a2a6385b03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -41,7 +41,7 @@ class PipelinedSubpartition extends ResultSubpartition { private boolean isFinished; /** Flag indicating whether the subpartition has been released. */ - private boolean isReleased; + private volatile boolean isReleased; /** * A data availability listener. Registered, when the consuming task is faster than the @@ -166,6 +166,11 @@ public int releaseMemory() { return 0; } + @Override + public boolean isReleased() { + return isReleased; + } + @Override public PipelinedSubpartitionView createReadView(BufferProvider bufferProvider) { synchronized (buffers) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java index e9dfe3213164f..b7ca9c4af569c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java @@ -81,4 +81,6 @@ protected Throwable getFailureCause() { abstract int releaseMemory() throws IOException; + abstract public boolean isReleased(); + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java index 4a18691b7b0c2..21e9cc6904c91 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -59,7 +59,7 @@ class SpillableSubpartition extends ResultSubpartition { private boolean isFinished; /** Flag indicating whether the subpartition has been released. */ - boolean isReleased; + private volatile boolean isReleased; /** The read view to consume this subpartition. */ private ResultSubpartitionView readView; @@ -167,6 +167,11 @@ public int releaseMemory() throws IOException { return 0; } + @Override + public boolean isReleased() { + return isReleased; + } + @Override public ResultSubpartitionView createReadView(BufferProvider bufferProvider) throws IOException { synchronized (buffers) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index 972e34bc07535..c9da40a33ef66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -73,7 +73,7 @@ public Buffer getNextBuffer() throws IOException, InterruptedException { // 1) In-memory synchronized (parent.buffers) { - if (parent.isReleased) { + if (parent.isReleased()) { return null; } @@ -162,7 +162,7 @@ public void releaseAllResources() throws IOException { @Override public boolean isReleased() { - return isReleased.get(); + return parent.isReleased() || isReleased.get(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java index ea5c20b740a17..052a7cd7a9166 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java @@ -187,7 +187,7 @@ public void releaseAllResources() throws IOException { @Override public boolean isReleased() { - return isReleased; + return parent.isReleased() || isReleased; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java index 24099a78abfaa..5b91668e8107f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java @@ -108,7 +108,7 @@ public void releaseAllResources() throws IOException { @Override public boolean isReleased() { - return isReleased.get(); + return parent.isReleased() || isReleased.get(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 8750a1a1e1c2b..652006629118a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -18,11 +18,14 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.api.common.JobID; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.util.TestBufferFactory; import org.apache.flink.runtime.io.network.util.TestConsumerCallback; import org.apache.flink.runtime.io.network.util.TestNotificationListener; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; @@ -36,6 +39,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode.ASYNC; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index cb0069b10e517..26a8f2904f432 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -19,10 +19,15 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.util.TestBufferFactory; +import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider; import org.apache.flink.util.TestLogger; import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; /** @@ -68,4 +73,41 @@ public void testAddAfterRelease() throws Exception { } } } + + @Test + public void testReleaseParent() throws Exception { + final ResultSubpartition partition = createSubpartition(); + verifyViewReleasedAfterParentRelease(partition); + } + + @Test + public void testReleaseParentAfterSpilled() throws Exception { + final ResultSubpartition partition = createSubpartition(); + partition.releaseMemory(); + + verifyViewReleasedAfterParentRelease(partition); + } + + private void verifyViewReleasedAfterParentRelease(ResultSubpartition partition) throws Exception { + // Add a buffer + Buffer buffer = TestBufferFactory.createBuffer(); + partition.add(buffer); + partition.finish(); + + TestInfiniteBufferProvider buffers = new TestInfiniteBufferProvider(); + + // Create the view + ResultSubpartitionView view = partition.createReadView(buffers); + + // The added buffer and end-of-partition event + assertNotNull(view.getNextBuffer()); + assertNotNull(view.getNextBuffer()); + + // Release the parent + assertFalse(view.isReleased()); + partition.release(); + + // Verify that parent release is reflected at partition view + assertTrue(view.isReleased()); + } } From 0cca527f769aeb3f89fa44cb8cdfe5410e552289 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 26 Aug 2015 10:56:27 +0200 Subject: [PATCH 172/175] [hotfix] Fixes auto type registration settings in ExecutionConfig --- .../java/org/apache/flink/api/common/ExecutionConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index b8fa2a2aaa72a..d93eba6844805 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -78,7 +78,7 @@ public class ExecutionConfig implements Serializable { private boolean objectReuse = false; - private boolean disableAutoTypeRegistration = false; + private boolean autoTypeRegistrationEnabled = true; private boolean forceAvro = false; @@ -618,7 +618,7 @@ public LinkedHashSet> getRegisteredPojoTypes() { public boolean isAutoTypeRegistrationDisabled() { - return disableAutoTypeRegistration; + return !autoTypeRegistrationEnabled; } /** @@ -627,7 +627,7 @@ public boolean isAutoTypeRegistrationDisabled() { * */ public void disableAutoTypeRegistration() { - this.disableAutoTypeRegistration = false; + this.autoTypeRegistrationEnabled = false; } From 438bb69bdbd8a1a4ce9090b64fe61f879f4ddc68 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 22 Jul 2015 20:53:11 +0300 Subject: [PATCH 173/175] Solved the #887 issue: removing the akka.framesize size limitation for the result of a job. --- .../apache/flink/client/program/Client.java | 5 +- .../accumulators/AccumulatorHelper.java | 17 ++ .../flink/configuration/ConfigConstants.java | 8 + flink-examples/flink-java-examples/pom.xml | 2 +- .../accumulators/AccumulatorRegistry.java | 43 ++- .../accumulators/AccumulatorSnapshot.java | 40 ++- .../accumulators/LargeAccumulatorHelper.java | 257 ++++++++++++++++++ .../accumulators/UserAccumulators.java | 86 ++++++ .../runtime/blob/BlobServerProtocol.java | 2 +- .../flink/runtime/client/JobClient.java | 72 +++-- .../client/SerializedJobExecutionResult.java | 87 +++++- .../librarycache/BlobLibraryCacheManager.java | 14 +- .../FallbackLibraryCacheManager.java | 12 +- .../librarycache/LibraryCacheManager.java | 14 + .../runtime/executiongraph/Execution.java | 39 ++- .../executiongraph/ExecutionGraph.java | 221 +++++++++++---- .../flink/runtime/taskmanager/Task.java | 34 ++- .../flink/runtime/util/SerializedValue.java | 22 ++ .../apache/flink/runtime/akka/AkkaUtils.scala | 16 ++ .../flink/runtime/jobmanager/JobManager.scala | 103 +++++-- .../runtime/jobmanager/MemoryArchivist.scala | 2 +- .../runtime/taskmanager/TaskManager.scala | 31 +-- .../SerializedJobExecutionResultTest.java | 5 +- .../testingUtils/TestingJobManager.scala | 2 +- .../test/misc/MiscellaneousIssuesITCase.java | 53 +++- 25 files changed, 1014 insertions(+), 173 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 78c82f6dc835c..28ad4709e1b69 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -418,10 +418,11 @@ public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramIn try{ if (wait) { - SerializedJobExecutionResult result = JobClient.submitJobAndWait(actorSystem, + SerializedJobExecutionResult partialResult = JobClient.submitJobAndWait(actorSystem, jobManagerGateway, jobGraph, timeout, printStatusDuringExecution); try { - return result.toJobExecutionResult(this.userCodeClassLoader); + return JobClient.returnFinalJobExecutionResult(jobManagerGateway, partialResult, + this.userCodeClassLoader, timeout); } catch (Exception e) { throw new ProgramInvocationException( diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java index 3907004369f66..16ca096df9912 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java @@ -51,6 +51,23 @@ public static void mergeInto(Map> target, Map target, Accumulator toMerge) { + // Both should have the same type + AccumulatorHelper.compareAccumulatorTypes(name, target.getClass(), toMerge.getClass()); + // Merge counter from chained task into counter from stub + mergeSingle(target, toMerge); + } + /** * Workaround method for type safety */ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index d145eb24e0abb..304ec547cc2fd 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -421,6 +421,12 @@ public final class ConfigConstants { * Exit JVM on fatal Akka errors */ public static final String AKKA_JVM_EXIT_ON_FATAL_ERROR = "akka.jvm-exit-on-fatal-error"; + + /** + * The threshold (in bytes) above which an object is considered too big + * to transfer using akka + */ + public static final String AKKA_FRAMESIZE_OVERSIZED_THRESHOLD = "akka.framesize.oversized.threshold"; // ----------------------------- Streaming -------------------------------- @@ -675,6 +681,8 @@ public final class ConfigConstants { public static String DEFAULT_AKKA_ASK_TIMEOUT = "100 s"; public static String DEFAULT_AKKA_LOOKUP_TIMEOUT = "10 s"; + + public static double DEFAULT_AKKA_FRAMESIZE_OVERSIZED_THRESHOLD = 0.8; // ----------------------------- Streaming Values -------------------------- diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml index 8448a93db434d..33190abbf9563 100644 --- a/flink-examples/flink-java-examples/pom.xml +++ b/flink-examples/flink-java-examples/pom.xml @@ -319,7 +319,7 @@ under the License.
    - +
    diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java index 0ef3650292b48..7ba00d6cc4487 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java @@ -21,16 +21,21 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; - /** * Main accumulator registry which encapsulates internal and user-defined accumulators. */ @@ -38,6 +43,8 @@ public class AccumulatorRegistry { protected static final Logger LOG = LoggerFactory.getLogger(AccumulatorRegistry.class); + protected final Configuration jobConfiguration; + protected final InetSocketAddress blobServerAddress; protected final JobID jobID; protected final ExecutionAttemptID taskID; @@ -62,22 +69,47 @@ public enum Metric { NUM_BYTES_OUT } - public AccumulatorRegistry(JobID jobID, ExecutionAttemptID taskID) { + this(null, jobID, taskID, null); + } + + public AccumulatorRegistry(Configuration jobConfig, JobID jobID, ExecutionAttemptID taskID, InetSocketAddress blobServerAddress) { + this.jobConfiguration = jobConfig; this.jobID = jobID; this.taskID = taskID; + this.blobServerAddress = blobServerAddress; this.reporter = new ReadWriteReporter(flinkAccumulators); } /** - * Creates a snapshot of this accumulator registry. + * Creates a snapshot of this accumulator registry. If they are oversized (i.e. bigger than + * akka.framesize), this method stores them in the BlobCache and sends only the + * corresponding BlobKeys in the final snapshot. If they are not, it sends the actual + * accumulators in the accumulator snapshot. + * * @return a serialized accumulator map */ public AccumulatorSnapshot getSnapshot() { + AccumulatorSnapshot snapshot; + Map> largeAccumulatorBlobKeys; + SerializedValue>> serializedAccumulators; + try { - return new AccumulatorSnapshot(jobID, taskID, flinkAccumulators, userAccumulators); + serializedAccumulators = new SerializedValue>>(userAccumulators); + + if (serializedAccumulators.getSizeInBytes() > AkkaUtils.getLargeAccumulatorThreshold(jobConfiguration)) { + largeAccumulatorBlobKeys = LargeAccumulatorHelper. + storeAccumulatorsToBlobCache(blobServerAddress, userAccumulators); + + snapshot = new AccumulatorSnapshot(jobID, taskID, + flinkAccumulators, largeAccumulatorBlobKeys); + } else { + snapshot = new AccumulatorSnapshot(jobID, taskID, + flinkAccumulators, serializedAccumulators); + } + return snapshot; } catch (IOException e) { - LOG.warn("Failed to serialize accumulators for task.", e); + LOG.warn("Error while creating a snapshot of current results: "+ e.getMessage()); return null; } } @@ -143,5 +175,4 @@ public void reportNumBytesOut(long value) { numBytesOut.add(value); } } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java index 0f1911db2f137..85fbad9d07e7f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java @@ -20,17 +20,24 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.util.SerializedValue; import java.io.IOException; import java.io.Serializable; +import java.util.List; import java.util.Map; /** - * This class encapsulates a map of accumulators for a single task. It is used - * for the transfer from TaskManagers to the JobManager and from the JobManager - * to the Client. + * This class encapsulates a map of accumulators referring to a single task. + * These accumulators come in two flavors: + *
      + *
    • system-defined accumulators containing system metrics for the task, and
    • + *
    • user-defined ones, containing the results of the task.
    • + *
    + * This class is used for the transfer these accumulators from TaskManagers to the JobManager, + * and from the JobManager to the Client. */ public class AccumulatorSnapshot implements Serializable { @@ -45,17 +52,26 @@ public class AccumulatorSnapshot implements Serializable { private final SerializedValue>> flinkAccumulators; /** - * Serialized user accumulators which may require the custom user class loader. + * User defined accumulators that may require user defined classloader. */ - private final SerializedValue>> userAccumulators; + private final UserAccumulators userAccumulators; public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, - Map> flinkAccumulators, - Map> userAccumulators) throws IOException { + Map> flinkAccumulators, + Map> oversizedUserAccumulatorBlobKeys) throws IOException { this.jobID = jobID; this.executionAttemptID = executionAttemptID; this.flinkAccumulators = new SerializedValue>>(flinkAccumulators); - this.userAccumulators = new SerializedValue>>(userAccumulators); + this.userAccumulators = new UserAccumulators(oversizedUserAccumulatorBlobKeys); + } + + public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, + Map> flinkAccumulators, + SerializedValue>> smallUserAccumulators) throws IOException { + this.jobID = jobID; + this.executionAttemptID = executionAttemptID; + this.flinkAccumulators = new SerializedValue>>(flinkAccumulators); + this.userAccumulators = new UserAccumulators(smallUserAccumulators); } public JobID getJobID() { @@ -75,10 +91,10 @@ public ExecutionAttemptID getExecutionAttemptID() { } /** - * Gets the user-defined accumulators values. - * @return the serialized map + * Gets the User-defined accumulators. + * @return the user accumulators. */ - public Map> deserializeUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { - return userAccumulators.deserializeValue(classLoader); + public UserAccumulators getUserAccumulators() { + return userAccumulators; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java new file mode 100644 index 0000000000000..2d5550f38b5fd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java @@ -0,0 +1,257 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.blob.BlobServerProtocol; +import org.apache.flink.runtime.util.SerializedValue; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This class contains general methods that help at handling oversized accumulators + * that are stored and fetched from the BlobCache, instead of being sent directly + * through akka and stored in memory. + * */ +public class LargeAccumulatorHelper { + + /** + * When the result of the job contains oversized (i.e. bigger than + * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( + org.apache.flink.configuration.Configuration)} bytes) + * accumulators then these are put in the BlobCache for the client to fetch and merge. + * This method serializes and stores the large accumulators in the BlobCache. + * + * @param blobServerAddress the address of the server to the blobCache. + * @param accumulators the accumulators to be stored in the cache. + * @return the name of each accumulator with the associated BlobKey that identifies + * its blob in the BlobCache. + */ + public static Map> storeAccumulatorsToBlobCache( + InetSocketAddress blobServerAddress, + Map> accumulators) throws IOException { + + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> serializedAccumulators = serializeAccumulators(accumulators); + return storeSerializedAccumulatorsToBlobCache(blobServerAddress, serializedAccumulators); + } + + /** + * When the result of the job contains oversized (i.e. bigger than + * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( + org.apache.flink.configuration.Configuration)} bytes) + * accumulators then these are put in the BlobCache for the client to fetch and merge. + * This method stores the blobs of the large accumulators in the BlobCache. Contrary to + * {@link LargeAccumulatorHelper#storeAccumulatorsToBlobCache(InetSocketAddress, Map)}, this + * method assumes that accumulators are already serialized. + * + * @param blobServerAddress the address of the server to the blobCache. + * @param accumulators a map with the names and the (serialized) accumulators to be + * stored in the BlobCache. + * @return the name of each accumulator with the associated BlobKey that identifies + * its blob in the BlobCache. + */ + public static Map> storeSerializedAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, + Map> accumulators) throws IOException { + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> keys = new HashMap>(); + + BlobClient bc = null; + try { + bc = new BlobClient(blobServerAddress); + + for (Map.Entry> entry : accumulators.entrySet()) { + + String accumulatorName = entry.getKey(); + byte[] accumulatorPayload = entry.getValue().getSerializedData(); + if (accumulatorPayload != null) { + BlobKey blobKey = bc.put(accumulatorPayload); + List accKeys = keys.get(accumulatorName); + if (accKeys == null) { + accKeys = new ArrayList(); + } + accKeys.add(blobKey); + keys.put(accumulatorName, accKeys); + } + } + } catch (IOException e) { + throw new IOException("Failed to send oversized accumulators to the BlobCache: ", e); + } finally { + try { + if (bc != null) { + bc.close(); + } + } catch (IOException e) { + throw new IOException("Failed to close BlobClient: ", e); + } + + } + return keys; + } + + /** + * When the result of the job contains oversized (i.e. bigger than + * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( + org.apache.flink.configuration.Configuration)} bytes) + * accumulators then these are put in the BlobCache for the client to fetch and merge. + * This method gets, deserializes, and merges these oversized user-defined accumulators. + * + * @param blobServerAddress the address that the BlobCache is listening to. + * @param keys the blob keys to fetch. + * @param loader the classloader used to deserialize the accumulators fetched. + * @return the accumulators, grouped by name. + * */ + public static Map> getDeserializeAndMergeAccumulatorsFromBlobCache( + InetSocketAddress blobServerAddress, Map> keys, ClassLoader loader) + throws IOException, ClassNotFoundException { + + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + + if (keys.isEmpty()) { + return Collections.emptyMap(); + } + + Map> accumulators = + new HashMap>(); + + Map>> accumulatorBlobs = + getSerializedAccumulatorsFromBlobCache(blobServerAddress, keys); + + for (String accumulatorName : accumulatorBlobs.keySet()) { + Accumulator existing = accumulators.get(accumulatorName); + + for (SerializedValue acc : accumulatorBlobs.get(accumulatorName)) { + Accumulator accumulator = (Accumulator) acc.deserializeValue(loader); + if(existing == null) { + existing = accumulator; + accumulators.put(accumulatorName, existing); + } else { + AccumulatorHelper.mergeAccumulators(accumulatorName, existing, accumulator); + } + } + } + return accumulators; + } + + /** + * When the result of the job contains oversized (i.e. bigger than + * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( + org.apache.flink.configuration.Configuration)} bytes) + * accumulators then these are put in the BlobCache for the client to fetch and merge. + * This methos gets the user-defined accumulators from the BlobCache and returns them in + * serialized form. Contrary to + * {@link LargeAccumulatorHelper#getDeserializeAndMergeAccumulatorsFromBlobCache( + InetSocketAddress, Map, ClassLoader)}, this method does nothing more than fetching + * the serialized data. + * + * @param blobServerAddress the address that the BlobCache is listening to. + * @param keys the blob keys to fetch. + * @return the accumulators, grouped by name. + */ + public static Map>> getSerializedAccumulatorsFromBlobCache( + InetSocketAddress blobServerAddress, Map> keys) throws IOException { + + if (blobServerAddress == null) { + throw new RuntimeException("Undefined Blob Server Address."); + } + + if (keys.isEmpty()) { + return Collections.emptyMap(); + } + + Map>> accumulatorBlobs = + new HashMap>>(); + + BlobClient bc = new BlobClient(blobServerAddress); + + final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE]; + for (String accName : keys.keySet()) { + List accBlobKeys = keys.get(accName); + List> accBlobs = new ArrayList>(); + + for (BlobKey bk : accBlobKeys) { + InputStream is = bc.get(bk); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + while (true) { + final int read = is.read(buf); + if (read < 0) { + break; + } + os.write(buf, 0, read); + } + os.flush(); + byte[] blob = os.toByteArray(); + accBlobs.add(new SerializedValue(blob)); + is.close(); + os.close(); + + // after getting them, clean up and delete the blobs from the BlobCache. + bc.delete(bk); + } + accumulatorBlobs.put(accName, accBlobs); + } + bc.close(); + return accumulatorBlobs; + } + + private static Map> serializeAccumulators(Map> accumulators) throws IOException { + if (accumulators.isEmpty()) { + return Collections.emptyMap(); + } + + Map> serializedAccumulators = + new HashMap>(); + try { + for (Map.Entry> entry : accumulators.entrySet()) { + String accumulatorName = entry.getKey(); + Accumulator accumulator = entry.getValue(); + serializedAccumulators.put(accumulatorName, new SerializedValue(accumulator)); + } + } catch (IOException e) { + throw new IOException("Failed to serialize accumulators.", e); + } + return serializedAccumulators; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java new file mode 100644 index 0000000000000..5e761e145a6f5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/UserAccumulators.java @@ -0,0 +1,86 @@ +/* + * 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.runtime.accumulators; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.util.SerializedValue; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * This class contains the user-defined accumulators of a task. + * Given that Flink handles differently accumulators that are bigger than the + * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( + org.apache.flink.configuration.Configuration)}-byte threshold, this class can + * accommodate both accumulator types, i.e. big and small. + * + * In fact, given that the accumulators of a task can either be small or oversized, this + * class will contain either one or the other type, as it can be seen by its constructors. + * */ +public class UserAccumulators implements java.io.Serializable { + + /** Serialized user accumulators which may require the custom user class loader. */ + private final SerializedValue>> smallUserAccumulators; + + /** + * In case some accumulators do not fit in an Akka message payload, we store them + * in the blobCache. This list holds the mapping between the name of the accumulator, + * and its blobKey in the cache is put in the snapshot. This list holds exactly this + * mapping. + * */ + private final Map> largeUserAccumulatorBlobs; + + public UserAccumulators(Map> oversizedUserAccumulatorBlobKeys) throws IOException { + this.smallUserAccumulators = null; + this.largeUserAccumulatorBlobs = oversizedUserAccumulatorBlobKeys; + } + + + public UserAccumulators(SerializedValue>> userAccumulators) throws IOException { + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulatorBlobs = null; + } + + /** + * Gets the user-defined accumulators values that fit in akka payload. + * @return the serialized map + */ + public Map> deserializeSmallUserAccumulators(ClassLoader classLoader) throws IOException, ClassNotFoundException { + if(largeUserAccumulatorBlobs != null) { + return Collections.emptyMap(); + } + return smallUserAccumulators.deserializeValue(classLoader); + } + + /** + * Gets the BlobKeys of the oversized accumulators that were too big to be sent through akka. + * These accumulators had to be stored in the BlobCache and their blobKeys are returned here. + * @return the maping between accumulator and its blobKeys. + */ + public Map> getLargeUserAccumulatorBlobKeys() { + if(smallUserAccumulators != null) { + return Collections.emptyMap(); + } + return largeUserAccumulatorBlobs; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java index 6df78110b9a2c..c5bcce52b1594 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java @@ -24,7 +24,7 @@ public class BlobServerProtocol { // -------------------------------------------------------------------------------------------- /** The buffer size in bytes for network transfers. */ - static final int BUFFER_SIZE = 65536; // 64 K + public static final int BUFFER_SIZE = 65536; // 64 K /** The maximum key length allowed for storing BLOBs. */ static final int MAX_KEY_LENGTH = 64; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index aeefa612ba4f9..c7020d96cfcb6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -27,10 +27,14 @@ import akka.pattern.Patterns; import akka.util.Timeout; import com.google.common.base.Preconditions; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.accumulators.LargeAccumulatorHelper; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.JobClientMessages; @@ -50,6 +54,8 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeoutException; /** @@ -187,9 +193,32 @@ public static SerializedJobExecutionResult submitJobAndWait( } } + public static JobExecutionResult returnFinalJobExecutionResult( + ActorGateway jobManagerGateway, SerializedJobExecutionResult partialResult, + ClassLoader userCodeClassLoader, FiniteDuration timeout) + throws IOException, ClassNotFoundException { + + if (jobManagerGateway == null || partialResult == null || userCodeClassLoader == null || timeout == null) { + throw new NullPointerException(); + } + + Map> largeAccumulators; + try { + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + Map> blobsKeysToFetch = partialResult.getBlobKeysToLargeAccumulators(); + + largeAccumulators = LargeAccumulatorHelper.getDeserializeAndMergeAccumulatorsFromBlobCache( + serverAddress, blobsKeysToFetch, userCodeClassLoader); + + } catch (IOException e) { + throw new IOException("Failed to fetch the oversized accumulators from the BlobCache", e); + } + return partialResult.mergeToJobExecutionResult(userCodeClassLoader, largeAccumulators); + } + /** * Submits a job in detached mode. The method sends the JobGraph to the - * JobManager and waits for the answer whether teh job could be started or not. + * JobManager and waits for the answer whether the job could be started or not. * * @param jobManagerGateway Gateway to the JobManager which will execute the jobs * @param jobGraph The job @@ -250,29 +279,30 @@ public static void uploadJarFiles( FiniteDuration timeout) throws IOException { if (jobGraph.hasUsercodeJarFiles()) { + InetSocketAddress serverAddress = getBlobServerAddress(jobManagerGateway, timeout); + jobGraph.uploadRequiredJarFiles(serverAddress); + } + } - Future futureBlobPort = jobManagerGateway.ask( - JobManagerMessages.getRequestBlobManagerPort(), - timeout); + private static InetSocketAddress getBlobServerAddress(ActorGateway jobManagerGateway, FiniteDuration timeout) throws IOException { + Future futureBlobPort = jobManagerGateway.ask( + JobManagerMessages.getRequestBlobManagerPort(), + timeout); - int port; - try { - Object result = Await.result(futureBlobPort, timeout); - if (result instanceof Integer) { - port = (Integer) result; - } else { - throw new Exception("Expected port number (int) as answer, received " + result); - } - } - catch (Exception e) { - throw new IOException("Could not retrieve the JobManager's blob port.", e); + int port; + try { + Object result = Await.result(futureBlobPort, timeout); + if (result instanceof Integer) { + port = (Integer) result; + } else { + throw new Exception("Expected port number (int) as answer, received " + result); } - - Option jmHost = jobManagerGateway.actor().path().address().host(); - String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; - InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, port); - - jobGraph.uploadRequiredJarFiles(serverAddress); + } catch (Exception e) { + throw new IOException("Could not retrieve the JobManager's blob port.", e); } + + Option jmHost = jobManagerGateway.actor().path().address().host(); + String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost"; + return new InetSocketAddress(jmHostname, port); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java index 029bc3876f738..d8b88e83a20cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java @@ -20,11 +20,14 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.util.SerializedValue; import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -38,7 +41,13 @@ public class SerializedJobExecutionResult implements java.io.Serializable { private final JobID jobId; - private final Map> accumulatorResults; + // holds the result value of an accumulator + private final Map> smallAccumulatorResults; + + // holds the BlobKeys pointing to the blobs in the BlobCache + // containing the serialized accumulators + private final Map> largeAccumulatorBlobRefs; + private final long netRuntime; @@ -47,13 +56,17 @@ public class SerializedJobExecutionResult implements java.io.Serializable { * * @param jobID The job's ID. * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer) in milliseconds - * @param accumulators A map of all accumulator results produced by the job, in serialized form + * @param smallAccumulators A map of all SMALL accumulator results produced by the job, in serialized form + * @param largeAccumulatorBlobRefs A map of all the references to the blobs holding the LARGE accumulators that had + * to be sent through the BlobCache. */ public SerializedJobExecutionResult(JobID jobID, long netRuntime, - Map> accumulators) { + Map> smallAccumulators, + Map> largeAccumulatorBlobRefs) { this.jobId = jobID; this.netRuntime = netRuntime; - this.accumulatorResults = accumulators; + this.smallAccumulatorResults = smallAccumulators; + this.largeAccumulatorBlobRefs = largeAccumulatorBlobRefs; } public JobID getJobId() { @@ -76,17 +89,28 @@ public long getNetRuntime(TimeUnit desiredUnit) { } public Map> getSerializedAccumulatorResults() { - return this.accumulatorResults; + return this.smallAccumulatorResults; + } + + /** + * If the result of the job contained oversized (i.e. bigger that the akka.framesize) accumulators + * then these are put in the BlobCache for the client to fetch and merge. This method gets + * their BlobKeys. If the result is empty, then this means that all (partial) accumulators + * were small enough to be sent directly to the JobManager and be merged there. + * @return the BlobKeys to the blobs containing the oversized accumulators. + * */ + public Map> getBlobKeysToLargeAccumulators() { + return this.largeAccumulatorBlobRefs; } public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOException, ClassNotFoundException { Map accumulators = null; - if (accumulatorResults != null) { - accumulators = accumulatorResults.isEmpty() ? + if (smallAccumulatorResults != null) { + accumulators = smallAccumulatorResults.isEmpty() ? Collections.emptyMap() : - new HashMap(this.accumulatorResults.size()); + new HashMap(this.smallAccumulatorResults.size()); - for (Map.Entry> entry : this.accumulatorResults.entrySet()) { + for (Map.Entry> entry : this.smallAccumulatorResults.entrySet()) { Object o = entry.getValue() == null ? null : entry.getValue().deserializeValue(loader); accumulators.put(entry.getKey(), o); } @@ -94,4 +118,49 @@ public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOExce return new JobExecutionResult(jobId, netRuntime, accumulators); } + + /** + * Merges the data of the small accumulators in this class, with the accumulatorsToMerge, and + * returns the result in a JobExecutionResult. This method is used by the Client to merge + * the results received from the JobManager, with those in the oversized accumulators that were fetched + * from the BlobCache. + * @param loader + * the ClassLoader to used to deserialize the data and the Accumulators. + * @param accumulatorsToMerge + * the Accumulators received from the BlobCache as blobs, that are to be merged with local data. + * @return the final result after the merging of the different partial accumulators. + * */ + public JobExecutionResult mergeToJobExecutionResult(ClassLoader loader, Map> accumulatorsToMerge) throws IOException, ClassNotFoundException { + if(accumulatorsToMerge == null || accumulatorsToMerge.isEmpty()) { + return toJobExecutionResult(loader); + } + + Map accumulators = new HashMap(); + + for(String name: accumulatorsToMerge.keySet()) { + Accumulator acc = accumulatorsToMerge.get(name); + + // add also the data from the non-oversized (i.e. the ones that were sent through akka) + // accumulators, if any + if (smallAccumulatorResults != null) { + SerializedValue localObject = smallAccumulatorResults.remove(name); + if (localObject != null) { + acc.add(localObject.deserializeValue(loader)); + } + } + + // and put the data with the associated accumulator name to the list + accumulators.put(name, acc.getLocalValue()); + } + + // finally, put the remaining accumulators in the list. + if (smallAccumulatorResults != null) { + for (Map.Entry> entry : this.smallAccumulatorResults.entrySet()) { + Object o = entry.getValue() == null ? null : entry.getValue().deserializeValue(loader); + accumulators.put(entry.getKey(), o); + } + } + return new JobExecutionResult(jobId, netRuntime, accumulators); + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 88be5e1f37735..16262a42e32cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -138,7 +138,16 @@ public void registerTask(JobID jobId, ExecutionAttemptID task, Collection> userAccumulators; + private volatile Map> smallUserAccumulators; + + /* Keep track of the large accumulators that are stored in the BlobCache */ + private volatile Map> largeUserAccumulators; /* Continuously updated map of internal accumulators */ private volatile Map> flinkAccumulators; @@ -603,12 +607,14 @@ void markFailed(Throwable t) { } void markFinished() { - markFinished(null, null); + markFinished(null, null, null); } - void markFinished(Map> flinkAccumulators, Map> userAccumulators) { + void markFinished(Map> flinkAccumulators, + Map> userAccumulators, + Map> userLargeAccumulators) { - // this call usually comes during RUNNING, but may also come while still in deploying (very fast tasks!) + // this call usually comes during RUNNING, but may also come while still in DEPLOYING (very fast tasks!) while (true) { ExecutionState current = this.state; @@ -629,7 +635,8 @@ void markFinished(Map> flinkAccumu synchronized (accumulatorLock) { this.flinkAccumulators = flinkAccumulators; - this.userAccumulators = userAccumulators; + this.smallUserAccumulators = userAccumulators; + this.largeUserAccumulators = userLargeAccumulators; } assignedResource.releaseSlot(); @@ -863,8 +870,7 @@ public void onComplete(Throwable failure, Object success) throws Throwable { } else { TaskOperationResult result = (TaskOperationResult) success; if (!result.success()) { - LOG.debug("Cancel task call did not find task. Probably akka message call" + - " race."); + LOG.debug("Cancel task call did not find task. Probably akka message call race."); } } } @@ -957,19 +963,28 @@ public String getVertexWithAttempt() { /** * Update accumulators (discarded when the Execution has already been terminated). * @param flinkAccumulators the flink internal accumulators - * @param userAccumulators the user accumulators + * @param smallUserAccumulators the user accumulators + * @param largeUserAccumulatorBlobKeys the keys to the blobs storing the oversized accumulators sent by + * (some) tasks. */ public void setAccumulators(Map> flinkAccumulators, - Map> userAccumulators) { + Map> smallUserAccumulators, + Map> largeUserAccumulatorBlobKeys) { synchronized (accumulatorLock) { if (!state.isTerminal()) { this.flinkAccumulators = flinkAccumulators; - this.userAccumulators = userAccumulators; + this.smallUserAccumulators = smallUserAccumulators; + this.largeUserAccumulators = largeUserAccumulatorBlobKeys; } } } - public Map> getUserAccumulators() { - return userAccumulators; + + public Map> getSmallUserAccumulators() { + return smallUserAccumulators; + } + + public Map> getLargeUserAccumulatorBlobKeys() { + return largeUserAccumulators; } public Map> getFlinkAccumulators() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index b52a4e814229d..c3cb2b54a9af9 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.accumulators.UserAccumulators; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.execution.ExecutionState; @@ -60,6 +61,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.Set; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -92,7 +95,7 @@ * about deployment of tasks and updates in the task status always use the ExecutionAttemptID to * address the message receiver. * - * + * *

    The ExecutionGraph implements {@link java.io.Serializable}, because it can be archived by * sending it to an archive actor via an actor message. The execution graph does contain some * non-serializable fields. These fields are not required in the archived form and are cleared @@ -107,7 +110,7 @@ public class ExecutionGraph implements Serializable { /** The log object used for debugging. */ static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class); - + private static final int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1; // -------------------------------------------------------------------------------------------- @@ -115,7 +118,7 @@ public class ExecutionGraph implements Serializable { /** The lock used to secure all access to mutable fields, especially the tracking of progress * within the job. */ private final SerializableObject progressLock = new SerializableObject(); - + /** The ID of the job this graph has been built for. */ private final JobID jobID; @@ -177,7 +180,7 @@ public class ExecutionGraph implements Serializable { /** Flag that indicate whether the executed dataflow should be periodically snapshotted */ private boolean snapshotCheckpointsEnabled; - + // ------ Execution status and progress. These values are volatile, and accessed under the lock ------- @@ -190,8 +193,8 @@ public class ExecutionGraph implements Serializable { /** The number of job vertices that have reached a terminal state */ private volatile int numFinishedJobVertices; - - + + // ------ Fields that are relevant to the execution and need to be cleared before archiving ------- /** The scheduler to use for scheduling new tasks as they are needed */ @@ -201,7 +204,7 @@ public class ExecutionGraph implements Serializable { /** The classloader for the user code. Needed for calls into user code classes */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private ClassLoader userClassLoader; - + /** The coordinator for checkpoints, if snapshot checkpoints are enabled */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private CheckpointCoordinator checkpointCoordinator; @@ -336,10 +339,10 @@ public void enableSnapshotCheckpointing( ExecutionVertex[] tasksToTrigger = collectExecutionVertices(verticesToTrigger); ExecutionVertex[] tasksToWaitFor = collectExecutionVertices(verticesToWaitFor); ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo); - + // disable to make sure existing checkpoint coordinators are cleared disableSnaphotCheckpointing(); - + // create the coordinator that triggers and commits checkpoints and holds the state snapshotCheckpointsEnabled = true; checkpointCoordinator = new CheckpointCoordinator( @@ -350,7 +353,7 @@ public void enableSnapshotCheckpointing( tasksToWaitFor, tasksToCommitTo, userClassLoader); - + // the periodic checkpoint scheduler is activated and deactivated as a result of // job status changes (running -> on, all other states -> off) registerJobStatusListener( @@ -359,19 +362,19 @@ public void enableSnapshotCheckpointing( interval, leaderSessionID)); } - + public void disableSnaphotCheckpointing() { if (state != JobStatus.CREATED) { throw new IllegalStateException("Job must be in CREATED state"); } - + snapshotCheckpointsEnabled = false; if (checkpointCoordinator != null) { checkpointCoordinator.shutdown(); checkpointCoordinator = null; } } - + public boolean isSnapshotCheckpointsEnabled() { return snapshotCheckpointsEnabled; } @@ -403,7 +406,7 @@ private ExecutionVertex[] collectExecutionVertices(List jobV // -------------------------------------------------------------------------------------------- // Properties and Status of the Execution Graph // -------------------------------------------------------------------------------------------- - + /** * Returns a list of BLOB keys referring to the JAR files required to run this job * @return list of BLOB keys referring to the JAR files required to run this job @@ -463,7 +466,7 @@ public Iterable getVerticesTopologically() { // we return a specific iterator that does not fail with concurrent modifications // the list is append only, so it is safe for that final int numElements = this.verticesInCreationOrder.size(); - + return new Iterable() { @Override public Iterator iterator() { @@ -535,54 +538,140 @@ public Map> return flinkAccumulators; } + /** + * This works as cache for already merged accumulators. If accumulators are already + * merged, we do not want to re-merge them, as this may lead to duplicate entries. + */ + private Map> mergedSmallUserAccumulators; + /** * Merges all accumulator results from the tasks previously executed in the Executions. * @return The accumulator map */ - public Map> aggregateUserAccumulators() { - - Map> userAccumulators = new HashMap>(); - + public Map> aggregateSmallUserAccumulators() { + this.mergedSmallUserAccumulators = new HashMap>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { - Map> next = vertex.getCurrentExecutionAttempt().getUserAccumulators(); + Map> next = vertex.getCurrentExecutionAttempt().getSmallUserAccumulators(); if (next != null) { - AccumulatorHelper.mergeInto(userAccumulators, next); + AccumulatorHelper.mergeInto(mergedSmallUserAccumulators, next); } } + return mergedSmallUserAccumulators; + } - return userAccumulators; + /** + * Merges the partial accumulators referring to the same global accumulator received + * from the tasks, and serializes the objects of the accumulators (not only the content as the + * {@link #getSmallAccumulatorsContentSerialized()}. This means that the actual + * objects are serialized, thus merging can still be applied after deserialization. + * Each of the partial accumulators contains the partial result produced by each task, + * for that specific accumulator. + * + * @return The accumulator map with serialized accumulator objects. + * @throws IOException + */ + public Map> getSmallAccumulatorsSerialized() throws IOException { + Map> accumulatorMap = (mergedSmallUserAccumulators != null) ? + mergedSmallUserAccumulators : aggregateSmallUserAccumulators(); + + Map> result = new HashMap>(); + for (Map.Entry> entry : accumulatorMap.entrySet()) { + result.put(entry.getKey(), new SerializedValue(entry.getValue())); + } + return result; } /** - * Gets a serialized accumulator map. + * Merges the partial accumulators referring to the same global accumulator received + * from the tasks, and serializes the content of the final accumulator, i.e. the result. + * This is unlike the {@link #getSmallAccumulatorsSerialized()} which serialized the whole + * accumulator object. Each of the partial accumulators contains the partial result + * produced by each task, for that specific accumulator. + * * @return The accumulator map with serialized accumulator values. * @throws IOException */ - public Map> getAccumulatorsSerialized() throws IOException { - - Map> accumulatorMap = aggregateUserAccumulators(); + public Map> getSmallAccumulatorsContentSerialized() throws IOException { + Map> accumulatorMap = (mergedSmallUserAccumulators != null) ? + mergedSmallUserAccumulators : aggregateSmallUserAccumulators(); Map> result = new HashMap>(); for (Map.Entry> entry : accumulatorMap.entrySet()) { result.put(entry.getKey(), new SerializedValue(entry.getValue().getLocalValue())); } - return result; } + private void aggregateLargeUserAccumulatorBlobKeys( + Map> target, Map> toMerge) { + if (target == null || toMerge == null || toMerge.isEmpty()) { + return; + } + + for (Map.Entry> otherEntry : toMerge.entrySet()) { + List existing = target.get(otherEntry.getKey()); + if (existing == null) { + target.put(otherEntry.getKey(), otherEntry.getValue()); + } else { + existing.addAll(otherEntry.getValue()); + } + } + } + + /** + * Merges all blobKeys referring to blobs of large accumulators. These refer to blobs in the + * blobCache holding accumulators (results of tasks) that did not fit in an akka frame, + * thus had to be sent through the BlobCache. + * + * @return The accumulator map + */ + public Map> aggregateLargeUserAccumulatorBlobKeys() { + Map> largeUserAccumulatorRefs = new HashMap>(); + + for (ExecutionVertex vertex : getAllExecutionVertices()) { + Map> next = vertex.getCurrentExecutionAttempt().getLargeUserAccumulatorBlobKeys(); + aggregateLargeUserAccumulatorBlobKeys(largeUserAccumulatorRefs, next); + } + return largeUserAccumulatorRefs; + } + + /** + * Adds new blobKeys pointing to blobs of large accumulators in the BlobCache, to the list + * of already received BlobKeys. These refer to blobs in the blobCache holding accumulators + * (results of tasks) that did not fit in an akka framesize, thus had to be sent through + * the BlobCache. + * + * @param target the initial blobKey map + * @param toMerge the new keys to add to the initial map + * @return The resulting accumulator map + */ + public Map> addLargeUserAccumulatorBlobKeys( + Map> target, Map> toMerge) { + + if (target == null) { + target = new HashMap>(); + } + aggregateLargeUserAccumulatorBlobKeys(target, toMerge); + return target; + } + /** * Returns the a stringified version of the user-defined accumulators. * @return an Array containing the StringifiedAccumulatorResult objects */ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { + Map> smallAccumulatorMap = mergedSmallUserAccumulators; + Map> largeAccumulatorMap = aggregateLargeUserAccumulatorBlobKeys(); - Map> accumulatorMap = aggregateUserAccumulators(); + // get the total number of (unique) accumulators + Set uniqueAccumulators = new HashSet(); + uniqueAccumulators.addAll(smallAccumulatorMap.keySet()); + uniqueAccumulators.addAll(largeAccumulatorMap.keySet()); + int num = uniqueAccumulators.size(); - int num = accumulatorMap.size(); StringifiedAccumulatorResult[] resultStrings = new StringifiedAccumulatorResult[num]; - int i = 0; - for (Map.Entry> entry : accumulatorMap.entrySet()) { + for (Map.Entry> entry : smallAccumulatorMap.entrySet()) { StringifiedAccumulatorResult result; Accumulator value = entry.getValue(); @@ -595,6 +684,21 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { resultStrings[i++] = result; } + for (Map.Entry> entry : largeAccumulatorMap.entrySet()) { + + if (!smallAccumulatorMap.containsKey(entry.getKey())) { + StringBuilder str = new StringBuilder(); + str.append("BlobKeys=[ "); + for (BlobKey bk : entry.getValue()) { + str.append(bk + " "); + } + str.append("]"); + + StringifiedAccumulatorResult result = + new StringifiedAccumulatorResult(entry.getKey(), "Blob/Serialized", str.toString()); + resultStrings[i++] = result; + } + } return resultStrings; } @@ -633,16 +737,16 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti this.verticesInCreationOrder.add(ejv); } } - + public void scheduleForExecution(Scheduler scheduler) throws JobException { if (scheduler == null) { throw new IllegalArgumentException("Scheduler must not be null."); } - + if (this.scheduler != null && this.scheduler != scheduler) { throw new IllegalArgumentException("Cannot use different schedulers for the same job"); } - + if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) { this.scheduler = scheduler; @@ -678,7 +782,7 @@ public void scheduleForExecution(Scheduler scheduler) throws JobException { public void cancel() { while (true) { JobStatus current = state; - + if (current == JobStatus.RUNNING || current == JobStatus.CREATED) { if (transitionState(current, JobStatus.CANCELLING)) { for (ExecutionJobVertex ejv : verticesInCreationOrder) { @@ -712,10 +816,10 @@ else if (transitionState(current, JobStatus.FAILING, t)) { // set the state of the job to failed transitionState(JobStatus.FAILING, JobStatus.FAILED, t); } - + return; } - + // no need to treat other states } } @@ -747,7 +851,7 @@ public void restart() { } numFinishedJobVertices = 0; transitionState(JobStatus.RESTARTING, JobStatus.CREATED); - + // if we have checkpointed state, reload it into the executions if (checkpointCoordinator != null) { checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), false, false); @@ -806,7 +910,7 @@ public void waitUntilFinished() throws InterruptedException { } } } - + private boolean transitionState(JobStatus current, JobStatus newState) { return transitionState(current, newState, null); } @@ -833,14 +937,14 @@ void jobVertexInFinalState(ExecutionJobVertex ev) { } numFinishedJobVertices++; - + if (numFinishedJobVertices == verticesInCreationOrder.size()) { - + // we are done, transition to the final state JobStatus current; while (true) { current = this.state; - + if (current == JobStatus.RUNNING) { if (transitionState(current, JobStatus.FINISHED)) { postRunCleanup(); @@ -888,7 +992,7 @@ else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, } } } - + private void postRunCleanup() { try { CheckpointCoordinator coord = this.checkpointCoordinator; @@ -919,18 +1023,23 @@ public boolean updateState(TaskExecutionState state) { case RUNNING: return attempt.switchToRunning(); case FINISHED: + AccumulatorSnapshot accumulatorSnapshot = state.getAccumulators(); + Map> flinkAccumulators = null; - Map> userAccumulators = null; + UserAccumulators userAccumulators = accumulatorSnapshot.getUserAccumulators(); + Map> smallAccumulators = null; + Map> largeAccumulators = null; try { - AccumulatorSnapshot accumulators = state.getAccumulators(); - flinkAccumulators = accumulators.deserializeFlinkAccumulators(); - userAccumulators = accumulators.deserializeUserAccumulators(userClassLoader); + flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); + + largeAccumulators = userAccumulators.getLargeUserAccumulatorBlobKeys(); + smallAccumulators = userAccumulators.deserializeSmallUserAccumulators(userClassLoader); } catch (Exception e) { // Exceptions would be thrown in the future here LOG.error("Failed to deserialize final accumulator results.", e); } - attempt.markFinished(flinkAccumulators, userAccumulators); + attempt.markFinished(flinkAccumulators, smallAccumulators, largeAccumulators); return true; case CANCELED: attempt.cancelingComplete(); @@ -992,15 +1101,19 @@ void deregisterExecution(Execution exec) { */ public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { Map> flinkAccumulators; - Map> userAccumulators; + UserAccumulators userAccumulators = accumulatorSnapshot.getUserAccumulators(); try { flinkAccumulators = accumulatorSnapshot.deserializeFlinkAccumulators(); - userAccumulators = accumulatorSnapshot.deserializeUserAccumulators(userClassLoader); ExecutionAttemptID execID = accumulatorSnapshot.getExecutionAttemptID(); Execution execution = currentExecutions.get(execID); if (execution != null) { - execution.setAccumulators(flinkAccumulators, userAccumulators); + Map> largeAccumulators = + userAccumulators.getLargeUserAccumulatorBlobKeys(); + Map> smallAccumulators = + userAccumulators.deserializeSmallUserAccumulators(userClassLoader); + + execution.setAccumulators(flinkAccumulators, smallAccumulators, largeAccumulators); } else { LOG.warn("Received accumulator result for unknown execution {}.", execID); } @@ -1024,8 +1137,8 @@ public void registerExecutionListener(ActorGateway listener) { this.executionListenerActors.add(listener); } } - - + + private void notifyJobStatusChange(JobStatus newState, Throwable error) { if (jobStatusListenerActors.size() > 0) { ExecutionGraphMessages.JobStatusChanged message = @@ -1036,7 +1149,7 @@ private void notifyJobStatusChange(JobStatus newState, Throwable error) { } } } - + void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionAttemptID executionID, ExecutionState newExecutionState, Throwable error) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 1b43139985251..14d3ef476e2a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.taskmanager; +import akka.actor.ActorRef; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; @@ -59,8 +60,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; +import scala.runtime.AbstractFunction0; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -253,7 +256,6 @@ public Task(TaskDeploymentDescriptor tdd, this.memoryManager = checkNotNull(memManager); this.ioManager = checkNotNull(ioManager); this.broadcastVariableManager = checkNotNull(bcVarManager); - this.accumulatorRegistry = new AccumulatorRegistry(jobId, executionId); this.jobManager = checkNotNull(jobManagerActor); this.taskManager = checkNotNull(taskManagerActor); @@ -264,6 +266,10 @@ public Task(TaskDeploymentDescriptor tdd, this.network = checkNotNull(networkEnvironment); this.taskManagerConfig = checkNotNull(taskManagerConfig); + + this.accumulatorRegistry = new AccumulatorRegistry(tdd.getJobConfiguration(), + jobId, executionId, getBlobCacheServerAddress()); + this.executionListenerActors = new CopyOnWriteArrayList(); // create the reader and writer structures @@ -314,6 +320,32 @@ public Task(TaskDeploymentDescriptor tdd, executingThread = new Thread(TASK_THREADS_GROUP, this, taskNameWithSubtask); } + /** + * Gets the address where the blobCache is listening to. + * @return the address where the blobCache is listening to. + * */ + private InetSocketAddress getBlobCacheServerAddress() { + if(jobManager == null || libraryCache == null) { + throw new RuntimeException("TaskManager not associated to JobManager."); + } + + final String jmHost; + ActorRef jobManagerActor = this.jobManager.actor(); + if (jobManagerActor == null) { + jmHost = "localhost"; + } else { + jmHost = jobManagerActor.path().address().host().getOrElse( + new AbstractFunction0() { + @Override + public String apply() { + return "localhost"; + } + }); + } + int blobPort = this.libraryCache.getBlobServerPort(); + return new InetSocketAddress(jmHost, blobPort); + } + // ------------------------------------------------------------------------ // Accessors // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java index 6a5468a5355ed..41cd5f17627c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedValue.java @@ -47,6 +47,9 @@ public SerializedValue(T value) throws IOException { this.serializedData = value == null ? null : InstantiationUtil.serializeObject(value); } + public SerializedValue(byte[] data) { + this.serializedData = data; + } @SuppressWarnings("unchecked") public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundException { @@ -57,6 +60,25 @@ public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundE return serializedData == null ? null : (T) InstantiationUtil.deserializeObject(serializedData, loader); } + /** + * Gets the content of the value in a byte array. + * @return the byte array backing the content of the object. + * */ + public byte[] getSerializedData() { + return serializedData; + } + + /** + * Gets the size of the serialized content. + * @return the size in bytes of the data stored in the blob. + * */ + public long getSizeInBytes() { + if(serializedData == null) { + throw new RuntimeException("Data Payload is NULL."); + } + return serializedData.length; + } + // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index b8cce41fe9109..2432a173a5764 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -421,4 +421,20 @@ object AkkaUtils { val duration = Duration(ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT) new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) } + + def getFramesize(config: Configuration): Long = config.getString( + ConfigConstants.AKKA_FRAMESIZE, + ConfigConstants.DEFAULT_AKKA_FRAMESIZE).replaceAll("[^\\d]", "").toLong + + def getLargeAccumulatorThreshold(config: Configuration): Long = { + var threshold = config.getDouble( + ConfigConstants.AKKA_FRAMESIZE_OVERSIZED_THRESHOLD, + ConfigConstants.DEFAULT_AKKA_FRAMESIZE_OVERSIZED_THRESHOLD) + + if(threshold < 0.0 || threshold > 0.8) { + threshold = ConfigConstants.AKKA_FRAMESIZE_OVERSIZED_THRESHOLD.toDouble + } + + (threshold * getFramesize(config)).toLong + } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 92688fa65989b..3eb9f366de717 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -25,20 +25,29 @@ import java.util.{UUID, Collections} import akka.actor.Status.{Failure, Success} import akka.actor._ +import _root_.akka.pattern.ask import grizzled.slf4j.Logger import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot -import org.apache.flink.runtime.blob.BlobServer +import org.apache.flink.runtime.accumulators.{AccumulatorSnapshot, LargeAccumulatorHelper} +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.blob.{BlobKey, BlobServer} import org.apache.flink.runtime.client._ +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex} +import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager} +import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID} +import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} import org.apache.flink.runtime.jobmanager.web.WebInfoServer import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged +import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect} +import org.apache.flink.runtime.messages.RegistrationMessages._ +import org.apache.flink.runtime.messages.TaskManagerMessages.{Heartbeat, SendStackTrace} import org.apache.flink.runtime.messages.TaskMessages.{PartitionState, UpdateTaskExecutionState} import org.apache.flink.runtime.messages.accumulators._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint} @@ -47,27 +56,16 @@ import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner import org.apache.flink.runtime.taskmanager.TaskManager -import org.apache.flink.runtime.util.ZooKeeperUtil -import org.apache.flink.runtime.util.{SerializedValue, EnvironmentInformation} +import org.apache.flink.runtime.util.{EnvironmentInformation, SerializedValue, ZooKeeperUtil} import org.apache.flink.runtime.webmonitor.WebMonitor -import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessages} -import org.apache.flink.runtime.{LogMessages} -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager -import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager} -import org.apache.flink.runtime.jobgraph.{JobVertexID, JobGraph, JobStatus} -import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} -import org.apache.flink.runtime.messages.JobManagerMessages._ -import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.TaskManagerMessages.{SendStackTrace, Heartbeat} +import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.util.{ExceptionUtils, InstantiationUtil} -import _root_.akka.pattern.ask +import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps -import scala.collection.JavaConverters._ /** * The job manager is responsible for receiving Flink jobs, scheduling the tasks, gathering the @@ -336,16 +334,51 @@ class JobManager( // is the client waiting for the job result? newJobStatus match { case JobStatus.FINISHED => - val accumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { - executionGraph.getAccumulatorsSerialized + + val jobConfig = currentJobs.getOrElse(jobID, + throw new RuntimeException("Unknown Job: " + jobID))._1.getJobConfiguration + + val smallAccumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try { + executionGraph.getSmallAccumulatorsContentSerialized } catch { case e: Exception => log.error(s"Cannot fetch serialized accumulators for job $jobID", e) Collections.emptyMap() } - val result = new SerializedJobExecutionResult(jobID, jobInfo.duration, - accumulatorResults) - jobInfo.client ! decorateMessage(JobResultSuccess(result)) + + var largeAccumulatorResults: java.util.Map[String, java.util.List[BlobKey]] = + executionGraph.aggregateLargeUserAccumulatorBlobKeys() + + /* + * The following covers the case where partial accumulator results are small, but + * when aggregated, they become big. In this case, this happens at the JobManager, + * and this code is responsible for detecting it, storing the oversized result in + * the BlobCache, and informing the Client accordingly. + * */ + + val totalSize: Long = smallAccumulatorResults.asScala.map(_._2.getSizeInBytes).sum + if (totalSize > AkkaUtils.getLargeAccumulatorThreshold(jobConfig)) { + + // given that the client is going to do the final merging, we serialize and + // store the accumulator objects, not only the content + val serializedSmallAccumulators = executionGraph.getSmallAccumulatorsSerialized + + // store the accumulators in the blobCache and get the keys. + val newBlobKeys = LargeAccumulatorHelper.storeSerializedAccumulatorsToBlobCache( + getBlobCacheServerAddress, serializedSmallAccumulators) + + // given that the result is going to be sent through the BlobCache, clear its + // in-memory version. + smallAccumulatorResults.clear() + + // and update the blobKeys to send to the client. + largeAccumulatorResults = executionGraph. + addLargeUserAccumulatorBlobKeys(largeAccumulatorResults, newBlobKeys) + } + + val result = new SerializedJobExecutionResult(jobID, + jobInfo.duration, smallAccumulatorResults, largeAccumulatorResults) + jobInfo.client ! JobResultSuccess(result) case JobStatus.CANCELED => jobInfo.client ! decorateMessage( @@ -513,6 +546,18 @@ class JobManager( sender() ! ResponseLeaderSessionID(leaderSessionID) } + /** + * Gets the address where the blobCache is listening to. + * @return the address where the blobCache is listening to. + **/ + private def getBlobCacheServerAddress: InetSocketAddress = { + if (libraryCacheManager == null) { + throw new RuntimeException("LibraryCacheManager is not initialized yet.") + } + val blobPort: Int = this.libraryCacheManager.getBlobServerPort + return new InetSocketAddress("localhost", blobPort) + } + /** * Submits a job to the job manager. The job is registered at the libraryCacheManager which * creates the job's class loader. The job graph is appended to the corresponding execution @@ -768,15 +813,15 @@ class JobManager( try { currentJobs.get(jobID) match { case Some((graph, jobInfo)) => - val accumulatorValues = graph.getAccumulatorsSerialized() - sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) + val accumulatorValues = graph.getSmallAccumulatorsContentSerialized + sender() ! decorateMessage(AccumulatorResultsFound(jobID, accumulatorValues)) case None => archive.forward(message) } } catch { - case e: Exception => - log.error("Cannot serialize accumulator result.", e) - sender() ! decorateMessage(AccumulatorResultsErroneous(jobID, e)) + case e: Exception => + log.error("Cannot serialize accumulator result.", e) + sender() ! decorateMessage(AccumulatorResultsErroneous(jobID, e)) } case RequestAccumulatorResultsStringified(jobId) => @@ -886,7 +931,7 @@ class JobManager( val finished = new java.util.ArrayList[JobID]() val canceled = new java.util.ArrayList[JobID]() val failed = new java.util.ArrayList[JobID]() - + currentJobs.values.foreach { case (graph, _) => graph.getState() match { case JobStatus.FINISHED => finished.add(graph.getJobID) @@ -1573,10 +1618,10 @@ object JobManager { /** * Starts the web runtime monitor. Because the actual implementation of the * runtime monitor is in another project, we load the runtime monitor dynamically. - * + * * Because failure to start the web runtime monitor is not considered fatal, * this method does not throw any exceptions, but only logs them. - * + * * @param config The configuration for the runtime monitor. * @param jobManager The JobManager actor gateway. * @param archiver The execution graph archive actor. diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala index 9f228ed4da833..d623595ce5b9f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala @@ -132,7 +132,7 @@ class MemoryArchivist(private val max_entries: Int) try { graphs.get(jobID) match { case Some(graph) => - val accumulatorValues = graph.getAccumulatorsSerialized() + val accumulatorValues = graph.getSmallAccumulatorsContentSerialized() sender() ! AccumulatorResultsFound(jobID, accumulatorValues) case None => sender() ! AccumulatorResultsNotFound(jobID) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index cc8b8ba320f34..5e908ceac4119 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -19,62 +19,59 @@ package org.apache.flink.runtime.taskmanager import java.io.{File, IOException} +import java.lang.management.{ManagementFactory, OperatingSystemMXBean} +import java.lang.reflect.Method import java.net.{InetAddress, InetSocketAddress} import java.util.UUID import java.util.concurrent.TimeUnit -import java.lang.reflect.Method -import java.lang.management.{OperatingSystemMXBean, ManagementFactory} import _root_.akka.actor._ import _root_.akka.pattern.ask import _root_.akka.util.Timeout - -import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.codahale.metrics.json.MetricsModule -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} - +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import com.codahale.metrics.{Gauge, MetricFilter, MetricRegistry} import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration._ +import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration, IllegalConfigurationException} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot -import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage} -import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.blob.{BlobService, BlobCache} +import org.apache.flink.runtime.blob.{BlobCache, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor} import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} import org.apache.flink.runtime.executiongraph.ExecutionAttemptID import org.apache.flink.runtime.filecache.FileCache -import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, -InstanceConnectionInfo, InstanceID} +import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID} import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync} import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.io.network.netty.NettyConfig import org.apache.flink.runtime.jobgraph.IntermediateDataSetID import org.apache.flink.runtime.jobmanager.JobManager -import org.apache.flink.runtime.memorymanager.{MemoryManager, DefaultMemoryManager} +import org.apache.flink.runtime.memorymanager.{DefaultMemoryManager, MemoryManager} import org.apache.flink.runtime.messages.Messages._ import org.apache.flink.runtime.messages.RegistrationMessages._ import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ +import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} import org.apache.flink.runtime.net.NetUtils import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner -import org.apache.flink.runtime.util.{ZooKeeperUtil, MathUtils, EnvironmentInformation} +import org.apache.flink.runtime.util.{EnvironmentInformation, MathUtils, ZooKeeperUtil} +import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode} +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool -import scala.util.{Failure, Success} -import scala.collection.JavaConverters._ -import scala.collection.JavaConversions._ - import scala.language.postfixOps +import scala.util.{Failure, Success} /** * The TaskManager is responsible for executing the individual tasks of a Flink job. It is diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java index 5c9ffa7c6747f..ee577fefff3c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java @@ -47,7 +47,7 @@ public void testSerialization() { origMap.put("name1", new SerializedValue(723L)); origMap.put("name2", new SerializedValue("peter")); - SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap); + SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap, null); // serialize and deserialize the object SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); @@ -84,12 +84,13 @@ public void testSerialization() { @Test public void testSerializationWithNullValues() { try { - SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null); + SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null, null); SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); assertNull(cloned.getJobId()); assertEquals(0L, cloned.getNetRuntime()); assertNull(cloned.getSerializedAccumulatorResults()); + assertNull(cloned.getBlobKeysToLargeAccumulators()); JobExecutionResult jResult = result.toJobExecutionResult(getClass().getClassLoader()); assertNull(jResult.getJobID()); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 987af4044e87e..456f4a165cc42 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -216,7 +216,7 @@ class TestingJobManager( currentJobs.get(jobID) match { case Some((graph, jobInfo)) => val flinkAccumulators = graph.getFlinkAccumulators - val userAccumulators = graph.aggregateUserAccumulators + val userAccumulators = graph.aggregateSmallUserAccumulators actors foreach { actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators) } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 01e6f624cda04..218b257bd841a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -44,7 +44,8 @@ * - when null records are passed through the system. * - when disjoint dataflows are executed * - when accumulators are used chained after a non-udf operator. - * + * - when an accumulator is bigger than the akka.framesize. + * * The tests are bundled into one class to reuse the same test cluster. This speeds * up test execution, as the majority of the test time goes usually into starting/stopping the * test cluster. @@ -61,6 +62,8 @@ public static void startCluster() { config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + config.setString(ConfigConstants.AKKA_FRAMESIZE, ConfigConstants.DEFAULT_AKKA_FRAMESIZE); + cluster = new ForkableFlinkMiniCluster(config, false); } catch (Exception e) { @@ -174,4 +177,52 @@ public void flatMap(Long value, Collector out) { fail(e.getMessage()); } } + + @Test + public void testOversizedAccumulatorsAtTaskManagers() { + try { + + ExecutionEnvironment env = + ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + + int noOfParallelism = 5; + int longsPerTask = 1200000; + + env.setParallelism(noOfParallelism); + env.getConfig().disableSysoutLogging(); + + DataSet bigEnough = env.generateSequence(1, noOfParallelism * longsPerTask); + long theCount = bigEnough.collect().size(); + + assertEquals(noOfParallelism * longsPerTask, theCount); + + }catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testOversizedAccumulatorsAtJobManager() { + try { + + ExecutionEnvironment env = + ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + + int noOfParallelism = 5; + int longsInTotal = 1200000; + + env.setParallelism(noOfParallelism); + env.getConfig().disableSysoutLogging(); + + DataSet bigEnough = env.generateSequence(1, longsInTotal); + long theCount = bigEnough.collect().size(); + + assertEquals(longsInTotal, theCount); + + }catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } } From 934dcb8c02cff842da2121fe0494d2c2f43759d0 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Thu, 27 Aug 2015 17:44:14 +0100 Subject: [PATCH 174/175] Updated documentation of method. --- .../accumulators/LargeAccumulatorHelper.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java index 2dab7336bbe6a..47072767d994e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java @@ -71,12 +71,19 @@ public static Map> storeAccumulatorsToBlobCache( } /** - * Puts the blobs of the large accumulators on the BlobCache. - * + * When the result of the job contains oversized (i.e. bigger than + * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( + org.apache.flink.configuration.Configuration)} bytes) + * accumulators then these are put in the BlobCache for the client to fetch and merge. + * This method stores the blobs of the large accumulators in the BlobCache. Contrary to + * {@link LargeAccumulatorHelper#storeAccumulatorsToBlobCache(InetSocketAddress, Map)}, this + * method assumes that accumulators are already serialized. * @param blobServerAddress the address of the server to the blobCache. - * @param accumulators the accumulators to be stored in the cache. - * @return the name of each accumulator with the BlobKey that identifies its blob in the BlobCache. - */ + * @param accumulators a map with the names and the (serialized) accumulators to be + * stored in the BlobCache. + * @return the name of each accumulator with the associated BlobKey that identifies + * its blob in the BlobCache. + * */ public static Map> storeSerializedAccumulatorsToBlobCache(InetSocketAddress blobServerAddress, Map> accumulators) throws IOException { if (blobServerAddress == null) { From 5b267ff9c6ba28a4ba1a8746645d48bc2f732187 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 27 Aug 2015 23:39:18 +0100 Subject: [PATCH 175/175] . --- .../flink/runtime/accumulators/LargeAccumulatorHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java index 47072767d994e..15131348c0b6f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/LargeAccumulatorHelper.java @@ -73,7 +73,7 @@ public static Map> storeAccumulatorsToBlobCache( /** * When the result of the job contains oversized (i.e. bigger than * {@link org.apache.flink.runtime.akka.AkkaUtils#getLargeAccumulatorThreshold( - org.apache.flink.configuration.Configuration)} bytes) + org.apache.flink.configuration.Configuration)} bytes) * accumulators then these are put in the BlobCache for the client to fetch and merge. * This method stores the blobs of the large accumulators in the BlobCache. Contrary to * {@link LargeAccumulatorHelper#storeAccumulatorsToBlobCache(InetSocketAddress, Map)}, this