[
https://issues.apache.org/jira/browse/FLINK-8620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16410992#comment-16410992
]
ASF GitHub Bot commented on FLINK-8620:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/5580#discussion_r176662599
--- Diff:
flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java
---
@@ -453,15 +460,51 @@ private void testGetFailsDuringStreaming(@Nullable
final JobID jobId, BlobKey.Bl
}
/**
- * Tests the static {@link BlobClient#uploadJarFiles(InetSocketAddress,
Configuration, JobID, List)} helper.
+ * Tests the static {@link BlobClient#uploadFiles(InetSocketAddress,
Configuration, JobID, List)} helper.
*/
@Test
public void testUploadJarFilesHelper() throws Exception {
uploadJarFile(getBlobServer(), getBlobClientConfig());
}
+ @Test
+ public void testDirectoryUploading() throws IOException {
+ final File newFolder = temporaryFolder.newFolder();
+ final File file1 = File.createTempFile("pre", "suff",
newFolder);
+ FileUtils.writeStringToFile(file1, "Test content");
+ final File file2 = File.createTempFile("pre", "suff",
newFolder);
+ FileUtils.writeStringToFile(file2, "Test content 2");
+
+ final Map<String, File> files = new HashMap<>();
+ files.put(file1.getName(), file1);
+ files.put(file2.getName(), file2);
+
+ BlobKey key;
+ final JobID jobId = new JobID();
+ final InetSocketAddress inetAddress = new
InetSocketAddress("localhost", getBlobServer().getPort());
+ try (
+ BlobClient client = new BlobClient(
+ inetAddress, getBlobClientConfig())) {
+
+ key = client.uploadFile(jobId, new
Path(newFolder.getPath()));
+ }
+
+ final File file = getBlobServer().getFile(jobId,
(PermanentBlobKey) key);
+
+ try (ZipInputStream zis = new ZipInputStream(new
FileInputStream(file))) {
+ ZipEntry entry;
+ while ((entry = zis.getNextEntry()) != null) {
--- End diff --
entry should be closed
> Enable shipping custom artifacts to BlobStore and accessing them through
> DistributedCache
> -----------------------------------------------------------------------------------------
>
> Key: FLINK-8620
> URL: https://issues.apache.org/jira/browse/FLINK-8620
> Project: Flink
> Issue Type: New Feature
> Reporter: Dawid Wysakowicz
> Assignee: Dawid Wysakowicz
> Priority: Major
>
> We should be able to distribute custom files to taskmanagers. To do that we
> can store those files in BlobStore and later on access them in TaskManagers
> through DistributedCache.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)