Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-9976

Odd signatures for streaming file sink format builders

    XMLWordPrintableJSON

Details

    Description

      There are 2 instances of apparently unnecessary generic parameters in the format builders for the StreamingFileSink.

      Both these methods have a generic parameter for the BucketID type, however the builder itself already has such a parameter. The methods use unchecked casts to make the types fit, so we should be able to modify the signature to use the builders parameter instead.

      public static class RowFormatBuilder<IN, BucketID> extends StreamingFileSink.BucketsBuilder<IN, BucketID> {
      ...
      	public <ID> StreamingFileSink.RowFormatBuilder<IN, ID> withBucketerAndPolicy(final Bucketer<IN, ID> bucketer, final RollingPolicy<IN, ID> policy) {
      		@SuppressWarnings("unchecked")
      		StreamingFileSink.RowFormatBuilder<IN, ID> reInterpreted = (StreamingFileSink.RowFormatBuilder<IN, ID>) this;
      		reInterpreted.bucketer = Preconditions.checkNotNull(bucketer);
      		reInterpreted.rollingPolicy = Preconditions.checkNotNull(policy);
      		return reInterpreted;
      	}
      ...
      
      public static class BulkFormatBuilder<IN, BucketID> extends StreamingFileSink.BucketsBuilder<IN, BucketID> {
      ...
      	public <ID> StreamingFileSink.BulkFormatBuilder<IN, ID> withBucketer(Bucketer<IN, ID> bucketer) {
      		@SuppressWarnings("unchecked")
      		StreamingFileSink.BulkFormatBuilder<IN, ID> reInterpreted = (StreamingFileSink.BulkFormatBuilder<IN, ID>) this;
      		reInterpreted.bucketer = Preconditions.checkNotNull(bucketer);
      		return reInterpreted;
      	}
      ...
      

      Attachments

        Issue Links

          Activity

            People

              chesnay Chesnay Schepler
              chesnay Chesnay Schepler
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: