[
https://issues.apache.org/jira/browse/FLINK-8543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16371613#comment-16371613
]
chris snow edited comment on FLINK-8543 at 2/21/18 4:59 PM:
------------------------------------------------------------
My apologies, the commented out close() didn't get deployed due to a compile
error including my own version of the class so I removed the close with a
bytecode modifier, i.e.
*class*: org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter
{code}
public void close() throws IOException {
if (this.keyValueWriter != null) {
this.keyValueWriter.close();
}
}
{code}
It is running without exception now, however, every file in COS S3 still has
the valid-length suffix:
{code}
...
transactions_load_20180221/2018-02-21--1454/_part-0-0.valid-length 7.0
bytes 21/02/2018 14:56:10
transactions_load_20180221/2018-02-21--1454/part-0-0 22.6
KB 21/02/2018 14:56:06
transactions_load_20180221/2018-02-21--1455/_part-0-0.valid-length 7.0
bytes 21/02/2018 14:56:05
transactions_load_20180221/2018-02-21--1455/part-0-0 14.5
KB 21/02/2018 14:56:03
...
{code}
I configured log output to debug level on Bucketing sink and see this in the
logs:
{code}
2018-02-21 16:45:18,993 DEBUG
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink - Truncate is
not supported.
java.lang.reflect.InvocationTargetException
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.reflectTruncate(BucketingSink.java:599)
at
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.initializeState(BucketingSink.java:362)
at
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.tryRestoreFunction(StreamingFunctionUtils.java:178)
at
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.restoreFunctionState(StreamingFunctionUtils.java:160)
at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:96)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:259)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:694)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:682)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.UnsupportedOperationException: Not implemented by the
S3AFileSystem FileSystem implementation
at org.apache.hadoop.fs.FileSystem.truncate(FileSystem.java:1364)
... 15 more
{code}
Is this the reason for the .valid-length files? If so, is this because the
java code doesn't support Truncate or because the filesystem (IBM COS S3)
doesn't support it? It looks as though it is not supported by my hadoop library:
*File*:
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
{code}
public boolean truncate(Path f, long newLength) throws IOException {
throw new UnsupportedOperationException("Not implemented by the " +
getClass().getSimpleName() + " FileSystem implementation");
}
{code}
and
{code}
$ grep 'truncate'
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
[[string not found]]
{code}
was (Author: snowch):
My apologies, the commented out close() didn't get deployed due to a compile
error including my own version of the class so I removed the close with a
bytecode modifier, i.e.
*class*: org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter
{code}
public void close() throws IOException {
if (this.keyValueWriter != null) {
this.keyValueWriter.close();
}
}
{code}
It is running without exception now, however, every file in COS S3 still has
the valid-length suffix:
{code}
...
transactions_load_20180221/2018-02-21--1454/_part-0-0.valid-length 7.0
bytes 21/02/2018 14:56:10
transactions_load_20180221/2018-02-21--1454/part-0-0 22.6
KB 21/02/2018 14:56:06
transactions_load_20180221/2018-02-21--1455/_part-0-0.valid-length 7.0
bytes 21/02/2018 14:56:05
transactions_load_20180221/2018-02-21--1455/part-0-0 14.5
KB 21/02/2018 14:56:03
...
{code}
I configured log output to debug level on Bucketing sink and see this in the
logs:
{code}
2018-02-21 16:45:18,993 DEBUG
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink - Truncate is
not supported.
java.lang.reflect.InvocationTargetException
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.reflectTruncate(BucketingSink.java:599)
at
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.initializeState(BucketingSink.java:362)
at
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.tryRestoreFunction(StreamingFunctionUtils.java:178)
at
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.restoreFunctionState(StreamingFunctionUtils.java:160)
at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:96)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:259)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:694)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:682)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.UnsupportedOperationException: Not implemented by the
S3AFileSystem FileSystem implementation
at org.apache.hadoop.fs.FileSystem.truncate(FileSystem.java:1364)
... 15 more
{code}
Is this the reason for the .valid-length files? If so, is this because the
java code doesn't support Truncate or because the filesystem (IBM COS S3)
doesn't support it? It looks as though it is not supported by my hadoop library:
*File*:
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
{code}
public boolean truncate(Path f, long newLength) throws IOException {
throw new UnsupportedOperationException("Not implemented by the " +
getClass().getSimpleName() + " FileSystem implementation");
}
{code}
> Output Stream closed at org.apache.hadoop.fs.s3a.S3AOutputStream.checkOpen
> --------------------------------------------------------------------------
>
> Key: FLINK-8543
> URL: https://issues.apache.org/jira/browse/FLINK-8543
> Project: Flink
> Issue Type: Bug
> Components: Streaming Connectors
> Affects Versions: 1.4.0
> Environment: IBM Analytics Engine -
> [https://console.bluemix.net/docs/services/AnalyticsEngine/index.html#introduction]
> The cluster is based on Hortonworks Data Platform 2.6.2. The following
> components are made available.
> Apache Spark 2.1.1 Hadoop 2.7.3
> Apache Livy 0.3.0
> Knox 0.12.0
> Ambari 2.5.2
> Anaconda with Python 2.7.13 and 3.5.2
> Jupyter Enterprise Gateway 0.5.0
> HBase 1.1.2 *
> Hive 1.2.1 *
> Oozie 4.2.0 *
> Flume 1.5.2 *
> Tez 0.7.0 *
> Pig 0.16.0 *
> Sqoop 1.4.6 *
> Slider 0.92.0 *
> Reporter: chris snow
> Priority: Blocker
> Fix For: 1.5.0
>
> Attachments: Screen Shot 2018-01-30 at 18.34.51.png
>
>
> I'm hitting an issue with my BucketingSink from a streaming job.
>
> {code:java}
> return new BucketingSink<Tuple2<String, Object>>(path)
> .setWriter(writer)
> .setBucketer(new DateTimeBucketer<Tuple2<String,
> Object>>(formatString));
> {code}
>
> I can see that a few files have run into issues with uploading to S3:
> !Screen Shot 2018-01-30 at 18.34.51.png!
> The Flink console output is showing an exception being thrown by
> S3AOutputStream, so I've grabbed the S3AOutputStream class from my cluster
> and added some additional logging to the checkOpen() method to log the 'key'
> just before the exception is thrown:
>
> {code:java}
> /*
> * Decompiled with CFR.
> */
> package org.apache.hadoop.fs.s3a;
> import com.amazonaws.AmazonClientException;
> import com.amazonaws.event.ProgressListener;
> import com.amazonaws.services.s3.model.ObjectMetadata;
> import com.amazonaws.services.s3.model.PutObjectRequest;
> import com.amazonaws.services.s3.transfer.Upload;
> import com.amazonaws.services.s3.transfer.model.UploadResult;
> import java.io.BufferedOutputStream;
> import java.io.File;
> import java.io.FileOutputStream;
> import java.io.IOException;
> import java.io.InterruptedIOException;
> import java.io.OutputStream;
> import java.util.concurrent.atomic.AtomicBoolean;
> import org.apache.hadoop.classification.InterfaceAudience;
> import org.apache.hadoop.classification.InterfaceStability;
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.fs.s3a.ProgressableProgressListener;
> import org.apache.hadoop.fs.s3a.S3AFileSystem;
> import org.apache.hadoop.fs.s3a.S3AUtils;
> import org.apache.hadoop.util.Progressable;
> import org.slf4j.Logger;
> @InterfaceAudience.Private
> @InterfaceStability.Evolving
> public class S3AOutputStream
> extends OutputStream {
> private final OutputStream backupStream;
> private final File backupFile;
> private final AtomicBoolean closed = new AtomicBoolean(false);
> private final String key;
> private final Progressable progress;
> private final S3AFileSystem fs;
> public static final Logger LOG = S3AFileSystem.LOG;
> public S3AOutputStream(Configuration conf, S3AFileSystem fs, String key,
> Progressable progress) throws IOException {
> this.key = key;
> this.progress = progress;
> this.fs = fs;
> this.backupFile = fs.createTmpFileForWrite("output-", -1, conf);
> LOG.debug("OutputStream for key '{}' writing to tempfile: {}",
> (Object)key, (Object)this.backupFile);
> this.backupStream = new BufferedOutputStream(new
> FileOutputStream(this.backupFile));
> }
> void checkOpen() throws IOException {
> if (!this.closed.get()) return;
> // vvvvvv-- Additional logging --vvvvvvv
> LOG.error("OutputStream for key '{}' closed.", (Object)this.key);
> throw new IOException("Output Stream closed");
> }
> @Override
> public void flush() throws IOException {
> this.checkOpen();
> this.backupStream.flush();
> }
> @Override
> public void close() throws IOException {
> if (this.closed.getAndSet(true)) {
> return;
> }
> this.backupStream.close();
> LOG.debug("OutputStream for key '{}' closed. Now beginning upload",
> (Object)this.key);
> try {
> ObjectMetadata om =
> this.fs.newObjectMetadata(this.backupFile.length());
> Upload upload =
> this.fs.putObject(this.fs.newPutObjectRequest(this.key, om, this.backupFile));
> ProgressableProgressListener listener = new
> ProgressableProgressListener(this.fs, this.key, upload, this.progress);
> upload.addProgressListener((ProgressListener)listener);
> upload.waitForUploadResult();
> listener.uploadCompleted();
> this.fs.finishedWrite(this.key);
> }
> catch (InterruptedException e) {
> throw (InterruptedIOException)new
> InterruptedIOException(e.toString()).initCause(e);
> }
> catch (AmazonClientException e) {
> throw S3AUtils.translateException("saving output", this.key, e);
> }
> finally {
> if (!this.backupFile.delete()) {
> LOG.warn("Could not delete temporary s3a file: {}",
> (Object)this.backupFile);
> }
> super.close();
> }
> LOG.debug("OutputStream for key '{}' upload complete",
> (Object)this.key);
> }
> @Override
> public void write(int b) throws IOException {
> this.checkOpen();
> this.backupStream.write(b);
> }
> @Override
> public void write(byte[] b, int off, int len) throws IOException {
> this.checkOpen();
> this.backupStream.write(b, off, len);
> }
> static {
> }
> }
> {code}
>
> You can see from this addition log output that the S3AOutputStream#close()
> method **appears** to be called before the S3AOutputStream#flush() method:
>
> {code:java}
> 2018-02-01 12:42:20,698 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem.Progress
> - PUT landingzone/2018-02-01--1240/_part-0-0.in-progress:
> 128497 bytes
> 2018-02-01 12:42:20,910 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem.Progress
> - PUT landingzone/2018-02-01--1240/_part-0-0.in-progress: 0
> bytes
> 2018-02-01 12:42:20,910 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem.Progress
> - PUT landingzone/2018-02-01--1240/_part-0-0.in-progress: 0
> bytes
> 2018-02-01 12:42:20,910 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem.Progress
> - PUT landingzone/2018-02-01--1240/_part-0-0.in-progress: 0
> bytes
> 2018-02-01 12:42:20,910 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem.Progress
> - PUT landingzone/2018-02-01--1240/_part-0-0.in-progress: 0
> bytes
> 2018-02-01 12:42:20,910 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem.Progress
> - PUT landingzone/2018-02-01--1240/_part-0-0.in-progress: 0
> bytes
> 2018-02-01 12:42:20,911 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem
> - Finished write to
> landingzone/2018-02-01--1240/_part-0-0.in-progress
> 2018-02-01 12:42:20,911 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem
> - object_delete_requests += 1 -> 3
> vvvvv- close() is called here? -vvvvv
> 2018-02-01 12:42:21,212 DEBUG org.apache.hadoop.fs.s3a.S3AFileSystem
>
> - OutputStream for key 'landingzone/2018-02-01--1240/_part-0-0.in-progress'
> upload complete
> vvvvv- flush() is called here? -vvvvv
> 2018-02-01 12:42:21,212 ERROR org.apache.hadoop.fs.s3a.S3AFileSystem
>
> - OutputStream for key 'landingzone/2018-02-01--1240/_part-0-0.in-progress'
> closed.
> 2018-02-01 12:42:21,212 INFO org.apache.flink.runtime.taskmanager.Task
>
> - Attempting to fail task externally Source: Custom Source -> Map -> Sink:
> Unnamed (1/2) (510c8316d3a249e5ea5b8d8e693f7beb).
> 2018-02-01 12:42:21,214 INFO org.apache.flink.runtime.taskmanager.Task
> - Source: Custom Source -> Map -> Sink: Unnamed (1/2)
> (510c8316d3a249e5ea5b8d8e693f7beb) switched from RUNNING to FAILED.
> TimerException{java.io.IOException: Output Stream closed}
> at
> org.apache.flink.streaming.runtime.tasks.SystemProcessingTimeService$TriggerTask.run(SystemProcessingTimeService.java:252)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: Output Stream closed
> at
> org.apache.hadoop.fs.s3a.S3AOutputStream.checkOpen(S3AOutputStream.java:83)
> at
> org.apache.hadoop.fs.s3a.S3AOutputStream.flush(S3AOutputStream.java:89)
> at java.io.FilterOutputStream.flush(FilterOutputStream.java:140)
> at java.io.DataOutputStream.flush(DataOutputStream.java:123)
> at java.io.FilterOutputStream.flush(FilterOutputStream.java:140)
> at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:141)
> at
> org.apache.avro.io.BufferedBinaryEncoder$OutputStreamSink.innerFlush(BufferedBinaryEncoder.java:220)
> at
> org.apache.avro.io.BufferedBinaryEncoder.flush(BufferedBinaryEncoder.java:85)
> at org.apache.avro.file.DataFileWriter.flush(DataFileWriter.java:368)
> at org.apache.avro.file.DataFileWriter.close(DataFileWriter.java:375)
> at
> org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter$AvroKeyValueWriter.close(AvroKeyValueSinkWriter.java:251)
> at
> org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter.close(AvroKeyValueSinkWriter.java:163)
> at
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.closeCurrentPartFile(BucketingSink.java:551)
> at
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.checkForInactiveBuckets(BucketingSink.java:493)
> at
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.onProcessingTime(BucketingSink.java:476)
> at
> org.apache.flink.streaming.runtime.tasks.SystemProcessingTimeService$TriggerTask.run(SystemProcessingTimeService.java:249)
> ... 7 more
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)