You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2019/11/12 20:13:05 UTC

[incubator-pinot] 06/12: change offset mapping strategy for immutable seg to reduce memory

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

jamesshao pushed a commit to branch upsert
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 6df85f3da0259aee319b4360653e54ae61ba2067
Author: james Shao <sj...@uber.com>
AuthorDate: Thu Oct 31 17:14:23 2019 -0700

    change offset mapping strategy for immutable seg to reduce memory
    
    Summary:
    update offset mapping to docId to use array to reduce memory
    right now we use hashmap for such mapping. It is very expensive to do so
    changing to array for mapping will use 4 byte/document.
    
    We only do this for immutable segment as it is easier to controll their size,
    also immutable segment is the majority where we incur the memory cost
    
    Reviewers: csoman, tingchen, bzzhang
    
    Reviewed By: bzzhang
    
    Differential Revision: https://code.uberinternal.com/D3534691
---
 pinot-api/pom.xml                                  |   2 +-
 pinot-azure-filesystem/pom.xml                     |   2 +-
 pinot-broker/pom.xml                               |   2 +-
 pinot-common/pom.xml                               |   2 +-
 .../pinot-connector-kafka-0.11/pom.xml             |   2 +-
 pinot-connectors/pinot-connector-kafka-0.9/pom.xml |   2 +-
 pinot-connectors/pom.xml                           |   2 +-
 pinot-controller/pom.xml                           |   2 +-
 pinot-core/pom.xml                                 |   2 +-
 .../immutable/ImmutableUpsertSegmentImpl.java      | 111 +++++++++++++--------
 .../mutable/MutableUpsertSegmentImpl.java          |   2 +
 pinot-distribution/pom.xml                         |   2 +-
 pinot-grigio/pinot-grigio-common/pom.xml           |   2 +-
 pinot-grigio/pinot-grigio-coordinator/pom.xml      |   4 +-
 pinot-grigio/pom.xml                               |   2 +-
 pinot-hadoop-filesystem/pom.xml                    |   2 +-
 pinot-hadoop/pom.xml                               |   2 +-
 pinot-integration-tests/pom.xml                    |   2 +-
 pinot-minion/pom.xml                               |   2 +-
 pinot-orc/pom.xml                                  |   2 +-
 pinot-parquet/pom.xml                              |   2 +-
 pinot-perf/pom.xml                                 |   2 +-
 pinot-server/pom.xml                               |   2 +-
 pinot-tools/pom.xml                                |   2 +-
 pinot-transport/pom.xml                            |   2 +-
 pom.xml                                            |   2 +-
 26 files changed, 97 insertions(+), 66 deletions(-)

diff --git a/pinot-api/pom.xml b/pinot-api/pom.xml
index f26d749..6d6e96b 100644
--- a/pinot-api/pom.xml
+++ b/pinot-api/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-api</artifactId>
   <name>Pinot API</name>
diff --git a/pinot-azure-filesystem/pom.xml b/pinot-azure-filesystem/pom.xml
index 7f3e432..fe6c1c3 100644
--- a/pinot-azure-filesystem/pom.xml
+++ b/pinot-azure-filesystem/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-azure-filesystem</artifactId>
   <name>Pinot Azure Filesystem</name>
diff --git a/pinot-broker/pom.xml b/pinot-broker/pom.xml
index 36d6eb1..fbeb628 100644
--- a/pinot-broker/pom.xml
+++ b/pinot-broker/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-broker</artifactId>
   <name>Pinot Broker</name>
diff --git a/pinot-common/pom.xml b/pinot-common/pom.xml
index 1939448..91df4fe 100644
--- a/pinot-common/pom.xml
+++ b/pinot-common/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-common</artifactId>
   <name>Pinot Common</name>
diff --git a/pinot-connectors/pinot-connector-kafka-0.11/pom.xml b/pinot-connectors/pinot-connector-kafka-0.11/pom.xml
index 2e9b3f6..50ef1ad 100644
--- a/pinot-connectors/pinot-connector-kafka-0.11/pom.xml
+++ b/pinot-connectors/pinot-connector-kafka-0.11/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>pinot-connectors</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
     <relativePath>..</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/pinot-connectors/pinot-connector-kafka-0.9/pom.xml b/pinot-connectors/pinot-connector-kafka-0.9/pom.xml
index f63166c..409e3cf 100644
--- a/pinot-connectors/pinot-connector-kafka-0.9/pom.xml
+++ b/pinot-connectors/pinot-connector-kafka-0.9/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>pinot-connectors</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
     <relativePath>..</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/pinot-connectors/pom.xml b/pinot-connectors/pom.xml
index 4784801..749ec5c 100644
--- a/pinot-connectors/pom.xml
+++ b/pinot-connectors/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
 
   <artifactId>pinot-connectors</artifactId>
diff --git a/pinot-controller/pom.xml b/pinot-controller/pom.xml
index 9a326a3..0e0af54 100644
--- a/pinot-controller/pom.xml
+++ b/pinot-controller/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-controller</artifactId>
   <name>Pinot Controller</name>
diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml
index 577c235..a4e6ca8 100644
--- a/pinot-core/pom.xml
+++ b/pinot-core/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-core</artifactId>
   <name>Pinot Core</name>
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
index f12de6f..f42255f 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
@@ -20,7 +20,6 @@ package org.apache.pinot.core.indexsegment.immutable;
 
 import com.clearspring.analytics.util.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Multimap;
 import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.data.Schema;
@@ -44,6 +43,7 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -59,9 +59,10 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
   private final int _totalDoc;
   private final ColumnIndexContainer _offsetColumnIndexContainer;
   private final UpsertWaterMarkManager _upsertWaterMarkManager;
-  // right now we use map for this storage. But it will cost 12 bytes per record. It will translate to 12GB for 1B records
-  // will look into reducing this cost later
-  private final ImmutableMap<Long, Integer> _sourceOffsetToDocId;
+  private long _minSourceOffset;
+  // use array for mapping bewteen offset to docId, where actual offset = min_offset + array_index
+  // use 4 bytes per record
+  private int[] _sourceOffsetToDocIdArray;
 
   public ImmutableUpsertSegmentImpl(SegmentDirectory segmentDirectory,
                                     SegmentMetadataImpl segmentMetadata,
@@ -83,13 +84,15 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
         _virtualColumnsReaderWriter.add((VirtualColumnLongValueReaderWriter) container.getForwardIndex());
       }
     }
-    _sourceOffsetToDocId = buildOffsetToDocIdMap();
+     buildOffsetToDocIdMap();
   }
 
-  private ImmutableMap<Long, Integer> buildOffsetToDocIdMap() {
+  private void buildOffsetToDocIdMap() {
     final DataFileReader reader = _offsetColumnIndexContainer.getForwardIndex();
     final Dictionary dictionary = _offsetColumnIndexContainer.getDictionary();
-    ImmutableMap.Builder<Long, Integer> kafkaOffsetToDocIdBuilder = ImmutableMap.builder();
+    Map<Long, Integer> kafkaOffsetToDocIdMap = new HashMap<>();
+    long minOffset = Long.MAX_VALUE;
+    long maxOffset = 0;
     if (reader instanceof BaseSingleColumnSingleValueReader) {
       BaseSingleColumnSingleValueReader scsvReader = (BaseSingleColumnSingleValueReader) reader;
       for (int docId = 0; docId < _totalDoc; docId++) {
@@ -102,10 +105,21 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
         if (offset == null) {
           LOGGER.error("kafka offset is null at docID {}", docId);
         } else {
-          kafkaOffsetToDocIdBuilder.put(offset, docId);
+          minOffset = Math.min(offset, minOffset);
+          maxOffset = Math.max(offset, maxOffset);
+          kafkaOffsetToDocIdMap.put(offset, docId);
+        }
+      }
+      _minSourceOffset = minOffset;
+      int size = Math.toIntExact(maxOffset - minOffset + 1);
+      _sourceOffsetToDocIdArray = new int[size];
+      for (int i = 0; i < size; i++) {
+        if (kafkaOffsetToDocIdMap.containsKey(i + minOffset)) {
+          _sourceOffsetToDocIdArray[i] = kafkaOffsetToDocIdMap.get(i + minOffset);
+        } else {
+          _sourceOffsetToDocIdArray[i] = -1;
         }
       }
-      return kafkaOffsetToDocIdBuilder.build();
     } else {
       throw new RuntimeException("unexpected forward reader type for kafka offset column " + reader.getClass());
     }
@@ -121,30 +135,22 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
   public void updateVirtualColumn(List<UpdateLogEntry> logEntryList) {
     for (UpdateLogEntry logEntry: logEntryList) {
       boolean updated = false;
-      Integer docId = _sourceOffsetToDocId.get(logEntry.getOffset());
-      if (docId == null) {
-        LOGGER.warn("segment {} failed to found docId for log update entry {}", _segmentName, logEntry.toString());
-      } else {
-        for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-          updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
-        }
-        if (updated) {
-          _upsertWaterMarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
-        }
+      int docId = getDocIdFromSourceOffset(logEntry.getOffset());
+      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+        updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
+      }
+      if (updated) {
+        _upsertWaterMarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
       }
     }
   }
 
   @Override
   public String getVirtualColumnInfo(long offset) {
-    Integer docId = _sourceOffsetToDocId.get(offset);
+    int docId = getDocIdFromSourceOffset(offset);
     StringBuilder result = new StringBuilder("matched: ");
-    if (docId == null) {
-      result = new StringBuilder("no doc id found ");
-    } else {
-      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-        result.append(readerWriter.getLong(docId)).append("; ");
-      }
+    for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+      result.append(readerWriter.getLong(docId)).append("; ");
     }
     return result.toString();
   }
@@ -162,25 +168,50 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
     for (UpdateLogEntry logEntry: updateLogEntries) {
       updateLogEntryMap.put(logEntry.getOffset(), logEntry);
     }
-    for (Map.Entry<Long, Integer> mapEntry: _sourceOffsetToDocId.entrySet()) {
-      final long offset = mapEntry.getKey();
-      final int docId = mapEntry.getValue();
-      if (updateLogEntryMap.containsKey(offset)) {
-        boolean updated = false;
-        Collection<UpdateLogEntry> entries = updateLogEntryMap.get(offset);
-        UpdateLogEntry lastEntry = null;
-        for (UpdateLogEntry entry : entries) {
-          lastEntry = entry;
-          for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-            updated = readerWriter.update(docId, entry.getValue(), entry.getType()) || updated;
+    for (int i = 0; i < _sourceOffsetToDocIdArray.length; i++) {
+      if (_sourceOffsetToDocIdArray[i] != -1) {
+        final long offset = i + _minSourceOffset;
+        final int docId = _sourceOffsetToDocIdArray[i];
+        if (updateLogEntryMap.containsKey(offset)) {
+          boolean updated = false;
+          Collection<UpdateLogEntry> entries = updateLogEntryMap.get(offset);
+          UpdateLogEntry lastEntry = null;
+          for (UpdateLogEntry entry : entries) {
+            lastEntry = entry;
+            for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+              updated = readerWriter.update(docId, entry.getValue(), entry.getType()) || updated;
+            }
+          }
+          if (updated) {
+            _upsertWaterMarkManager.processMessage(_tableNameWithType, _segmentName, lastEntry);
           }
-        }
-        if (updated) {
-          _upsertWaterMarkManager.processMessage(_tableNameWithType, _segmentName, lastEntry);
         }
       }
     }
     LOGGER.info("loaded {} update log entries for current immutable segment {} in {} ms", updateLogEntries.size(),
         _segmentName, System.currentTimeMillis() - start);
   }
+
+  /**
+   * given a offset from source kafka topic, return the docId associated with it
+   * throw exception if there is no docId for the associated kafka offset (because kafka offset might not be continuous)
+   * @param offset offset in the source topic
+   * @return the corresponding docId
+   */
+  private int getDocIdFromSourceOffset(long offset) throws RuntimeException {
+    if (offset < _minSourceOffset || offset - _minSourceOffset >= _sourceOffsetToDocIdArray.length) {
+      LOGGER.error("offset {} is outside range for current segment {} start offset {} size {}",
+          offset, _segmentName, _minSourceOffset, _sourceOffsetToDocIdArray.length);
+      throw new RuntimeException("offset outside range");
+    } else {
+      int position = Math.toIntExact(offset - _minSourceOffset);
+      if (_sourceOffsetToDocIdArray[position] == -1) {
+        LOGGER.error("no docId associated with offset {} for segment {}", offset, _segmentName);
+        throw new RuntimeException("docId not found");
+      } else {
+        return _sourceOffsetToDocIdArray[position];
+      }
+    }
+
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
index 253c76f..b4d2e15 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
@@ -45,6 +45,8 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
   private final String _kafkaOffsetColumnName;
 
   private final List<VirtualColumnLongValueReaderWriter> _mutableSegmentReaderWriters = new ArrayList<>();
+  // use map for mapping between kafka offset and docId because we at-most have 1 mutable segment per consumer
+  // will use more memory, but we can update this later
   private final Map<Long, Integer> _sourceOffsetToDocId = new ConcurrentHashMap<>();
   // to store the update event that arrive before my current record
   // TODO remove this in the later version of design if necessary
diff --git a/pinot-distribution/pom.xml b/pinot-distribution/pom.xml
index 49e3d9c..fb652d0 100644
--- a/pinot-distribution/pom.xml
+++ b/pinot-distribution/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-distribution</artifactId>
   <name>Pinot Distribution</name>
diff --git a/pinot-grigio/pinot-grigio-common/pom.xml b/pinot-grigio/pinot-grigio-common/pom.xml
index 53adbbf..6d9d4bb 100644
--- a/pinot-grigio/pinot-grigio-common/pom.xml
+++ b/pinot-grigio/pinot-grigio-common/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>pinot-grigio</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/pinot-grigio/pinot-grigio-coordinator/pom.xml b/pinot-grigio/pinot-grigio-coordinator/pom.xml
index a7d6db2..bc85f0f 100644
--- a/pinot-grigio/pinot-grigio-coordinator/pom.xml
+++ b/pinot-grigio/pinot-grigio-coordinator/pom.xml
@@ -23,16 +23,14 @@
   <parent>
     <artifactId>pinot-grigio</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
-
   <artifactId>pinot-grigio-coordinator</artifactId>
   <name>Pinot Grigio Coordinator</name>
   <properties>
     <pinot.root>${basedir}/../..</pinot.root>
   </properties>
-
   <dependencies>
     <dependency>
       <groupId>org.glassfish.jersey.containers</groupId>
diff --git a/pinot-grigio/pom.xml b/pinot-grigio/pom.xml
index 5ac6c5a..2766d3a 100644
--- a/pinot-grigio/pom.xml
+++ b/pinot-grigio/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>pinot-grigio</artifactId>
diff --git a/pinot-hadoop-filesystem/pom.xml b/pinot-hadoop-filesystem/pom.xml
index e130a69..6d44587 100644
--- a/pinot-hadoop-filesystem/pom.xml
+++ b/pinot-hadoop-filesystem/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-hadoop-filesystem</artifactId>
   <name>Pinot Hadoop Filesystem</name>
diff --git a/pinot-hadoop/pom.xml b/pinot-hadoop/pom.xml
index 2be8498..2272b24 100644
--- a/pinot-hadoop/pom.xml
+++ b/pinot-hadoop/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-hadoop</artifactId>
   <name>Pinot Hadoop</name>
diff --git a/pinot-integration-tests/pom.xml b/pinot-integration-tests/pom.xml
index 46ac371..1095763 100644
--- a/pinot-integration-tests/pom.xml
+++ b/pinot-integration-tests/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-integration-tests</artifactId>
   <name>Pinot Integration Tests</name>
diff --git a/pinot-minion/pom.xml b/pinot-minion/pom.xml
index 4d37500..a7352ee 100644
--- a/pinot-minion/pom.xml
+++ b/pinot-minion/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-minion</artifactId>
   <name>Pinot Minion</name>
diff --git a/pinot-orc/pom.xml b/pinot-orc/pom.xml
index 2e8672a..e046ca8 100644
--- a/pinot-orc/pom.xml
+++ b/pinot-orc/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/pinot-parquet/pom.xml b/pinot-parquet/pom.xml
index 690924a..5c5b9b0 100644
--- a/pinot-parquet/pom.xml
+++ b/pinot-parquet/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/pinot-perf/pom.xml b/pinot-perf/pom.xml
index 113d974..3312026 100644
--- a/pinot-perf/pom.xml
+++ b/pinot-perf/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-perf</artifactId>
   <name>Pinot Perf</name>
diff --git a/pinot-server/pom.xml b/pinot-server/pom.xml
index c1536cb..9463c64 100644
--- a/pinot-server/pom.xml
+++ b/pinot-server/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-server</artifactId>
   <name>Pinot Server</name>
diff --git a/pinot-tools/pom.xml b/pinot-tools/pom.xml
index 4078c18..7aa523f 100644
--- a/pinot-tools/pom.xml
+++ b/pinot-tools/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-tools</artifactId>
   <name>Pinot Tools</name>
diff --git a/pinot-transport/pom.xml b/pinot-transport/pom.xml
index 7b31dd0..dc748c7 100644
--- a/pinot-transport/pom.xml
+++ b/pinot-transport/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>pinot</artifactId>
     <groupId>org.apache.pinot</groupId>
-    <version>0.2.2.5.44-SNAPSHOT</version>
+    <version>0.2.2.5.45-SNAPSHOT</version>
   </parent>
   <artifactId>pinot-transport</artifactId>
   <name>Pinot Transport</name>
diff --git a/pom.xml b/pom.xml
index d43b23c..c3d0cf0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -30,7 +30,7 @@
 
   <groupId>org.apache.pinot</groupId>
   <artifactId>pinot</artifactId>
-  <version>0.2.2.5.44-SNAPSHOT</version>
+  <version>0.2.2.5.45-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Pinot</name>
   <description>A realtime distributed OLAP datastore</description>


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