You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by jl...@apache.org on 2021/10/28 17:55:08 UTC

[pinot] branch master updated: Support segmentNamePostfix in segment name (#7646)

This is an automated email from the ASF dual-hosted git repository.

jlli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 611f3b1  Support segmentNamePostfix in segment name (#7646)
611f3b1 is described below

commit 611f3b11b1336bc9f426ead5ba908a9632a50fd9
Author: Jialiang Li <jl...@linkedin.com>
AuthorDate: Thu Oct 28 10:54:49 2021 -0700

    Support segmentNamePostfix in segment name (#7646)
    
    Co-authored-by: Jack Li(Analytics Engineering) <jl...@jlli-mn1.linkedin.biz>
---
 .../batch/common/SegmentGenerationTaskRunner.java  |  3 ++-
 .../hadoop/job/mappers/SegmentCreationMapper.java  |  3 ++-
 .../job/mappers/SegmentPreprocessingMapper.java    |  3 ++-
 .../spark/jobs/SparkSegmentCreationFunction.java   |  3 ++-
 .../pinot/segment/local/utils/IngestionUtils.java  |  3 ++-
 .../name/NormalizedDateSegmentNameGenerator.java   | 22 +++++++++------
 .../creator/name/SegmentNameGeneratorFactory.java  |  2 +-
 .../NormalizedDateSegmentNameGeneratorTest.java    | 31 +++++++++++++---------
 .../segment/converter/SegmentMergeCommand.java     |  3 ++-
 9 files changed, 45 insertions(+), 28 deletions(-)

diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java
index 534c7d3..8757040 100644
--- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java
+++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java
@@ -157,7 +157,8 @@ public class SegmentGenerationTaskRunner implements Serializable {
         return new NormalizedDateSegmentNameGenerator(tableName, segmentNameGeneratorConfigs.get(SEGMENT_NAME_PREFIX),
             Boolean.parseBoolean(segmentNameGeneratorConfigs.get(EXCLUDE_SEQUENCE_ID)),
             IngestionConfigUtils.getBatchSegmentIngestionType(tableConfig),
-            IngestionConfigUtils.getBatchSegmentIngestionFrequency(tableConfig), dateTimeFormatSpec);
+            IngestionConfigUtils.getBatchSegmentIngestionFrequency(tableConfig), dateTimeFormatSpec,
+            segmentNameGeneratorConfigs.get(SEGMENT_NAME_POSTFIX));
       case INPUT_FILE_SEGMENT_NAME_GENERATOR:
         String inputFileUri = _taskSpec.getCustomProperty(BatchConfigProperties.INPUT_DATA_FILE_URI_KEY);
         return new InputFileSegmentNameGenerator(segmentNameGeneratorConfigs.get(FILE_PATH_PATTERN),
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java
index 5808b42..4291790 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java
@@ -165,7 +165,8 @@ public class SegmentCreationMapper extends Mapper<LongWritable, Text, LongWritab
             new NormalizedDateSegmentNameGenerator(_rawTableName, _jobConf.get(JobConfigConstants.SEGMENT_NAME_PREFIX),
                 _jobConf.getBoolean(JobConfigConstants.EXCLUDE_SEQUENCE_ID, false),
                 IngestionConfigUtils.getBatchSegmentIngestionType(_tableConfig),
-                IngestionConfigUtils.getBatchSegmentIngestionFrequency(_tableConfig), dateTimeFormatSpec);
+                IngestionConfigUtils.getBatchSegmentIngestionFrequency(_tableConfig), dateTimeFormatSpec,
+                _jobConf.get(JobConfigConstants.SEGMENT_NAME_POSTFIX));
         break;
       default:
         throw new UnsupportedOperationException("Unsupported segment name generator type: " + segmentNameGeneratorType);
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentPreprocessingMapper.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentPreprocessingMapper.java
index 9726c38..5ae8966 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentPreprocessingMapper.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentPreprocessingMapper.java
@@ -78,7 +78,8 @@ public class SegmentPreprocessingMapper
             _jobConf.get(InternalConfigConstants.SEGMENT_TIME_SDF_PATTERN));
       }
       _normalizedDateSegmentNameGenerator =
-          new NormalizedDateSegmentNameGenerator(tableName, null, false, "APPEND", pushFrequency, dateTimeFormatSpec);
+          new NormalizedDateSegmentNameGenerator(tableName, null, false, "APPEND", pushFrequency, dateTimeFormatSpec,
+              null);
       _sampleNormalizedTimeColumnValue = _normalizedDateSegmentNameGenerator.getNormalizedDate(timeColumnValue);
     }
 
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java
index 3f1ecf8..a5c7921 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java
@@ -132,7 +132,8 @@ public class SparkSegmentCreationFunction implements Serializable {
             new NormalizedDateSegmentNameGenerator(_rawTableName, _jobConf.get(JobConfigConstants.SEGMENT_NAME_PREFIX),
                 _jobConf.getBoolean(JobConfigConstants.EXCLUDE_SEQUENCE_ID, false),
                 IngestionConfigUtils.getBatchSegmentIngestionType(_tableConfig),
-                IngestionConfigUtils.getBatchSegmentIngestionFrequency(_tableConfig), dateTimeFormatSpec);
+                IngestionConfigUtils.getBatchSegmentIngestionFrequency(_tableConfig), dateTimeFormatSpec,
+                _jobConf.get(JobConfigConstants.SEGMENT_NAME_POSTFIX));
         break;
       default:
         throw new UnsupportedOperationException("Unsupported segment name generator type: " + segmentNameGeneratorType);
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java
index ec50e07..a8449cd 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java
@@ -158,7 +158,8 @@ public final class IngestionUtils {
           }
         }
         return new NormalizedDateSegmentNameGenerator(rawTableName, batchConfig.getSegmentNamePrefix(),
-            batchConfig.isExcludeSequenceId(), pushType, pushFrequency, dateTimeFormatSpec);
+            batchConfig.isExcludeSequenceId(), pushType, pushFrequency, dateTimeFormatSpec,
+            batchConfig.getSegmentNamePostfix());
 
       case BatchConfigProperties.SegmentNameGeneratorType.SIMPLE:
         return new SimpleSegmentNameGenerator(rawTableName, batchConfig.getSegmentNamePostfix());
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGenerator.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGenerator.java
index a1ae0f6..629c611 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGenerator.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGenerator.java
@@ -38,25 +38,27 @@ public class NormalizedDateSegmentNameGenerator implements SegmentNameGenerator
   // we will need to create a new top level module for such constants and define them there.
   private static final String PUSH_FREQUENCY_HOURLY = "hourly";
 
-  private String _segmentNamePrefix;
-  private boolean _excludeSequenceId;
-  private boolean _appendPushType;
+  private final String _segmentNamePrefix;
+  private final boolean _excludeSequenceId;
+  private final boolean _appendPushType;
+  private final String _segmentNamePostfix;
 
   // For APPEND tables
-  private SimpleDateFormat _outputSDF;
+  private final SimpleDateFormat _outputSDF;
   // For EPOCH time format
-  private TimeUnit _inputTimeUnit;
+  private final TimeUnit _inputTimeUnit;
   // For SIMPLE_DATE_FORMAT time format
-  private SimpleDateFormat _inputSDF;
+  private final SimpleDateFormat _inputSDF;
 
   public NormalizedDateSegmentNameGenerator(String tableName, @Nullable String segmentNamePrefix,
       boolean excludeSequenceId, @Nullable String pushType, @Nullable String pushFrequency,
-      @Nullable DateTimeFormatSpec dateTimeFormatSpec) {
+      @Nullable DateTimeFormatSpec dateTimeFormatSpec, @Nullable String segmentNamePostfix) {
     _segmentNamePrefix = segmentNamePrefix != null ? segmentNamePrefix.trim() : tableName;
     Preconditions.checkArgument(
         _segmentNamePrefix != null && isValidSegmentName(_segmentNamePrefix));
     _excludeSequenceId = excludeSequenceId;
     _appendPushType = "APPEND".equalsIgnoreCase(pushType);
+    _segmentNamePostfix = segmentNamePostfix;
 
     // Include time info for APPEND push type
     if (_appendPushType) {
@@ -96,7 +98,11 @@ public class NormalizedDateSegmentNameGenerator implements SegmentNameGenerator
       return JOINER.join(_segmentNamePrefix, getNormalizedDate(Preconditions.checkNotNull(minTimeValue)),
           getNormalizedDate(Preconditions.checkNotNull(maxTimeValue)), sequenceIdInSegmentName);
     } else {
-      return JOINER.join(_segmentNamePrefix, sequenceIdInSegmentName);
+      if (_segmentNamePostfix != null) {
+        return JOINER.join(_segmentNamePrefix, _segmentNamePostfix, sequenceIdInSegmentName);
+      } else {
+        return JOINER.join(_segmentNamePrefix, sequenceIdInSegmentName);
+      }
     }
   }
 
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/SegmentNameGeneratorFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/SegmentNameGeneratorFactory.java
index 85a8d14e..072572b 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/SegmentNameGeneratorFactory.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/name/SegmentNameGeneratorFactory.java
@@ -64,7 +64,7 @@ public class SegmentNameGeneratorFactory {
         }
         return new NormalizedDateSegmentNameGenerator(tableName, prefix, excludeSequenceId,
             IngestionConfigUtils.getBatchSegmentIngestionType(tableConfig),
-            IngestionConfigUtils.getBatchSegmentIngestionFrequency(tableConfig), dateTimeFormatSpec);
+            IngestionConfigUtils.getBatchSegmentIngestionFrequency(tableConfig), dateTimeFormatSpec, postfix);
       default:
         throw new UnsupportedOperationException("Unsupported segment name generator type: " + segmentNameGeneratorType);
     }
diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGeneratorTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGeneratorTest.java
index abc0e3e..02ab1d2 100644
--- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGeneratorTest.java
+++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/creator/name/NormalizedDateSegmentNameGeneratorTest.java
@@ -46,7 +46,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   @Test
   public void testRefresh() {
     SegmentNameGenerator segmentNameGenerator =
-        new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, REFRESH_PUSH_TYPE, null, null);
+        new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, REFRESH_PUSH_TYPE, null, null, null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=false");
     assertEquals(segmentNameGenerator.generateSegmentName(INVALID_SEQUENCE_ID, null, null), "myTable");
@@ -56,7 +56,8 @@ public class NormalizedDateSegmentNameGeneratorTest {
   @Test
   public void testWithSegmentNamePrefix() {
     SegmentNameGenerator segmentNameGenerator =
-        new NormalizedDateSegmentNameGenerator(TABLE_NAME, SEGMENT_NAME_PREFIX, false, REFRESH_PUSH_TYPE, null, null);
+        new NormalizedDateSegmentNameGenerator(TABLE_NAME, SEGMENT_NAME_PREFIX, false, REFRESH_PUSH_TYPE, null, null,
+            null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable_daily, appendPushType=false");
     assertEquals(segmentNameGenerator.generateSegmentName(INVALID_SEQUENCE_ID, null, null), "myTable_daily");
@@ -67,7 +68,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testWithUntrimmedSegmentNamePrefix() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, SEGMENT_NAME_PREFIX + "  ", false, REFRESH_PUSH_TYPE, null,
-            null);
+            null, null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable_daily, appendPushType=false");
     assertEquals(segmentNameGenerator.generateSegmentName(INVALID_SEQUENCE_ID, null, null), "myTable_daily");
@@ -77,7 +78,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   @Test
   public void testExcludeSequenceId() {
     SegmentNameGenerator segmentNameGenerator =
-        new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, true, REFRESH_PUSH_TYPE, null, null);
+        new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, true, REFRESH_PUSH_TYPE, null, null, null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=false, excludeSequenceId=true");
     assertEquals(segmentNameGenerator.generateSegmentName(INVALID_SEQUENCE_ID, null, null), "myTable");
@@ -87,7 +88,8 @@ public class NormalizedDateSegmentNameGeneratorTest {
   @Test
   public void testWithPrefixExcludeSequenceId() {
     SegmentNameGenerator segmentNameGenerator =
-        new NormalizedDateSegmentNameGenerator(TABLE_NAME, SEGMENT_NAME_PREFIX, true, REFRESH_PUSH_TYPE, null, null);
+        new NormalizedDateSegmentNameGenerator(TABLE_NAME, SEGMENT_NAME_PREFIX, true, REFRESH_PUSH_TYPE, null, null,
+            null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable_daily, appendPushType=false, "
             + "excludeSequenceId=true");
@@ -99,7 +101,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testAppend() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), EPOCH_TIME_FORMAT));
+            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), EPOCH_TIME_FORMAT), null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=true, outputSDF=yyyy-MM-dd, "
             + "inputTimeUnit=DAYS");
@@ -113,7 +115,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testHoursTimeType() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-            new DateTimeFormatSpec(1, TimeUnit.HOURS.toString(), EPOCH_TIME_FORMAT));
+            new DateTimeFormatSpec(1, TimeUnit.HOURS.toString(), EPOCH_TIME_FORMAT), null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=true, outputSDF=yyyy-MM-dd, "
             + "inputTimeUnit=HOURS");
@@ -127,7 +129,8 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testLongSimpleDateFormat() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, LONG_SIMPLE_DATE_FORMAT));
+            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, LONG_SIMPLE_DATE_FORMAT),
+            null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=true, outputSDF=yyyy-MM-dd, "
             + "inputSDF=yyyyMMdd");
@@ -141,7 +144,8 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testStringSimpleDateFormat() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SIMPLE_DATE_FORMAT));
+            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SIMPLE_DATE_FORMAT),
+            null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=true, outputSDF=yyyy-MM-dd, "
             + "inputSDF=yyyy-MM-dd");
@@ -155,7 +159,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testMalFormedTableNameAndSegmentNamePrefix() {
     try {
       new NormalizedDateSegmentNameGenerator(MALFORMED_TABLE_NAME, null, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-          new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SLASH_DATE_FORMAT));
+          new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SLASH_DATE_FORMAT), null);
       Assert.fail();
     } catch (IllegalArgumentException e) {
       // Expected
@@ -163,7 +167,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
     try {
       new NormalizedDateSegmentNameGenerator(
           TABLE_NAME, MALFORMED_SEGMENT_NAME_PREFIX, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-          new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SLASH_DATE_FORMAT));
+          new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SLASH_DATE_FORMAT), null);
       Assert.fail();
     } catch (IllegalArgumentException e) {
       // Expected
@@ -174,7 +178,8 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testMalFormedDateFormatAndTimeValue() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, APPEND_PUSH_TYPE, DAILY_PUSH_FREQUENCY,
-            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SLASH_DATE_FORMAT));
+            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), SIMPLE_DATE_TIME_FORMAT, STRING_SLASH_DATE_FORMAT),
+            null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, "
             + "appendPushType=true, outputSDF=yyyy-MM-dd, inputSDF=yyyy/MM/dd");
@@ -188,7 +193,7 @@ public class NormalizedDateSegmentNameGeneratorTest {
   public void testHourlyPushFrequency() {
     SegmentNameGenerator segmentNameGenerator =
         new NormalizedDateSegmentNameGenerator(TABLE_NAME, null, false, APPEND_PUSH_TYPE, HOURLY_PUSH_FREQUENCY,
-            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), EPOCH_TIME_FORMAT));
+            new DateTimeFormatSpec(1, TimeUnit.DAYS.toString(), EPOCH_TIME_FORMAT), null);
     assertEquals(segmentNameGenerator.toString(),
         "NormalizedDateSegmentNameGenerator: segmentNamePrefix=myTable, appendPushType=true, outputSDF=yyyy-MM-dd-HH,"
             + " inputTimeUnit=DAYS");
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/SegmentMergeCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/SegmentMergeCommand.java
index 278e0bd..18358a3 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/SegmentMergeCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/SegmentMergeCommand.java
@@ -259,7 +259,8 @@ public class SegmentMergeCommand extends AbstractBaseAdminCommand implements Com
 
     // Generate the final segment name using segment name generator
     NormalizedDateSegmentNameGenerator segmentNameGenerator =
-        new NormalizedDateSegmentNameGenerator(tableName, null, false, pushType, pushFrequency, dateTimeFormatSpec);
+        new NormalizedDateSegmentNameGenerator(tableName, null, false, pushType, pushFrequency, dateTimeFormatSpec,
+            null);
 
     return segmentNameGenerator.generateSegmentName(DEFAULT_SEQUENCE_ID, minStartTime, maxEndTime);
   }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org