Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-6177] Add support for "Distributed Cache" in streaming applica… #3741

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.functions.StoppableFunction;
import org.apache.flink.api.common.io.FileInputFormat;
Expand All @@ -36,6 +37,7 @@
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.TextInputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.typeutils.MissingTypeInfo;
import org.apache.flink.api.java.typeutils.PojoTypeInfo;
Expand Down Expand Up @@ -134,6 +136,8 @@ public abstract class StreamExecutionEnvironment {
/** The time characteristic used by the data streams */
private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;

protected final List<Tuple2<String, DistributedCache.DistributedCacheEntry>> cacheFile = new ArrayList<>();


// --------------------------------------------------------------------------------------------
// Constructor and Properties
Expand All @@ -146,6 +150,13 @@ public ExecutionConfig getConfig() {
return config;
}

/**
* Get the list of cached files that were registered for distribution among the task managers
*/
public List<Tuple2<String, DistributedCache.DistributedCacheEntry>> getCacheFile() {
return cacheFile;
}

/**
* Sets the parallelism for operations executed through this environment.
* Setting a parallelism of x here will cause all operators (such as map,
Expand Down Expand Up @@ -1642,11 +1653,11 @@ public static LocalStreamEnvironment createLocalEnvironment(int parallelism, Con
/**
* Creates a {@link LocalStreamEnvironment} for local program execution that also starts the
* web monitoring UI.
*
*
* <p>The local execution environment will run the program in a multi-threaded fashion in
* the same JVM as the environment was created in. It will use the parallelism specified in the
* parameter.
*
*
* <p>If the configuration key 'jobmanager.web.port' was set in the configuration, that particular
* port will be used for the web UI. Otherwise, the default port (8081) will be used.
*/
Expand Down Expand Up @@ -1776,8 +1787,45 @@ public static void setDefaultLocalParallelism(int parallelism) {
protected static void initializeContextEnvironment(StreamExecutionEnvironmentFactory ctx) {
contextEnvironmentFactory = ctx;
}

protected static void resetContextEnvironment() {
contextEnvironmentFactory = null;
}

/**
* Registers a file at the distributed cache under the given name. The file will be accessible
* from any user-defined function in the (distributed) runtime under a local path. Files
* may be local files (as long as all relevant workers have access to it), or files in a distributed file system.
* The runtime will copy the files temporarily to a local cache, if needed.
* <p>
* The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via
* {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access
* {@link org.apache.flink.api.common.cache.DistributedCache} via
* {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}.
*
* @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path")
* @param name The name under which the file is registered.
*/
public void registerCachedFile(String filePath, String name){
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

missing space before {.

registerCachedFile(filePath, name, false);
}

/**
* Registers a file at the distributed cache under the given name. The file will be accessible
* from any user-defined function in the (distributed) runtime under a local path. Files
* may be local files (as long as all relevant workers have access to it), or files in a distributed file system.
* The runtime will copy the files temporarily to a local cache, if needed.
* <p>
* The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via
* {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access
* {@link org.apache.flink.api.common.cache.DistributedCache} via
* {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}.
*
* @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path")
* @param name The name under which the file is registered.
* @param executable flag indicating whether the file should be executable
*/
public void registerCachedFile(String filePath, String name, boolean executable){
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

missing space before }.

this.cacheFile.add(new Tuple2<>(name, new DistributedCache.DistributedCacheEntry(filePath, executable)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.commons.lang3.StringUtils;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.java.tuple.Tuple2;
Expand Down Expand Up @@ -128,6 +129,11 @@ public JobGraph createJobGraph() {

configureCheckpointing();

// add registered cache file into job configuration
for (Tuple2<String, DistributedCache.DistributedCacheEntry> e : streamGraph.getEnvironment().getCacheFile()) {
DistributedCache.writeFileInfoToConfig(e.f0, e.f1, jobGraph.getJobConfiguration());
}

// set the ExecutionConfig last when it has been finalized
jobGraph.setExecutionConfig(streamGraph.getExecutionConfig());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,11 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
*/
def getConfig = javaEnv.getConfig

/**
* Gets cache files.
*/
def getCacheFile = javaEnv.getCacheFile

/**
* Sets the parallelism for operations executed through this environment.
* Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
Expand Down Expand Up @@ -669,6 +674,49 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
}
f
}


/**
* Registers a file at the distributed cache under the given name. The file will be accessible
* from any user-defined function in the (distributed) runtime under a local path. Files
* may be local files (as long as all relevant workers have access to it), or files in a
* distributed file system. The runtime will copy the files temporarily to a local cache,
* if needed.
* <p>
* The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs
* via {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and
* provides access {@link org.apache.flink.api.common.cache.DistributedCache} via
* {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}.
*
* @param filePath The path of the file, as a URI (e.g. "file:///some/path" or
* "hdfs://host:port/and/path")
* @param name The name under which the file is registered.
*/
def registerCachedFile(filePath: String, name: String): Unit = {
javaEnv.registerCachedFile(filePath, name)
}


/**
* Registers a file at the distributed cache under the given name. The file will be accessible
* from any user-defined function in the (distributed) runtime under a local path. Files
* may be local files (as long as all relevant workers have access to it), or files in a
* distributed file system. The runtime will copy the files temporarily to a local cache,
* if needed.
* <p>
* The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs
* via {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and
* provides access {@link org.apache.flink.api.common.cache.DistributedCache} via
* {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}.
*
* @param filePath The path of the file, as a URI (e.g. "file:///some/path" or
* "hdfs://host:port/and/path")
* @param name The name under which the file is registered.
* @param executable flag indicating whether the file should be executable
*/
def registerCachedFile(filePath: String, name: String, executable: Boolean): Unit = {
javaEnv.registerCachedFile(filePath, name, executable)
}
}

object StreamExecutionEnvironment {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public abstract class AbstractTestBase extends TestBaseUtils {
/** Configuration to start the testing cluster with */
protected final Configuration config;

private final List<File> tempFiles;
protected final List<File> tempFiles;

private final FiniteDuration timeout;

Expand Down Expand Up @@ -143,7 +143,7 @@ public File createAndRegisterTempFile(String fileName) throws IOException {
return f;
}

private void deleteAllTempFiles() throws IOException {
protected void deleteAllTempFiles() throws IOException {
for (File f : this.tempFiles) {
if (f.exists()) {
deleteRecursively(f);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,63 +15,60 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.test.distributedCache;

import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.FileNotFoundException;
import java.io.FileReader;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
package org.apache.flink.test.distributedCache;

import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.test.util.JavaProgramTestBase;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
import org.apache.flink.util.Collector;
import org.junit.Test;

import static org.junit.Assert.assertTrue;

import java.io.*;

import java.util.*;

/**
* Tests the distributed cache by comparing a text file with a distributed copy.
*/
public class DistributedCacheTest extends JavaProgramTestBase {

public class DistributedCacheTest extends StreamingMultipleProgramsTestBase {
public static final String data
= "machen\n"
+ "zeit\n"
+ "heerscharen\n"
+ "keiner\n"
+ "meine\n";

protected String textPath;

@Override
protected void preSubmit() throws Exception {
textPath = createTempFile("count.txt", data);
@Test
public void testStreamingDistributedCache() throws Exception {
String textPath = createTempFile("count.txt", data);
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.registerCachedFile(textPath, "cache_test");
env.readTextFile(textPath).flatMap(new WordChecker());
env.execute();
deleteAllTempFiles();
}

@Override
protected void testProgram() throws Exception {
@Test
public void testBatchDistributedCache() throws Exception {
String textPath = createTempFile("count.txt", data);
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.registerCachedFile(textPath, "cache_test");

List<Tuple1<String>> result = env
.readTextFile(textPath)
.flatMap(new WordChecker())
.collect();

compareResultAsTuples(result, data);
env.readTextFile(textPath).flatMap(new WordChecker()).count();
deleteAllTempFiles();
}

public static class WordChecker extends RichFlatMapFunction<String, Tuple1<String>> {
private static final long serialVersionUID = 1L;

private final Set<String> wordList = new HashSet<>();
private final List<String> wordList = new ArrayList<>();

@Override
public void open(Configuration conf) throws FileNotFoundException, IOException {
public void open(Configuration conf) throws IOException {
File file = getRuntimeContext().getDistributedCache().getFile("cache_test");
BufferedReader reader = new BufferedReader(new FileReader(file));
String tempString;
Expand All @@ -83,9 +80,10 @@ public void open(Configuration conf) throws FileNotFoundException, IOException {

@Override
public void flatMap(String word, Collector<Tuple1<String>> out) throws Exception {
if (wordList.contains(word)) {
out.collect(new Tuple1<>(word));
}
assertTrue("Unexpected word in stream! wordFromStream: " + word + ", shouldBeOneOf: " +
wordList.toString(), wordList.contains(word));

out.collect(new Tuple1<>(word));
}
}
}