[ 
https://issues.apache.org/jira/browse/FLINK-11140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16718666#comment-16718666
 ] 

ASF GitHub Bot commented on FLINK-11140:
----------------------------------------

yanghua commented on a change in pull request #7287: [FLINK-11140][streaming] 
Fix empty child path check in Buckets
URL: https://github.com/apache/flink/pull/7287#discussion_r240931304
 
 

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java
 ##########
 @@ -304,6 +304,10 @@ void close() {
        }
 
        private Path assembleBucketPath(BucketID bucketId) {
+               String child = bucketId.toString();
+               if ("".equals(child)) {
+                       return basePath;
+               }
                return new Path(basePath, bucketId.toString());
 
 Review comment:
   Now that you have given `bucketId.toString()` to the variable `child`, you 
can change it to: `return new Path(basePath, child);`

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


> Can not create a Path from an empty string while use BasePathBucketAssigner
> ---------------------------------------------------------------------------
>
>                 Key: FLINK-11140
>                 URL: https://issues.apache.org/jira/browse/FLINK-11140
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming
>            Reporter: Matrix42
>            Assignee: Matrix42
>            Priority: Major
>              Labels: pull-request-available
>
> while use BasePathBucketAssigner, Flink throw an exception:
> {code:java}
> Caused by: java.lang.IllegalArgumentException: Can not create a Path from an 
> empty string
> at org.apache.flink.core.fs.Path.checkAndTrimPathArg(Path.java:168)
> at org.apache.flink.core.fs.Path.<init>(Path.java:181)
> at org.apache.flink.core.fs.Path.<init>(Path.java:108)
> at 
> org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.assembleBucketPath(Buckets.java:309)
> at 
> org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.getOrCreateBucketForBucketId(Buckets.java:278)
> at 
> org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.onElement(Buckets.java:265)
> at 
> org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink.invoke(StreamingFileSink.java:370)
> at 
> org.apache.flink.streaming.api.operators.StreamSink.processElement(StreamSink.java:56)
> at 
> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:579)
> at 
> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:554)
> at 
> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:534)
> at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:718)
> at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:696)
> at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$NonTimestampContext.collect(StreamSourceContexts.java:104)
> at 
> org.apache.flink.streaming.api.functions.source.FromElementsFunction.run(FromElementsFunction.java:164)
> at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:94)
> at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
> at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:99)
> at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
> at java.lang.Thread.run(Thread.java:748)
> {code}
> reason:
> BasePathBucketAssigner#getBucketId return an empty string.
> {code:java}
> @Override
> public String getBucketId(T element, BucketAssigner.Context context) {
>    return "";
> }
> {code}
> while construct a Path, checkAndTrimPathArg methoad will check pathString, if 
> pathString is empty will throw IllegalArgumentException.
> {code:java}
> public Path(String pathString) {
>    pathString = checkAndTrimPathArg(pathString);
>   ......
> }{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to