veysiertekin commented on a change in pull request #7991: 
[FLINK-11928][Connectors / FileSystem]Create a Compression string writer for 
flink-connectors-filesystem
URL: https://github.com/apache/flink/pull/7991#discussion_r365549543
 
 

 ##########
 File path: 
flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/CompressionStringWriter.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.connectors.fs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+
+import java.io.IOException;
+
+/**
+ * A base class that compresses the input element and write them to the 
filesystem. Default serialization is to
+ * write events separates by newline.
+ * Extends the class and override write() to make custom writing
+ */
+public class CompressionStringWriter<T> extends StreamWriterBase<T> implements 
Writer<T>{
+       private static final long serialVersionUID = 1L;
+
+       // The codec name is the codec class name from 
org.apache.hadoop.io.compress.
+       // Candidate: GzipCodec, Lz4Codec, SnappyCodec, BZip2Codec
+       private String codecName;
+
+       private String separator;
+
+       public String getCodecName() {
+               return codecName;
+       }
+
+       public String getSeparator() {
+               return separator;
+       }
+
+       private transient CompressionOutputStream compressedOutputStream;
+
+       public CompressionStringWriter(String codecName, String separator) {
+               this.codecName = codecName;
+               this.separator = separator;
+       }
+
+       public CompressionStringWriter(String codecName) {
+               this(codecName, System.lineSeparator());
+       }
+
+       protected CompressionStringWriter(CompressionStringWriter<T> other) {
+               super(other);
+               this.codecName = other.codecName;
+               this.separator = other.separator;
+       }
+
+       @Override
+       public void open(FileSystem fs, Path path) throws IOException {
+               super.open(fs, path);
+               Configuration conf = fs.getConf();
+               CompressionCodecFactory codecFactory = new 
CompressionCodecFactory(conf);
+               CompressionCodec codec = codecFactory.getCodecByName(codecName);
+               if (codec == null) {
+                       throw new RuntimeException("Codec " + codecName + " not 
found");
+               }
+               Compressor compressor = CodecPool.getCompressor(codec, conf);
+               compressedOutputStream = codec.createOutputStream(getStream(), 
compressor);
+
+       }
+
+       @Override
+       public void close() throws IOException {
+               if (compressedOutputStream != null) {
 
 Review comment:
   There is a problem with closing operation:
   
   ```java
   2020-01-11 14:41:02
   java.lang.RuntimeException: Exception occurred while processing valve output 
watermark: 
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor$ForwardingValveOutputHandler.handleWatermark(StreamOneInputProcessor.java:216)
        at 
org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve.findAndOutputNewMinWatermarkAcrossAlignedChannels(StatusWatermarkValve.java:189)
        at 
org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve.inputWatermark(StatusWatermarkValve.java:111)
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processElement(StreamOneInputProcessor.java:169)
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:143)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:279)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.run(StreamTask.java:301)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:406)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:705)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:530)
        at java.lang.Thread.run(Thread.java:748)
   Caused by: 
org.apache.flink.streaming.runtime.tasks.ExceptionInChainedOperatorException: 
Could not forward element to next operator
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:654)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:612)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:592)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:727)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:705)
        at 
org.apache.flink.streaming.api.operators.TimestampedCollector.collect(TimestampedCollector.java:51)
        at flink.test.job.TestWindowFunction$.apply(TestJob.scala:83)
        at flink.test.job.TestWindowFunction$.apply(TestJob.scala:78)
        at 
org.apache.flink.streaming.api.scala.function.util.ScalaWindowFunctionWrapper.apply(ScalaWindowFunctionWrapper.scala:44)
        at 
org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction.process(InternalIterableWindowFunction.java:44)
        at 
org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction.process(InternalIterableWindowFunction.java:32)
        at 
org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.emitWindowContents(WindowOperator.java:549)
        at 
org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.onEventTime(WindowOperator.java:457)
        at 
org.apache.flink.streaming.api.operators.InternalTimerServiceImpl.advanceWatermark(InternalTimerServiceImpl.java:276)
        at 
org.apache.flink.streaming.api.operators.InternalTimeServiceManager.advanceWatermark(InternalTimeServiceManager.java:128)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:784)
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor$ForwardingValveOutputHandler.handleWatermark(StreamOneInputProcessor.java:213)
        ... 10 more
   Caused by: java.lang.IllegalStateException: Writer has already been opened
        at 
org.apache.flink.streaming.connectors.fs.StreamWriterBase.open(StreamWriterBase.java:69)
        at 
flink.test.sink.CompressionStringWriter.open(CompressionStringWriter.java:77)
        at 
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.openNewPartFile(BucketingSink.java:588)
        at 
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.invoke(BucketingSink.java:463)
        at 
org.apache.flink.streaming.api.functions.sink.SinkFunction.invoke(SinkFunction.java:52)
        at 
org.apache.flink.streaming.api.operators.StreamSink.processElement(StreamSink.java:56)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:637)
        ... 26 more
   ```
   
   I had to fixed with (maybe there is a better solution for this):
   
   ```java
       @Override
       public void close() throws IOException {
           if (compressedOutputStream != null) {
               compressedOutputStream.finish();
               compressedOutputStream = null;
           }
   
           try {
               super.close();
           } finally {
               if (compressor != null) {
                   CodecPool.returnCompressor(compressor);
                   compressor = null;
               }
           }
       }
   ```
   
   (`compressor` is an instance variable while creating it in the `open` 
function)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to