Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577304282


##
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##
@@ -76,22 +79,52 @@ void init(InstanceDataManagerConfig 
instanceDataManagerConfig, TableConfig table
 
   boolean isShutDown();
 
+  /**
+   * Returns the segment lock for a segment in the table.
+   */
+  Lock getSegmentLock(String segmentName);
+
+  /**
+   * Returns whether the segment is loaded in the table.
+   */
+  boolean hasSegment(String segmentName);
+
   /**
* Adds a loaded immutable segment into the table.
*/
+  @VisibleForTesting
   void addSegment(ImmutableSegment immutableSegment);

Review Comment:
   That requires refactoring some classes. I think we can do it separately. 
Added some more javadoc explaining it is not designed for direct use



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577299251


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##
@@ -570,165 +563,57 @@ private void handleUpsert(ImmutableSegment 
immutableSegment) {
   _logger.info("Preloaded immutable segment: {} to upsert-enabled table: 
{}", segmentName, _tableNameWithType);
   return;
 }
-// Replacing segment takes multiple steps, and particularly need to access 
the oldSegment. Replace segment may
-// happen in two threads, i.e. the consuming thread that's committing the 
mutable segment and a HelixTaskExecutor
-// thread that's bringing segment from ONLINE to CONSUMING when the server 
finds consuming thread can't commit
-// the segment in time. The slower thread takes the reference of the 
oldSegment here, but it may get closed by
-// the faster thread if not synchronized. In particular, the slower thread 
may iterate the primary keys in the
-// oldSegment, causing seg fault. So we have to take a lock here.
-// However, we can't just reuse the existing segmentLocks. Because many 
methods of partitionUpsertMetadataManager
-// takes this lock internally, but after taking snapshot RW lock. If we 
take segmentLock here (before taking
-// snapshot RW lock), we can get into deadlock with threads calling 
partitionUpsertMetadataManager's other
-// methods, like removeSegment.
-// Adding segment should be done by a single HelixTaskExecutor thread, but 
do it with lock here for simplicity
-// otherwise, we'd need to double-check if oldSegmentManager is null.
-Lock segmentLock = SEGMENT_UPSERT_LOCKS.getLock(_tableNameWithType, 
segmentName);
-segmentLock.lock();
-try {
-  SegmentDataManager oldSegmentManager = 
_segmentDataManagerMap.get(segmentName);
-  if (oldSegmentManager == null) {
-// When adding a new segment, we should register it 'before' it is 
fully initialized by
-// partitionUpsertMetadataManager. Because when processing docs in the 
new segment, the docs in the other
-// segments may be invalidated, making the queries see less valid docs 
than expected. We should let query
-// access the new segment asap even though its validDocId bitmap is 
still being filled by
-// partitionUpsertMetadataManager.
-registerSegment(segmentName, newSegmentManager);
-partitionUpsertMetadataManager.addSegment(immutableSegment);
-_logger.info("Added new immutable segment: {} to upsert-enabled table: 
{}", segmentName, _tableNameWithType);
-  } else {
-// When replacing a segment, we should register the new segment 
'after' it is fully initialized by
-// partitionUpsertMetadataManager to fill up its validDocId bitmap. 
Otherwise, the queries will lose the access
-// to the valid docs in the old segment immediately, but the 
validDocId bitmap of the new segment is still
-// being filled by partitionUpsertMetadataManager, making the queries 
see less valid docs than expected.
-// When replacing a segment, the new and old segments are assumed to 
have same set of valid docs for data
-// consistency, otherwise the new segment should be named differently 
to go through the addSegment flow above.
-IndexSegment oldSegment = oldSegmentManager.getSegment();
-partitionUpsertMetadataManager.replaceSegment(immutableSegment, 
oldSegment);
-registerSegment(segmentName, newSegmentManager);
-_logger.info("Replaced {} segment: {} of upsert-enabled table: {}",
-oldSegment instanceof ImmutableSegment ? "immutable" : "mutable", 
segmentName, _tableNameWithType);
-releaseSegment(oldSegmentManager);
-  }
-} finally {
-  segmentLock.unlock();
-}
-  }
-
-  @Override
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata 
zkMetadata) {
-// Cannot download consuming segment
-if (zkMetadata.getStatus() == Status.IN_PROGRESS) {
-  return false;
-}
-// TODO: may support download from peer servers as well.
-return !METADATA_URI_FOR_PEER_DOWNLOAD.equals(zkMetadata.getDownloadUrl());
-  }
-
-  void downloadAndReplaceSegment(String segmentName, SegmentZKMetadata 
segmentZKMetadata,
-  IndexLoadingConfig indexLoadingConfig, TableConfig tableConfig) {
-String uri = segmentZKMetadata.getDownloadUrl();
-if (!METADATA_URI_FOR_PEER_DOWNLOAD.equals(uri)) {
-  try {
-// TODO: cleanup and consolidate the segment loading logic a bit for 
OFFLINE and REALTIME tables.
-//   https://github.com/apache/pinot/issues/9752
-downloadSegmentFromDeepStore(segmentName, indexLoadingConfig, uri);
-  } catch (Exception e) {
-_logger.warn("Download segment {} from deepstore uri {} failed.", 
segmentName, uri, e);
-// Download from deep store failed; 

Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577266557


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java:
##
@@ -752,14 +752,20 @@ public void run() {
   _state = State.DISCARDED;
   break;
 case DEFAULT:
-  success = buildSegmentAndReplace();
-  if (success) {
-_state = State.RETAINED;
-  } else {
-// Could not build segment for some reason. We can only 
download it.
-_state = State.ERROR;
-_segmentLogger.error("Could not build segment for {}", 
_segmentNameStr);
+  // Lock the segment to avoid multiple threads touching the 
same segment.
+  Lock segmentLock = 
_realtimeTableDataManager.getSegmentLock(_segmentNameStr);
+  segmentLock.lock();
+  try {
+if (buildSegmentAndReplace()) {

Review Comment:
   It is intentional to lock here. This is the control flow from the consuming 
segment. `buildSegmentAndReplace()` can be invoked from Helix thread as well, 
where it is already locked.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577264671


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java:
##
@@ -608,172 +721,153 @@ protected SegmentDataManager unregisterSegment(String 
segmentName) {
 }
   }
 
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata 
zkMetadata) {
-return true;
-  }
-
-  protected File downloadSegment(String segmentName, SegmentZKMetadata 
zkMetadata)
-  throws Exception {
-// TODO: may support download from peer servers for RealTime table.
-return downloadSegmentFromDeepStore(segmentName, zkMetadata);
-  }
-
-  private File downloadSegmentFromDeepStore(String segmentName, 
SegmentZKMetadata zkMetadata)
+  /**
+   * Downloads an immutable segment into the index directory.
+   * Segment can be downloaded from deep store or from peer servers. 
Downloaded segment might be compressed or
+   * encrypted, and this method takes care of decompressing and decrypting the 
segment.
+   */
+  protected File downloadSegment(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
-if (_isStreamSegmentDownloadUntar && zkMetadata.getCrypterName() == null) {
-  try {
-File untaredSegDir = downloadAndStreamUntarWithRateLimit(segmentName, 
zkMetadata, tempRootDir,
-_streamSegmentDownloadUntarRateLimitBytesPerSec);
-return moveSegment(segmentName, untaredSegDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-} else {
-  try {
-File tarFile = downloadAndDecrypt(segmentName, zkMetadata, 
tempRootDir);
-return untarAndMoveSegment(segmentName, tarFile, tempRootDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-}
-  }
-
-  private File moveSegment(String segmentName, File untaredSegDir)
-  throws IOException {
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+Preconditions.checkState(downloadUrl != null,
+"Failed to find download URL in ZK metadata for segment: %s of table: 
%s", segmentName, _tableNameWithType);
 try {
-  File indexDir = getSegmentDataDir(segmentName);
-  FileUtils.deleteDirectory(indexDir);
-  FileUtils.moveDirectory(untaredSegDir, indexDir);
-  return indexDir;
+  if 
(!CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD.equals(downloadUrl)) {
+try {
+  return downloadSegmentFromDeepStore(zkMetadata);
+} catch (Exception e) {
+  if (_peerDownloadScheme != null) {
+return downloadSegmentFromPeers(zkMetadata);
+  } else {
+throw e;
+  }
+}
+  } else {
+return downloadSegmentFromPeers(zkMetadata);
+  }
 } catch (Exception e) {
-  LOGGER.error("Failed to move segment: {} of table: {}", segmentName, 
_tableNameWithType);
-  _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.SEGMENT_DIR_MOVEMENT_FAILURES, 1L);
+  _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.SEGMENT_DOWNLOAD_FAILURES, 1);
   throw e;
 }
   }
 
   @VisibleForTesting
-  File downloadAndDecrypt(String segmentName, SegmentZKMetadata zkMetadata, 
File tempRootDir)
+  File downloadSegmentFromDeepStore(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tarFile = new File(tempRootDir, segmentName + 
TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION);
-String uri = zkMetadata.getDownloadUrl();
-boolean downloadSuccess = false;
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+_logger.info("Downloading segment: {} from: {}", segmentName, downloadUrl);
+File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
+if (_segmentDownloadSemaphore != null) {
+  long startTime = System.currentTimeMillis();
+  _logger.info("Acquiring segment download semaphore for segment: {}, 
queue-length: {} ", segmentName,
+  _segmentDownloadSemaphore.getQueueLength());
+  _segmentDownloadSemaphore.acquire();
+  _logger.info("Acquired segment download semaphore for segment: {} 
(lock-time={}ms, queue-length={}).",
+  segmentName, System.currentTimeMillis() - startTime, 
_segmentDownloadSemaphore.getQueueLength());
+}
 try {
-  if (_segmentDownloadSemaphore != null) {
-long startTime = System.currentTimeMillis();
-LOGGER.info("Trying to acquire segment download semaphore for: {}. 
queue-length: {} ", segmentName,
-_segmentDownloadSemaphore.getQueueLength());
-_segmentDownloadSemaphore.acquire();
-LOGGER.info("Acquired segment download semaphore for: {} 
(lock-time={}ms, queue-length={}).", segmentName,
- 

(pinot) branch master updated: Upgrade Pulsar to 3.2.2 (#12967)

2024-04-23 Thread kharekartik
This is an automated email from the ASF dual-hosted git repository.

kharekartik 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 bc9e8ee541 Upgrade Pulsar to 3.2.2 (#12967)
bc9e8ee541 is described below

commit bc9e8ee5413c8611fe2be3ed6c3d7073e750d608
Author: Xiaotian (Jackie) Jiang <1751+jackie-ji...@users.noreply.github.com>
AuthorDate: Tue Apr 23 21:43:12 2024 -0700

Upgrade Pulsar to 3.2.2 (#12967)
---
 .../pinot-stream-ingestion/pinot-pulsar/pom.xml| 134 +++--
 .../pinot/plugin/stream/pulsar/PulsarUtils.java|  27 ++---
 .../plugin/stream/pulsar/PulsarConsumerTest.java   |   2 +-
 pinot-tools/pom.xml|  38 --
 pom.xml|  31 ++---
 5 files changed, 37 insertions(+), 195 deletions(-)

diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml 
b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml
index f742f52f45..cb13fb9bba 100644
--- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml
+++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml
@@ -37,137 +37,33 @@
   
 package
 ${basedir}/../../..
-0.16.0
-1.63.0
-1.63.0
-2.6.2
-1.17
+3.2.2
+1.19.7
   
 
   
-
-  org.testcontainers
-  pulsar
-  1.19.7
-  test
-
-
-  org.mockito
-  mockito-core
-  test
-
 
   org.apache.pulsar
-  pulsar-client-original
-  
-
-  commons-configuration
-  commons-configuration
-
-
-  org.bouncycastle
-  bcpkix-jdk15on
-
-
-  org.bouncycastle
-  bcprov-ext-jdk15on
-
-
-  org.eclipse.jetty
-  jetty-util
-
-  
+  pulsar-client
+  ${pulsar.version}
 
+
 
   org.apache.pulsar
-  pulsar-client-admin-original
-
-
-  org.glassfish.jersey.core
-  jersey-server
-
-
-  org.glassfish.jersey.containers
-  jersey-container-grizzly2-http
-
-
-  org.glassfish.jersey.containers
-  jersey-container-servlet-core
-
-
-  io.netty
-  netty-resolver
-
-
-  io.prometheus
-  simpleclient_common
-  ${simpleclient_common.version}
-
-
-  com.google.api.grpc
-  proto-google-common-protos
-
-
-  io.grpc
-  grpc-context
-  ${grpc-context.version}
-
-
-  io.grpc
-  grpc-protobuf-lite
-  ${grpc-protobuf-lite.version}
-
-
-  io.prometheus
-  simpleclient
-  ${simpleclient_common.version}
-
-
-  org.eclipse.jetty
-  jetty-server
-
-
-  org.eclipse.jetty
-  jetty-servlet
-
-
-  com.squareup.okio
-  okio
-
-
-  io.prometheus
-  simpleclient_hotspot
-  ${simpleclient_common.version}
-
-
-  org.codehaus.mojo
-  animal-sniffer-annotations
-  ${codehaus-annotations.version}
-
-
-  com.github.ben-manes.caffeine
-  caffeine
-  ${caffeine.version}
-
-
-  io.netty
-  netty-codec-socks
-
-
-  org.bouncycastle
-  bcpkix-jdk15to18
-
-
-  org.bouncycastle
-  bcprov-ext-jdk15to18
+  pulsar-client-admin
+  ${pulsar.version}
+  test
 
 
-  org.bouncycastle
-  bcprov-jdk15to18
+  org.testcontainers
+  pulsar
+  ${testcontainers.pulsar.version}
+  test
 
 
-  org.apache.pinot
-  pinot-spi
+  org.mockito
+  mockito-core
+  test
 
   
 
diff --git 
a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java
 
b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java
index 0ccacc3047..e1b7b50c21 100644
--- 
a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java
+++ 
b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 import java.nio.ByteBuffer;
 import java.util.Base64;
+import java.util.BitSet;
 import java.util.Map;
 import java.util.Set;
 import org.apache.commons.lang3.StringUtils;
@@ -31,12 +32,11 @@ import org.apache.pinot.spi.stream.OffsetCriteria;
 import org.apache.pinot.spi.stream.StreamMessageMetadata;
 import org.apache.pulsar.client.api.Message;
 import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MessageIdAdv;
 import org.apache.pulsar.client.api.Reader;
 import org.apache.pulsar.client.api.SubscriptionInitialPosition;
-import 

Re: [PR] Upgrade Pulsar to 3.2.2 [pinot]

2024-04-23 Thread via GitHub


KKcorps merged PR #12967:
URL: https://github.com/apache/pinot/pull/12967


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [I] Ingest from Pulsar: relying on/ compatible to EOL version of Pulsar [pinot]

2024-04-23 Thread via GitHub


KKcorps closed issue #12713: Ingest from Pulsar: relying on/ compatible to EOL 
version of Pulsar
URL: https://github.com/apache/pinot/issues/12713


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Refactor WindowFunction process rows in batched partition [pinot]

2024-04-23 Thread via GitHub


codecov-commenter commented on PR #12993:
URL: https://github.com/apache/pinot/pull/12993#issuecomment-2073982624

   ## 
[Codecov](https://app.codecov.io/gh/apache/pinot/pull/12993?dropdown=coverage=pr=h1_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 Report
   Attention: Patch coverage is `84.40860%` with `29 lines` in your changes are 
missing coverage. Please review.
   > Project coverage is 35.03%. Comparing base 
[(`59551e4`)](https://app.codecov.io/gh/apache/pinot/commit/59551e45224f1535c4863fd577622b37366ccc97?dropdown=coverage=desc_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 to head 
[(`e1a8fec`)](https://app.codecov.io/gh/apache/pinot/pull/12993?dropdown=coverage=pr=desc_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache).
   > Report is 355 commits behind head on master.
   
   | 
[Files](https://app.codecov.io/gh/apache/pinot/pull/12993?dropdown=coverage=pr=tree_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | Patch % | Lines |
   |---|---|---|
   | 
[...ime/operator/window/range/RangeWindowFunction.java](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree=pinot-query-runtime%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fpinot%2Fquery%2Fruntime%2Foperator%2Fwindow%2Frange%2FRangeWindowFunction.java_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci93aW5kb3cvcmFuZ2UvUmFuZ2VXaW5kb3dGdW5jdGlvbi5qYXZh)
 | 68.18% | [4 Missing and 3 partials :warning: 
](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 |
   | 
[...ator/window/aggregate/AggregateWindowFunction.java](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree=pinot-query-runtime%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fpinot%2Fquery%2Fruntime%2Foperator%2Fwindow%2Faggregate%2FAggregateWindowFunction.java_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci93aW5kb3cvYWdncmVnYXRlL0FnZ3JlZ2F0ZVdpbmRvd0Z1bmN0aW9uLmphdmE=)
 | 88.23% | [2 Missing and 4 partials :warning: 
](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 |
   | 
[...perator/window/value/FirstValueWindowFunction.java](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree=pinot-query-runtime%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fpinot%2Fquery%2Fruntime%2Foperator%2Fwindow%2Fvalue%2FFirstValueWindowFunction.java_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci93aW5kb3cvdmFsdWUvRmlyc3RWYWx1ZVdpbmRvd0Z1bmN0aW9uLmphdmE=)
 | 0.00% | [3 Missing :warning: 
](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 |
   | 
[.../operator/window/value/LagValueWindowFunction.java](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree=pinot-query-runtime%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fpinot%2Fquery%2Fruntime%2Foperator%2Fwindow%2Fvalue%2FLagValueWindowFunction.java_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci93aW5kb3cvdmFsdWUvTGFnVmFsdWVXaW5kb3dGdW5jdGlvbi5qYXZh)
 | 0.00% | [3 Missing :warning: 
](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 |
   | 
[...operator/window/value/LastValueWindowFunction.java](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree=pinot-query-runtime%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fpinot%2Fquery%2Fruntime%2Foperator%2Fwindow%2Fvalue%2FLastValueWindowFunction.java_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci93aW5kb3cvdmFsdWUvTGFzdFZhbHVlV2luZG93RnVuY3Rpb24uamF2YQ==)
 | 0.00% | [3 Missing :warning: 
](https://app.codecov.io/gh/apache/pinot/pull/12993?src=pr=tree_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 |
   | 

[PR] Refactor WindowFunction process rows in batched partition [pinot]

2024-04-23 Thread via GitHub


xiangfu0 opened a new pull request, #12993:
URL: https://github.com/apache/pinot/pull/12993

   1. Partition the data to containers first
   2. For each partition, generate the window agg results. using merger 
functions for now


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


abhioncbr commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1577155394


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -128,13 +155,34 @@ public static PropertiesConfiguration fromFile(File file, 
boolean setIOFactory,
 return config;
   }
 
+  /**
+   * save the segment metadata configuration content into the provided file 
based on the version header.
+   * @param propertiesConfiguration a {@link PropertiesConfiguration} instance.
+   * @param file a {@link File} instance.
+   * @param versionHeader a {@link String} instance.
+   */
+  public static void saveSegmentMetadataToFile(PropertiesConfiguration 
propertiesConfiguration, File file,

Review Comment:
   not merging two checks because, in-future, we might want to set the version 
header with a different IO factory.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577055850


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java:
##
@@ -608,172 +721,153 @@ protected SegmentDataManager unregisterSegment(String 
segmentName) {
 }
   }
 
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata 
zkMetadata) {
-return true;
-  }
-
-  protected File downloadSegment(String segmentName, SegmentZKMetadata 
zkMetadata)
-  throws Exception {
-// TODO: may support download from peer servers for RealTime table.
-return downloadSegmentFromDeepStore(segmentName, zkMetadata);
-  }
-
-  private File downloadSegmentFromDeepStore(String segmentName, 
SegmentZKMetadata zkMetadata)
+  /**
+   * Downloads an immutable segment into the index directory.
+   * Segment can be downloaded from deep store or from peer servers. 
Downloaded segment might be compressed or
+   * encrypted, and this method takes care of decompressing and decrypting the 
segment.
+   */
+  protected File downloadSegment(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
-if (_isStreamSegmentDownloadUntar && zkMetadata.getCrypterName() == null) {
-  try {
-File untaredSegDir = downloadAndStreamUntarWithRateLimit(segmentName, 
zkMetadata, tempRootDir,
-_streamSegmentDownloadUntarRateLimitBytesPerSec);
-return moveSegment(segmentName, untaredSegDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-} else {
-  try {
-File tarFile = downloadAndDecrypt(segmentName, zkMetadata, 
tempRootDir);
-return untarAndMoveSegment(segmentName, tarFile, tempRootDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-}
-  }
-
-  private File moveSegment(String segmentName, File untaredSegDir)
-  throws IOException {
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+Preconditions.checkState(downloadUrl != null,
+"Failed to find download URL in ZK metadata for segment: %s of table: 
%s", segmentName, _tableNameWithType);
 try {
-  File indexDir = getSegmentDataDir(segmentName);
-  FileUtils.deleteDirectory(indexDir);
-  FileUtils.moveDirectory(untaredSegDir, indexDir);
-  return indexDir;
+  if 
(!CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD.equals(downloadUrl)) {
+try {
+  return downloadSegmentFromDeepStore(zkMetadata);
+} catch (Exception e) {
+  if (_peerDownloadScheme != null) {
+return downloadSegmentFromPeers(zkMetadata);
+  } else {
+throw e;
+  }
+}
+  } else {
+return downloadSegmentFromPeers(zkMetadata);
+  }
 } catch (Exception e) {
-  LOGGER.error("Failed to move segment: {} of table: {}", segmentName, 
_tableNameWithType);
-  _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.SEGMENT_DIR_MOVEMENT_FAILURES, 1L);
+  _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.SEGMENT_DOWNLOAD_FAILURES, 1);
   throw e;
 }
   }
 
   @VisibleForTesting
-  File downloadAndDecrypt(String segmentName, SegmentZKMetadata zkMetadata, 
File tempRootDir)
+  File downloadSegmentFromDeepStore(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tarFile = new File(tempRootDir, segmentName + 
TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION);
-String uri = zkMetadata.getDownloadUrl();
-boolean downloadSuccess = false;
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+_logger.info("Downloading segment: {} from: {}", segmentName, downloadUrl);
+File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
+if (_segmentDownloadSemaphore != null) {
+  long startTime = System.currentTimeMillis();
+  _logger.info("Acquiring segment download semaphore for segment: {}, 
queue-length: {} ", segmentName,
+  _segmentDownloadSemaphore.getQueueLength());
+  _segmentDownloadSemaphore.acquire();
+  _logger.info("Acquired segment download semaphore for segment: {} 
(lock-time={}ms, queue-length={}).",
+  segmentName, System.currentTimeMillis() - startTime, 
_segmentDownloadSemaphore.getQueueLength());
+}
 try {
-  if (_segmentDownloadSemaphore != null) {
-long startTime = System.currentTimeMillis();
-LOGGER.info("Trying to acquire segment download semaphore for: {}. 
queue-length: {} ", segmentName,
-_segmentDownloadSemaphore.getQueueLength());
-_segmentDownloadSemaphore.acquire();
-LOGGER.info("Acquired segment download semaphore for: {} 
(lock-time={}ms, queue-length={}).", segmentName,
- 

Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577054823


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java:
##
@@ -608,172 +721,153 @@ protected SegmentDataManager unregisterSegment(String 
segmentName) {
 }
   }
 
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata 
zkMetadata) {
-return true;
-  }
-
-  protected File downloadSegment(String segmentName, SegmentZKMetadata 
zkMetadata)
-  throws Exception {
-// TODO: may support download from peer servers for RealTime table.
-return downloadSegmentFromDeepStore(segmentName, zkMetadata);
-  }
-
-  private File downloadSegmentFromDeepStore(String segmentName, 
SegmentZKMetadata zkMetadata)
+  /**
+   * Downloads an immutable segment into the index directory.
+   * Segment can be downloaded from deep store or from peer servers. 
Downloaded segment might be compressed or
+   * encrypted, and this method takes care of decompressing and decrypting the 
segment.
+   */
+  protected File downloadSegment(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
-if (_isStreamSegmentDownloadUntar && zkMetadata.getCrypterName() == null) {
-  try {
-File untaredSegDir = downloadAndStreamUntarWithRateLimit(segmentName, 
zkMetadata, tempRootDir,
-_streamSegmentDownloadUntarRateLimitBytesPerSec);
-return moveSegment(segmentName, untaredSegDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-} else {
-  try {
-File tarFile = downloadAndDecrypt(segmentName, zkMetadata, 
tempRootDir);
-return untarAndMoveSegment(segmentName, tarFile, tempRootDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-}
-  }
-
-  private File moveSegment(String segmentName, File untaredSegDir)
-  throws IOException {
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+Preconditions.checkState(downloadUrl != null,
+"Failed to find download URL in ZK metadata for segment: %s of table: 
%s", segmentName, _tableNameWithType);
 try {
-  File indexDir = getSegmentDataDir(segmentName);
-  FileUtils.deleteDirectory(indexDir);
-  FileUtils.moveDirectory(untaredSegDir, indexDir);
-  return indexDir;
+  if 
(!CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD.equals(downloadUrl)) {
+try {
+  return downloadSegmentFromDeepStore(zkMetadata);
+} catch (Exception e) {
+  if (_peerDownloadScheme != null) {
+return downloadSegmentFromPeers(zkMetadata);
+  } else {
+throw e;
+  }
+}
+  } else {
+return downloadSegmentFromPeers(zkMetadata);
+  }
 } catch (Exception e) {
-  LOGGER.error("Failed to move segment: {} of table: {}", segmentName, 
_tableNameWithType);
-  _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.SEGMENT_DIR_MOVEMENT_FAILURES, 1L);
+  _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.SEGMENT_DOWNLOAD_FAILURES, 1);
   throw e;
 }
   }
 
   @VisibleForTesting
-  File downloadAndDecrypt(String segmentName, SegmentZKMetadata zkMetadata, 
File tempRootDir)
+  File downloadSegmentFromDeepStore(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tarFile = new File(tempRootDir, segmentName + 
TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION);
-String uri = zkMetadata.getDownloadUrl();
-boolean downloadSuccess = false;
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+_logger.info("Downloading segment: {} from: {}", segmentName, downloadUrl);
+File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
+if (_segmentDownloadSemaphore != null) {
+  long startTime = System.currentTimeMillis();
+  _logger.info("Acquiring segment download semaphore for segment: {}, 
queue-length: {} ", segmentName,
+  _segmentDownloadSemaphore.getQueueLength());
+  _segmentDownloadSemaphore.acquire();
+  _logger.info("Acquired segment download semaphore for segment: {} 
(lock-time={}ms, queue-length={}).",
+  segmentName, System.currentTimeMillis() - startTime, 
_segmentDownloadSemaphore.getQueueLength());
+}
 try {
-  if (_segmentDownloadSemaphore != null) {
-long startTime = System.currentTimeMillis();
-LOGGER.info("Trying to acquire segment download semaphore for: {}. 
queue-length: {} ", segmentName,
-_segmentDownloadSemaphore.getQueueLength());
-_segmentDownloadSemaphore.acquire();
-LOGGER.info("Acquired segment download semaphore for: {} 
(lock-time={}ms, queue-length={}).", segmentName,
- 

Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577051333


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java:
##
@@ -608,172 +721,153 @@ protected SegmentDataManager unregisterSegment(String 
segmentName) {
 }
   }
 
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata 
zkMetadata) {
-return true;
-  }
-
-  protected File downloadSegment(String segmentName, SegmentZKMetadata 
zkMetadata)
-  throws Exception {
-// TODO: may support download from peer servers for RealTime table.
-return downloadSegmentFromDeepStore(segmentName, zkMetadata);
-  }
-
-  private File downloadSegmentFromDeepStore(String segmentName, 
SegmentZKMetadata zkMetadata)
+  /**
+   * Downloads an immutable segment into the index directory.
+   * Segment can be downloaded from deep store or from peer servers. 
Downloaded segment might be compressed or
+   * encrypted, and this method takes care of decompressing and decrypting the 
segment.
+   */
+  protected File downloadSegment(SegmentZKMetadata zkMetadata)
   throws Exception {
-File tempRootDir = getTmpSegmentDataDir("tmp-" + segmentName + "-" + 
UUID.randomUUID());
-if (_isStreamSegmentDownloadUntar && zkMetadata.getCrypterName() == null) {
-  try {
-File untaredSegDir = downloadAndStreamUntarWithRateLimit(segmentName, 
zkMetadata, tempRootDir,
-_streamSegmentDownloadUntarRateLimitBytesPerSec);
-return moveSegment(segmentName, untaredSegDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-} else {
-  try {
-File tarFile = downloadAndDecrypt(segmentName, zkMetadata, 
tempRootDir);
-return untarAndMoveSegment(segmentName, tarFile, tempRootDir);
-  } finally {
-FileUtils.deleteQuietly(tempRootDir);
-  }
-}
-  }
-
-  private File moveSegment(String segmentName, File untaredSegDir)
-  throws IOException {
+String segmentName = zkMetadata.getSegmentName();
+String downloadUrl = zkMetadata.getDownloadUrl();
+Preconditions.checkState(downloadUrl != null,
+"Failed to find download URL in ZK metadata for segment: %s of table: 
%s", segmentName, _tableNameWithType);
 try {
-  File indexDir = getSegmentDataDir(segmentName);
-  FileUtils.deleteDirectory(indexDir);
-  FileUtils.moveDirectory(untaredSegDir, indexDir);
-  return indexDir;
+  if 
(!CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD.equals(downloadUrl)) {
+try {
+  return downloadSegmentFromDeepStore(zkMetadata);
+} catch (Exception e) {
+  if (_peerDownloadScheme != null) {

Review Comment:
   Don't follow the second part



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Make Helix state transition handling idempotent [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12886:
URL: https://github.com/apache/pinot/pull/12886#discussion_r1577046142


##
pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java:
##
@@ -428,17 +570,25 @@ public ExecutorService getSegmentPreloadExecutor() {
 
   @Override
   public void addSegmentError(String segmentName, SegmentErrorInfo 
segmentErrorInfo) {
-_errorCache.put(Pair.of(_tableNameWithType, segmentName), 
segmentErrorInfo);
+if (_errorCache != null) {
+  _errorCache.put(Pair.of(_tableNameWithType, segmentName), 
segmentErrorInfo);
+}
   }
 
   @Override
   public Map getSegmentErrors() {
-if (_errorCache == null) {
-  return Collections.emptyMap();
+if (_errorCache != null) {
+  // Filter out entries that match the table name
+  Map segmentErrors = new HashMap<>();
+  for (Map.Entry, SegmentErrorInfo> entry : 
_errorCache.asMap().entrySet()) {
+Pair tableSegmentPair = entry.getKey();
+if (tableSegmentPair.getLeft().equals(_tableNameWithType)) {
+  segmentErrors.put(tableSegmentPair.getRight(), entry.getValue());
+}
+  }
+  return segmentErrors;
 } else {
-  // Filter out entries that match the table name.
-  return _errorCache.asMap().entrySet().stream().filter(map -> 
map.getKey().getLeft().equals(_tableNameWithType))
-  .collect(Collectors.toMap(map -> map.getKey().getRight(), 
Map.Entry::getValue));
+  return Map.of();

Review Comment:
   Since `1.0` release, we have officially dropped the support for java 8, and 
we can use the more concise APIs introduced since java 9, such as `List.of()`, 
`Map.of()` etc.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Add Prefix, Suffix and Ngram UDFs [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang merged PR #12392:
URL: https://github.com/apache/pinot/pull/12392


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot) branch master updated (0caeccfc13 -> 36c4b9a86f)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

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


from 0caeccfc13 Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0 (#12989)
 add 36c4b9a86f Add Prefix, Suffix and Ngram UDFs (#12392)

No new revisions were added by this update.

Summary of changes:
 .../common/function/scalar/StringFunctions.java| 108 +
 .../function/scalar/StringFunctionsTest.java   |  50 ++
 2 files changed, 158 insertions(+)


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



(pinot) branch dependabot/maven/org.webjars-swagger-ui-5.17.0 deleted (was 9ac0fa8507)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie pushed a change to branch dependabot/maven/org.webjars-swagger-ui-5.17.0
in repository https://gitbox.apache.org/repos/asf/pinot.git


 was 9ac0fa8507 Fix swagger resources path

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.


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



(pinot) branch master updated: Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0 (#12989)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie 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 0caeccfc13 Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0 (#12989)
0caeccfc13 is described below

commit 0caeccfc1399087885205e7e796d1ee8037f7867
Author: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
AuthorDate: Tue Apr 23 15:12:07 2024 -0700

Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0 (#12989)
---
 pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java | 2 +-
 pom.xml | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java 
b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
index bbf3b30342..befd5b5763 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
@@ -51,7 +51,7 @@ public class CommonConstants {
   "org.apache.pinot.spi.eventlistener.query.NoOpBrokerQueryEventListener";
 
   public static final String SWAGGER_AUTHORIZATION_KEY = "oauth";
-  public static final String CONFIG_OF_SWAGGER_RESOURCES_PATH = 
"META-INF/resources/webjars/swagger-ui/5.15.0/";
+  public static final String CONFIG_OF_SWAGGER_RESOURCES_PATH = 
"META-INF/resources/webjars/swagger-ui/5.17.0/";
   public static final String CONFIG_OF_TIMEZONE = "pinot.timezone";
 
   public static final String DATABASE = "database";
diff --git a/pom.xml b/pom.xml
index f9d4f7f328..b52922f01e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -147,7 +147,7 @@
 2.6.1
 3.30.2-GA
 1.6.14
-5.15.0
+5.17.0
 3.3.6
 2.9.0
 2.5.1


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



Re: [PR] Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0 [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang merged PR #12989:
URL: https://github.com/apache/pinot/pull/12989


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


abhioncbr commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576806764


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";

Review Comment:
   I think we can use this header version in specific other scenarios, like 
moving away from the usage of `LegacyListDelimiterHandler` in future. 



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [WIP] add locking logic to get consistent table view for upsert tables [pinot]

2024-04-23 Thread via GitHub


klsince commented on code in PR #12976:
URL: https://github.com/apache/pinot/pull/12976#discussion_r1576744981


##
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java:
##
@@ -1103,6 +1210,48 @@ private static MutableRoaringBitmap 
getQueryableDocIdsSnapshotFromSegment(IndexS
 return queryableDocIdsSnapshot;
   }
 
+  private void setSegmentContexts(List segmentContexts) {
+for (SegmentContext segmentContext : segmentContexts) {
+  IndexSegment segment = segmentContext.getIndexSegment();
+  if (_trackedSegments.contains(segment)) {
+
segmentContext.setQueryableDocIdsSnapshot(getQueryableDocIdsSnapshotFromSegment(segment));
+  }
+}
+  }
+
+  private boolean skipUpsertViewRefresh(long upsertViewFreshnessMs) {
+long nowMs = System.currentTimeMillis();
+if (upsertViewFreshnessMs >= 0) {
+  return _lastUpsertViewRefreshTimeMs + upsertViewFreshnessMs > nowMs;
+}
+return _lastUpsertViewRefreshTimeMs + _upsertViewRefreshIntervalMs > nowMs;
+  }
+
+  private void doBatchRefreshUpsertView(long upsertViewFreshnessMs) {

Review Comment:
   Thanks for sharing the code changes, and I read it through. The refreshing 
logic is similar with the batch mode here although done at table mgr level. But 
your changes didn't allow query threads to force update the view to ensure data 
freshness.
   
   I think it's more intuitive to handle data consistency in table partition 
mgr, as table partition is the unit for managing upsert states in upsert table.
   
   Today, the table partition mgr `enriches` the segment contexts, which are 
provided by table partition mgr, so there shouldn't be limitation to extending 
segment contexts later. In fact hiding the consistency logic away from table 
mgr would make it easier to iterate in future.
   ```
   public void setSegmentContexts(List segmentContexts) {
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576704306


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review Comment:
   Maybe keeping a default version:
   ```
   public static final int SEGMENT_METADATA_DEFAULT_VERSION = 1;
   ```
   We probably don't need other constants for versions as version should be int



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576704306


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review Comment:
   Maybe keeping a default version:
   ```
   public static final int DEFAULT_SEGMENT_METADATA_VERSION = 1;
   ```
   We probably don't need other constants for versions as version should be int



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576701431


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";

Review Comment:
   I think this can be common though. If it is explicitly specified (in any 
configuration file), we take it; if it is not explicitly specified, we treat it 
as default version



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576701431


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";

Review Comment:
   I think this can be common though. If it is explicitly specified (in any 
configuration file), we take it; if it is not explicitly specified, we treat it 
as default version (maybe adding a constant for default version?)



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


klsince commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576653550


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review Comment:
   I don't think we need "1" and can use `null` if there is no header from 
segment metadata file. 
   
   How about name the constant below `SEGMENT_METADATA_FILE_FORMAT_VERSION = 
1`, and leave some comments about what the new format would be like.
   ```
   public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_2 = "2";
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


klsince commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576653550


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review Comment:
   I don't think we need "1" and can use `null` if there is no header from 
segment metadata file. 
   
   How about `SEGMENT_METADATA_FILE_FORMAT_VERSION = 1` and use null as 
default. And leave some comments about what the new format would be like.
   ```
   public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_2 = "2";
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


klsince commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576653550


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review Comment:
   I don't think we need "1" and can use `null` if there is no header from 
segment metadata file. 
   
   The var name is not good as it encodes the value in the name. how about 
`SEGMENT_METADATA_FILE_FORMAT_VERSION = 1` and use null as default. And better 
leave some comments about what the new format version would be like.
   ```
   public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_2 = "2";
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


klsince commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576653550


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review Comment:
   I don't think we need "1" and can use `null` if there is no header from 
segment metadata file. 
   
   The var name is not good as it encodes the value in the name. how about 
`SEGMENT_METADATA_VERSION = 1` as default it's null. and do it need to be 
public?
   ```
   public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_2 = "2";
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Custom configuration property reader for segment metadata files [pinot]

2024-04-23 Thread via GitHub


klsince commented on code in PR #12440:
URL: https://github.com/apache/pinot/pull/12440#discussion_r1576650089


##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";

Review Comment:
   SEGMENT_METADATA_VERSION_HEADER_IDENTIFIER = "version" as we only use 
versions to decide the reader/writer for the segment metadata, and leaving all 
other users of this config util intact.



##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -271,20 +319,73 @@ public static String 
recoverSpecialCharacterInPropertyValue(String value) {
 return value.replace("\0\0", ",");
   }
 
-  private static PropertiesConfiguration createPropertiesConfiguration(boolean 
setIOFactory,
-  boolean setDefaultDelimiter) {
+  /**
+   * creates the instance of the {@link 
org.apache.commons.configuration2.PropertiesConfiguration}
+   * with custom IO factory based on kind {@link 
org.apache.commons.configuration2.PropertiesConfiguration.IOFactory}
+   * and legacy list delimiter {@link 
org.apache.commons.configuration2.convert.LegacyListDelimiterHandler}
+   *
+   * @param setDefaultDelimiter sets the default list delimiter.
+   * @param ioFactoryKind IOFactory kind
+   * @return PropertiesConfiguration
+   */
+  private static PropertiesConfiguration createPropertiesConfiguration(boolean 
setDefaultDelimiter,
+  PropertyIOFactoryKind ioFactoryKind) {
 PropertiesConfiguration config = new PropertiesConfiguration();
 
-// setting IO Reader Factory
-if (setIOFactory) {
-  config.setIOFactory(new ConfigFilePropertyReaderFactory());
-}
+// setting IO Reader Factory of the configuration.
+config.setIOFactory(createPropertyIOFactory(ioFactoryKind));
 
-// setting DEFAULT_LIST_DELIMITER
+// setting the DEFAULT_LIST_DELIMITER
 if (setDefaultDelimiter) {
   config.setListDelimiterHandler(new 
LegacyListDelimiterHandler(DEFAULT_LIST_DELIMITER));
 }
 
 return config;
   }
+
+  /**
+   * Creates the IOFactory based on the provided kind.
+   * @param ioFactoryKind IOFactory kind
+   * @return IOFactory
+   */
+  private static IOFactory createPropertyIOFactory(PropertyIOFactoryKind 
ioFactoryKind) {

Review Comment:
   We can define a get() or getInstance() method in the Enum 
`PropertyIOFactoryKind` so that we can remove this helper method 
createPropertyIOFactory(). 



##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -128,6 +156,18 @@ public static PropertiesConfiguration fromFile(File file, 
boolean setIOFactory,
 return config;
   }
 
+  public static void saveSegmentMetadataToFile(PropertiesConfiguration 
propertiesConfiguration, File file,
+  String versionHeader) {
+if (StringUtils.isNotEmpty(versionHeader)) {

Review Comment:
   I think you can do the check as part of the if-check condition.



##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -74,19 +65,19 @@ public static PropertiesConfiguration 
fromInputStream(InputStream stream)
*/
   public static PropertiesConfiguration fromPath(String path)
   throws ConfigurationException {
-return fromPath(path, false, true);
+return fromPath(path, true, 
PropertyIOFactoryKind.DefaultPropertyConfigurationIOFactory);
   }
 
   /**
* Instantiate a {@link PropertiesConfiguration} from an {@link String}.
* @param path representing the path of file
-   * @param setIOFactory representing to set the IOFactory or not
* @param setDefaultDelimiter representing to set the default list delimiter.
* @return a {@link PropertiesConfiguration} instance.
*/
-  public static PropertiesConfiguration fromPath(String path, boolean 
setIOFactory, boolean setDefaultDelimiter)
+  public static PropertiesConfiguration fromPath(String path, boolean 
setDefaultDelimiter,
+  PropertyIOFactoryKind ioFactoryKind)

Review Comment:
   how about pass in `@Nullable IOFactory ioFactory` to those util methods? so 
that we decouple those util methods from the logic of deciding the type and 
initializing the ioFactory object.



##
pinot-spi/src/main/java/org/apache/pinot/spi/env/CommonsConfigurationUtils.java:
##
@@ -43,28 +46,16 @@
  */
 public class CommonsConfigurationUtils {
   private static final Character DEFAULT_LIST_DELIMITER = ',';
+  public static final String VERSION_HEADER_IDENTIFIER = "version";
 
-  private CommonsConfigurationUtils() {
-  }
+  // usage: default header version of all configurations.
+  // if properties configuration doesn't contain header version, it will be 
considered as 1
+  public static final String PROPERTIES_CONFIGURATION_HEADER_VERSION_1 = "1";

Review 

(pinot) branch dependabot/maven/org.webjars-swagger-ui-5.17.0 updated (c2c795c58a -> 9ac0fa8507)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie pushed a change to branch dependabot/maven/org.webjars-swagger-ui-5.17.0
in repository https://gitbox.apache.org/repos/asf/pinot.git


from c2c795c58a Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0
 add 9ac0fa8507 Fix swagger resources path

No new revisions were added by this update.

Summary of changes:
 pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


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



(pinot) branch dependabot/maven/aws.sdk.version-2.25.36 deleted (was 9e21da3528)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie pushed a change to branch dependabot/maven/aws.sdk.version-2.25.36
in repository https://gitbox.apache.org/repos/asf/pinot.git


 was 9e21da3528 Bump aws.sdk.version from 2.25.35 to 2.25.36

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.


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



Re: [PR] Bump aws.sdk.version from 2.25.35 to 2.25.36 [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang merged PR #12990:
URL: https://github.com/apache/pinot/pull/12990


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot) branch master updated: Bump aws.sdk.version from 2.25.35 to 2.25.36 (#12990)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie 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 40cf5a7ba8 Bump aws.sdk.version from 2.25.35 to 2.25.36 (#12990)
40cf5a7ba8 is described below

commit 40cf5a7ba82fcc4fc10d8b8efe800d3f732c0655
Author: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
AuthorDate: Tue Apr 23 10:50:24 2024 -0700

Bump aws.sdk.version from 2.25.35 to 2.25.36 (#12990)
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index 9f6838ec78..f9d4f7f328 100644
--- a/pom.xml
+++ b/pom.xml
@@ -172,7 +172,7 @@
 0.15.0
 0.4.4
 4.2.2
-2.25.35
+2.25.36
 2.12.7
 3.1.12
 7.10.1


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



(pinot) branch dependabot/maven/org.jline-jline-3.26.0 deleted (was cad0a671ea)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie pushed a change to branch dependabot/maven/org.jline-jline-3.26.0
in repository https://gitbox.apache.org/repos/asf/pinot.git


 was cad0a671ea Remove wrong comment

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.


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



Re: [PR] Bump org.jline:jline from 3.24.1 to 3.26.0 [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang merged PR #12991:
URL: https://github.com/apache/pinot/pull/12991


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Bump org.jline:jline from 3.24.1 to 3.26.0 [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12991:
URL: https://github.com/apache/pinot/pull/12991#discussion_r1576650849


##
pom.xml:
##
@@ -244,7 +244,7 @@
 3.9.0
 2.0.3
 

Review Comment:
   ```suggestion
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot) branch dependabot/maven/org.jline-jline-3.26.0 updated (8bb3ae7d0e -> cad0a671ea)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

jackie pushed a change to branch dependabot/maven/org.jline-jline-3.26.0
in repository https://gitbox.apache.org/repos/asf/pinot.git


from 8bb3ae7d0e Bump org.jline:jline from 3.24.1 to 3.26.0
 add cad0a671ea Remove wrong comment

No new revisions were added by this update.

Summary of changes:
 pom.xml | 1 -
 1 file changed, 1 deletion(-)


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



(pinot) branch master updated (8e10320595 -> dd4f0acabd)

2024-04-23 Thread jackie
This is an automated email from the ASF dual-hosted git repository.

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


from 8e10320595 handle absent segments so that catchup checker doesn't get 
stuck on them (#12883)
 add dd4f0acabd Bump org.jline:jline from 3.24.1 to 3.26.0 (#12991)

No new revisions were added by this update.

Summary of changes:
 pom.xml | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)


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



Re: [PR] Re-enable the Spotless plugin for Java 21 [pinot]

2024-04-23 Thread via GitHub


yashmayya commented on code in PR #12992:
URL: https://github.com/apache/pinot/pull/12992#discussion_r1576609447


##
pinot-common/pom.xml:
##
@@ -62,6 +62,27 @@
 protobuf-maven-plugin
   
 
+  
+com.diffplug.spotless
+spotless-maven-plugin
+

Review Comment:
   Yeah, I assumed that https://github.com/apache/pinot/pull/11670 introduced 
the `pluginManagement` in this POM for the JDK 21 build profile (i.e., to avoid 
running this plugin's goal with Java 21)? Also we're overriding the plugin 
configuration here (for the separate excludes I presume), so it's not like we 
can simply define it here without the configuration right?
   
   Edit: We're still leveraging the plugin management from the parent POM since 
we aren't redefining the plugin version or executions here.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Add Prefix, Suffix and Ngram UDFs [pinot]

2024-04-23 Thread via GitHub


deemoliu commented on code in PR #12392:
URL: https://github.com/apache/pinot/pull/12392#discussion_r1576631085


##
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/StringFunctions.java:
##
@@ -581,6 +584,111 @@ public static String[] split(String input, String 
delimiter, int limit) {
 return StringUtils.splitByWholeSeparator(input, delimiter, limit);
   }
 
+  /**
+   * @param input an input string for prefix strings generations.
+   * @param maxlength the max length of the prefix strings for the string.
+   * @return generate an array of prefix strings of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction
+  public static String[] prefixes(String input, int maxlength) {
+int arrLength = Math.min(maxlength, input.length());
+String[] prefixArr = new String[arrLength];
+for (int prefixIdx = 1; prefixIdx <= arrLength; prefixIdx++) {
+  prefixArr[prefixIdx - 1] = input.substring(0, prefixIdx);
+}
+return prefixArr;
+  }
+
+  /**
+   * @param input an input string for prefix strings generations.
+   * @param maxlength the max length of the prefix strings for the string.
+   * @param prefix the prefix to be prepended to prefix strings generated. 
e.g. '^' for regex matching
+   * @return generate an array of prefix matchers of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction(nullableParameters = true, names = {"prefix"})
+  public static String[] prefixesWithPrefix(String input, int maxlength, 
@Nullable String prefix) {
+if (prefix == null) {
+  return prefixes(input, maxlength);
+}
+int arrLength = Math.min(maxlength, input.length());
+String[] prefixArr = new String[arrLength];
+for (int prefixIdx = 1; prefixIdx <= arrLength; prefixIdx++) {
+  prefixArr[prefixIdx - 1] = prefix + input.substring(0, prefixIdx);
+}
+return prefixArr;
+  }
+
+  /**
+   * @param input an input string for suffix strings generations.
+   * @param maxlength the max length of the suffix strings for the string.
+   * @return generate an array of suffix strings of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction
+  public static String[] suffixes(String input, int maxlength) {
+int arrLength = Math.min(maxlength, input.length());
+String[] suffixArr = new String[arrLength];
+for (int suffixIdx = 1; suffixIdx <= arrLength; suffixIdx++) {
+  suffixArr[suffixIdx - 1] = input.substring(input.length() - suffixIdx);
+}
+return suffixArr;
+  }
+
+  /**
+   * @param input an input string for suffix strings generations.
+   * @param maxlength the max length of the suffix strings for the string.
+   * @param suffix the suffix string to be appended for suffix strings 
generated. e.g. '$' for regex matching.
+   * @return generate an array of suffix matchers of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction(nullableParameters = true, names = {"suffix"})

Review Comment:
   updated, thanks @Jackie-Jiang 



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Add Prefix, Suffix and Ngram UDFs [pinot]

2024-04-23 Thread via GitHub


deemoliu commented on code in PR #12392:
URL: https://github.com/apache/pinot/pull/12392#discussion_r1576618558


##
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/StringFunctions.java:
##
@@ -581,6 +584,111 @@ public static String[] split(String input, String 
delimiter, int limit) {
 return StringUtils.splitByWholeSeparator(input, delimiter, limit);
   }
 
+  /**
+   * @param input an input string for prefix strings generations.
+   * @param maxlength the max length of the prefix strings for the string.
+   * @return generate an array of prefix strings of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction
+  public static String[] prefixes(String input, int maxlength) {
+int arrLength = Math.min(maxlength, input.length());
+String[] prefixArr = new String[arrLength];
+for (int prefixIdx = 1; prefixIdx <= arrLength; prefixIdx++) {
+  prefixArr[prefixIdx - 1] = input.substring(0, prefixIdx);
+}
+return prefixArr;
+  }
+
+  /**
+   * @param input an input string for prefix strings generations.
+   * @param maxlength the max length of the prefix strings for the string.
+   * @param prefix the prefix to be prepended to prefix strings generated. 
e.g. '^' for regex matching
+   * @return generate an array of prefix matchers of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction(nullableParameters = true, names = {"prefix"})
+  public static String[] prefixesWithPrefix(String input, int maxlength, 
@Nullable String prefix) {
+if (prefix == null) {
+  return prefixes(input, maxlength);
+}
+int arrLength = Math.min(maxlength, input.length());
+String[] prefixArr = new String[arrLength];
+for (int prefixIdx = 1; prefixIdx <= arrLength; prefixIdx++) {
+  prefixArr[prefixIdx - 1] = prefix + input.substring(0, prefixIdx);
+}
+return prefixArr;
+  }
+
+  /**
+   * @param input an input string for suffix strings generations.
+   * @param maxlength the max length of the suffix strings for the string.
+   * @return generate an array of suffix strings of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction
+  public static String[] suffixes(String input, int maxlength) {
+int arrLength = Math.min(maxlength, input.length());
+String[] suffixArr = new String[arrLength];
+for (int suffixIdx = 1; suffixIdx <= arrLength; suffixIdx++) {
+  suffixArr[suffixIdx - 1] = input.substring(input.length() - suffixIdx);
+}
+return suffixArr;
+  }
+
+  /**
+   * @param input an input string for suffix strings generations.
+   * @param maxlength the max length of the suffix strings for the string.
+   * @param suffix the suffix string to be appended for suffix strings 
generated. e.g. '$' for regex matching.
+   * @return generate an array of suffix matchers of the string that are 
shorter than the specified length.
+   */
+  @ScalarFunction(nullableParameters = true, names = {"suffix"})
+  public static String[] suffixesWithSuffix(String input, int maxlength, 
@Nullable String suffix) {
+if (suffix == null) {
+  return suffixes(input, maxlength);
+}
+int arrLength = Math.min(maxlength, input.length());
+String[] suffixArr = new String[arrLength];
+for (int suffixIdx = 1; suffixIdx <= arrLength; suffixIdx++) {
+  suffixArr[suffixIdx - 1] = input.substring(input.length() - suffixIdx) + 
suffix;
+}
+return suffixArr;
+  }
+
+  /**
+   * @param input an input string for ngram generations.
+   * @param length the max length of the ngram for the string.
+   * @return generate an array of unique ngram of the string that length are 
exactly matching the specified length.
+   */
+  @ScalarFunction
+  public static String[] uniqueNgrams(String input, int length) {
+if (length == 0 || length > input.length()) {
+  return new String[0];
+}
+ObjectSet ngramSet = new ObjectLinkedOpenHashSet<>();
+for (int i = 0; i < input.length() - length + 1; i++) {
+  ngramSet.add(input.substring(i, i + length));
+}
+return ngramSet.toArray(new String[0]);
+  }
+
+  /**
+   * @param input an input string for ngram generations.
+   * @param minGram the min length of the ngram for the string.
+   * @param maxGram the max length of the ngram for the string.
+   * @return generate an array of ngram of the string that length are within 
the specified range [minGram, maxGram].
+   */
+  @ScalarFunction
+  public static String[] uniqueNgrams(String input, int minGram, int maxGram) {
+ObjectSet ngramSet = new ObjectLinkedOpenHashSet<>();

Review Comment:
   hi @Jackie-Jiang ngrams doesn't guarantee to be unique, right? so the usage 
of Set is to dedup and avoid duplicates.



-- 
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.

To unsubscribe, e-mail: 

Re: [PR] Re-enable the Spotless plugin for Java 21 [pinot]

2024-04-23 Thread via GitHub


yashmayya commented on code in PR #12992:
URL: https://github.com/apache/pinot/pull/12992#discussion_r1576609447


##
pinot-common/pom.xml:
##
@@ -62,6 +62,27 @@
 protobuf-maven-plugin
   
 
+  
+com.diffplug.spotless
+spotless-maven-plugin
+

Review Comment:
   Yeah, I assumed that https://github.com/apache/pinot/pull/11670 introduced 
the `pluginManagement` in this POM for the JDK 21 build profile (i.e., to avoid 
running this plugin's goal with Java 21)? Also we're overriding the plugin 
configuration here (for the separate excludes I presume), so it's not like we 
can simply define it here without the configuration right?



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Re-enable the Spotless plugin for Java 21 [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12992:
URL: https://github.com/apache/pinot/pull/12992#discussion_r1576604451


##
pinot-common/pom.xml:
##
@@ -62,6 +62,27 @@
 protobuf-maven-plugin
   
 
+  
+com.diffplug.spotless
+spotless-maven-plugin
+

Review Comment:
   I think this change is fine. Basically he moved this from pluginManagement 
to plugin. Given there is no child project that inherits its pom, I think this 
is the right move.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Re-enable the Spotless plugin for Java 21 [pinot]

2024-04-23 Thread via GitHub


Jackie-Jiang commented on code in PR #12992:
URL: https://github.com/apache/pinot/pull/12992#discussion_r1576601232


##
pinot-common/pom.xml:
##
@@ -62,6 +62,27 @@
 protobuf-maven-plugin
   
 
+  
+com.diffplug.spotless
+spotless-maven-plugin
+

Review Comment:
   We usually keep the plugin management inside the root pom, and then we can 
simply use it here



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot-site) branch new-site-dev updated: WB-284 - Add more (25) company logos to the Powered By page (#121)

2024-04-23 Thread xiangfu
This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch new-site-dev
in repository https://gitbox.apache.org/repos/asf/pinot-site.git


The following commit(s) were added to refs/heads/new-site-dev by this push:
 new 087db244 WB-284 - Add more (25) company logos to the Powered By page 
(#121)
087db244 is described below

commit 087db24465807cb087a23424dfffadf86f1b4775
Author: Gio <153032991+gio-start...@users.noreply.github.com>
AuthorDate: Tue Apr 23 17:54:57 2024 +0200

WB-284 - Add more (25) company logos to the Powered By page (#121)
---
 data/companiesUsingPinot.ts| 125 ++
 public/static/images/companies/cloud/hubspot.svg   |  17 ++
 public/static/images/companies/cloud/momentive.svg |  10 ++
 public/static/images/companies/cloud/phenom.svg|  12 ++
 public/static/images/companies/cloud/zuora.svg |  18 ++
 .../images/companies/fintech/10x_banking.svg   |  20 +++
 .../images/companies/fintech/goldman_sachs.svg |   4 +
 .../static/images/companies/fintech/phone_pe.svg   |  13 ++
 .../static/images/companies/fintech/pine_labs.svg  |  10 ++
 .../static/images/companies/fintech/robinhood.svg  |  24 +++
 public/static/images/companies/food/careem.svg |  10 ++
 public/static/images/companies/food/ola.svg|  13 ++
 public/static/images/companies/media/media_net.svg |  12 ++
 .../static/images/companies/media/promoted_ai.svg  |  23 +++
 .../images/companies/other/boond_manager.svg   |  72 
 public/static/images/companies/other/broadcom.svg  |  22 +++
 public/static/images/companies/other/factual.svg   |  10 ++
 public/static/images/companies/other/hire_ez.svg   |  17 ++
 public/static/images/companies/other/link_labs.svg | 187 +
 public/static/images/companies/other/mobileum.svg  |  10 ++
 .../images/companies/other/palmyra_solutions.svg   |  10 ++
 public/static/images/companies/other/rapid_1.svg   |  10 ++
 public/static/images/companies/other/sim_soft.svg  |  18 ++
 public/static/images/companies/other/vedantu.svg   |  11 ++
 public/static/images/companies/other/you_gov.svg   |  32 
 public/static/images/companies/retail/cora.svg |  13 ++
 26 files changed, 723 insertions(+)

diff --git a/data/companiesUsingPinot.ts b/data/companiesUsingPinot.ts
index 933cfc9f..37a0346a 100644
--- a/data/companiesUsingPinot.ts
+++ b/data/companiesUsingPinot.ts
@@ -43,6 +43,11 @@ const companiesUsingPinot: Company[] = [
 logo: '/static/images/companies/retail/walmart.svg',
 type: 'retail'
 },
+{
+name: 'Cora',
+logo: '/static/images/companies/retail/cora.svg',
+type: 'retail'
+},
 // Media
 {
 name: 'AdBeat',
@@ -119,6 +124,16 @@ const companiesUsingPinot: Company[] = [
 logo: '/static/images/companies/media/roku.svg',
 type: 'media'
 },
+{
+name: 'PromotedAI',
+logo: '/static/images/companies/media/promoted_ai.svg',
+type: 'media'
+},
+{
+name: 'MediaNet',
+logo: '/static/images/companies/media/media_net.svg',
+type: 'media'
+},
 // Fintech
 {
 name: 'Citi',
@@ -155,6 +170,31 @@ const companiesUsingPinot: Company[] = [
 logo: '/static/images/companies/fintech/tradeweb.svg',
 type: 'fintech'
 },
+{
+name: 'Goldman Sachs',
+logo: '/static/images/companies/fintech/goldman_sachs.svg',
+type: 'fintech'
+},
+{
+name: 'PhonePe',
+logo: '/static/images/companies/fintech/phone_pe.svg',
+type: 'fintech'
+},
+{
+name: '10x Banking',
+logo: '/static/images/companies/fintech/10x_banking.svg',
+type: 'fintech'
+},
+{
+name: 'Robinhood',
+logo: '/static/images/companies/fintech/robinhood.svg',
+type: 'fintech'
+},
+{
+name: 'Pine Labs',
+logo: '/static/images/companies/fintech/pine_labs.svg',
+type: 'fintech'
+},
 // Cloud
 {
 name: 'Slack',
@@ -191,6 +231,26 @@ const companiesUsingPinot: Company[] = [
 logo: '/static/images/companies/cloud/hrt.svg',
 type: 'cloud'
 },
+{
+name: 'Momentive',
+logo: '/static/images/companies/cloud/momentive.svg',
+type: 'cloud'
+},
+{
+name: 'Hubspot',
+logo: '/static/images/companies/cloud/hubspot.svg',
+type: 'cloud'
+},
+{
+name: 'Phenom',
+logo: '/static/images/companies/cloud/phenom.svg',
+type: 'cloud'
+},
+{
+name: 'Zuora',
+logo: '/static/images/companies/cloud/zuora.svg',
+type: 'cloud'
+},
 // Food
 {
 name: 'Uber',
@@ -222,6 +282,16 @@ const companiesUsingPinot: Company[] = [
 logo: '/static/images/companies/food/door_dash.svg',
 type: 'food'
 },
+{
+name: 'Careem',
+logo: 

Re: [PR] WB-284 - Add 25 new company logos to the Powered By page [pinot-site]

2024-04-23 Thread via GitHub


xiangfu0 merged PR #121:
URL: https://github.com/apache/pinot-site/pull/121


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Re-enable the Spotless plugin for Java 21 [pinot]

2024-04-23 Thread via GitHub


codecov-commenter commented on PR #12992:
URL: https://github.com/apache/pinot/pull/12992#issuecomment-2072735999

   ## 
[Codecov](https://app.codecov.io/gh/apache/pinot/pull/12992?dropdown=coverage=pr=h1_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 Report
   All modified and coverable lines are covered by tests :white_check_mark:
   > Project coverage is 0.00%. Comparing base 
[(`59551e4`)](https://app.codecov.io/gh/apache/pinot/commit/59551e45224f1535c4863fd577622b37366ccc97?dropdown=coverage=desc_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 to head 
[(`e49f3d1`)](https://app.codecov.io/gh/apache/pinot/pull/12992?dropdown=coverage=pr=desc_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache).
   > Report is 351 commits behind head on master.
   
   
   Additional details and impacted files
   
   
   ```diff
   @@  Coverage Diff  @@
   ## master   #12992   +/-   ##
   =
   - Coverage 61.75%0.00%   -61.76% 
   =
 Files  2436 2427-9 
 Lines133233   132903  -330 
 Branches  2063620587   -49 
   =
   - Hits  822740-82274 
   - Misses44911   132903+87992 
   + Partials   60480 -6048 
   ```
   
   | 
[Flag](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flags_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | Coverage Δ | |
   |---|---|---|
   | 
[custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   | 
[integration](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `0.00% <ø> (-0.01%)` | :arrow_down: |
   | 
[integration1](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   | 
[integration2](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `0.00% <ø> (ø)` | |
   | 
[java-11](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   | 
[java-21](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `0.00% <ø> (-61.63%)` | :arrow_down: |
   | 
[skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `0.00% <ø> (-61.75%)` | :arrow_down: |
   | 
[skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   | 
[temurin](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `0.00% <ø> (-61.76%)` | :arrow_down: |
   | 
[unittests](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   | 
[unittests1](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   | 
[unittests2](https://app.codecov.io/gh/apache/pinot/pull/12992/flags?src=pr=flag_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache)
 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click 
here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache#carryforward-flags-in-the-pull-request-comment)
 to find out more.
   
   
   
   
   [:umbrella: View full report in Codecov by 
Sentry](https://app.codecov.io/gh/apache/pinot/pull/12992?dropdown=coverage=pr=continue_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache).
   
   :loudspeaker: Have feedback on the report? [Share it 
here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral_source=github_content=comment_campaign=pr+comments_term=apache).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, 

[PR] Re-enable the Spotless plugin for Java 21 [pinot]

2024-04-23 Thread via GitHub


yashmayya opened a new pull request, #12992:
URL: https://github.com/apache/pinot/pull/12992

   - The Spotless plugin was disabled for Java 21 in 
https://github.com/apache/pinot/pull/11670 because Spotless didn't support Java 
21 at the time.
   - However, https://github.com/diffplug/spotless/pull/1920 fixed the 
incompatibility and was released in version `2.41.1` of the 
`spotless-maven-plugin`.
   - https://github.com/apache/pinot/pull/12827 bumped up the Spotless plugin 
version from `2.28.0` to `2.43.0` and it can now be re-enabled for Java 21.
   - This has been verified locally by running `mvn spotless:check` and `mvn 
spotless:apply` while using Java 21.


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[PR] WB-284 - Add 25 new company logos to the Powered By page [pinot-site]

2024-04-23 Thread via GitHub


gio-startree opened a new pull request, #121:
URL: https://github.com/apache/pinot-site/pull/121

   This pull request adds 25 new company logos to the Powered By page. The 
logos are in SVG format and have been included in the appropriate files.


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Multi stage stats [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12704:
URL: https://github.com/apache/pinot/pull/12704#discussion_r1576220503


##
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##
@@ -19,44 +19,50 @@
 package org.apache.pinot.query.runtime.blocks;
 
 import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import javax.annotation.Nullable;
 import org.apache.pinot.common.datablock.ColumnarDataBlock;
 import org.apache.pinot.common.datablock.DataBlock;
 import org.apache.pinot.common.datablock.DataBlockUtils;
 import org.apache.pinot.common.datablock.MetadataBlock;
 import org.apache.pinot.common.datablock.RowDataBlock;
-import org.apache.pinot.common.response.ProcessingException;
 import org.apache.pinot.common.utils.DataSchema;
 import org.apache.pinot.core.common.Block;
 import org.apache.pinot.core.common.datablock.DataBlockBuilder;
 import org.apache.pinot.core.util.DataBlockExtractUtils;
-import org.apache.pinot.query.runtime.operator.OperatorStats;
-import org.apache.pinot.query.runtime.operator.utils.OperatorUtils;
-
+import org.apache.pinot.query.runtime.plan.MultiStageQueryStats;
 
 /**
  * A {@code TransferableBlock} is a wrapper around {@link DataBlock} for 
transferring data using
  * {@link org.apache.pinot.common.proto.Mailbox}.
  */
 public class TransferableBlock implements Block {
   private final DataBlock.Type _type;
+  @Nullable
   private final DataSchema _dataSchema;
   private final int _numRows;
 
   private List _container;
   private DataBlock _dataBlock;
   private Map _errCodeToExceptionMap;
+  @Nullable
+  private final MultiStageQueryStats _queryStats;
 
-  public TransferableBlock(List container, DataSchema dataSchema, 
DataBlock.Type type) {
+  public TransferableBlock(List container, @Nullable DataSchema 
dataSchema, DataBlock.Type type) {

Review Comment:
   I don't remember where, but I've found a path in the code this method could 
be called with a null data schema.
   
   I may also be wrong, maybe I've just found that call during my debugs in a 
moment there was an error in my code. I guess I can remove the annotation and 
just add it again in case I find the problematic call.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [I] Explore to use Trie Tree to speed up both json_match and json_extract_index transform function for mutable json indexing segment [pinot]

2024-04-23 Thread via GitHub


mghildiy commented on issue #12645:
URL: https://github.com/apache/pinot/issues/12645#issuecomment-2072241205

   Maybe I can give it a try if noone else is working on it.


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Multi stage stats [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12704:
URL: https://github.com/apache/pinot/pull/12704#discussion_r1576213860


##
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##
@@ -72,11 +81,53 @@ public String getId() {
 return _id;
   }
 
+  /**
+   * Offers a raw block into the mailbox within the timeout specified, returns 
whether the block is successfully added.
+   * If the block is not added, an error block is added to the mailbox.
+   * 
+   * Contrary to {@link #offer(TransferableBlock, long)}, the block may be an
+   * {@link TransferableBlock#isErrorBlock() error block}.
+   */
+  public ReceivingMailboxStatus offerRaw(ByteBuffer byteBuffer, long timeoutMs)
+  throws IOException {
+TransferableBlock block;
+long now = System.currentTimeMillis();
+_stats.merge(StatKey.WAIT_CPU_TIME_MS, now - _lastArriveTime);
+_lastArriveTime = now;
+_stats.merge(StatKey.DESERIALIZED_BYTES, byteBuffer.remaining());
+_stats.merge(StatKey.DESERIALIZED_MESSAGES, 1);
+
+now = System.currentTimeMillis();
+DataBlock dataBlock = DataBlockUtils.getDataBlock(byteBuffer);
+_stats.merge(StatKey.DESERIALIZATION_TIME_MS, System.currentTimeMillis() - 
now);
+
+if (dataBlock instanceof MetadataBlock) {
+  Map exceptions = dataBlock.getExceptions();
+  if (exceptions.isEmpty()) {
+block = TransferableBlockUtils.wrap(dataBlock);
+  } else {
+
setErrorBlock(TransferableBlockUtils.getErrorTransferableBlock(exceptions));
+return ReceivingMailboxStatus.FIRST_ERROR;

Review Comment:
   Older code was treating this case in a bit different way. This method was 
mostly moved from MailboxContentObserver, which in this case the coded was 
doing:
   
   ```
 
_mailbox.setErrorBlock(TransferableBlockUtils.getErrorTransferableBlock(exceptions));
 return;
   ```
   
   The only reason to add this `FIRST_ERROR` enum is to keep the same behavior 
in `MailboxContentObserver`. I would prefer to just return `ERROR` here, but 
that would have the of printing the following log in `MailboxContentObserver`.
   
   ```
 LOGGER.warn("Mailbox: {} already errored out (received error block 
before)", mailboxId);
   ```
   
   I guess we can remove this new enum and change the code in 
`MailboxContentObserver` a little bit.
   
   > Also seems this message doesn't apply to in-memory mailbox, some comments 
would help explain it
   
   `InMemorySendingMailbox` does never call this code because this enum is only 
returned when there is an error deserializing the message. Given messages are 
always heap based in InMemorySendingMailbox, it can never fail in that way.
   
   



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Multi stage stats [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12704:
URL: https://github.com/apache/pinot/pull/12704#discussion_r1576205973


##
pinot-common/src/main/java/org/apache/pinot/common/datablock/V1MetadataBlock.java:
##
@@ -0,0 +1,165 @@
+/**
+ * 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.pinot.common.datablock;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * The datablock used before the introduction of {@link 
org.apache.pinot.common.datatable.StatMap}.
+ * 
+ * This version stored the metadata in a {@code Map} which was 
encoded as JSON and stored in the
+ * variable size data buffer.
+ * 
+ * Instances of this class are not actually seen by the operators. Instead, 
they are converted to {@link MetadataBlock}
+ * in {@link MetadataBlock#deserialize(ByteBuffer, int)}.
+ * 
+ * The reason to keep it here is mostly for backwards compatibility and 
testing. In order to simplify the code, the
+ * stats engine just ignores the metadata of these objects, but we need to be 
able to deserialize them anyway.
+ */
+public class V1MetadataBlock extends BaseDataBlock {

Review Comment:
   This is not only for testing proposes. We use this class to read metadata 
blocks send by servers running previous versions. By doing that we can read the 
actual metadata and do whatever we want with that.
   
   This metadata may be an error or an EOS with stats in the older format. The 
current implementation only cares about the error case. It is important to keep 
using that information (whose format wasn't changed in this PR) in order to 
correctly abort the execution if needed.
   
   Right now we ignore the stats because it doesn't seem worth it to translate 
the stats given the idea is to run with different versions only while the new 
release is being deployed, so losing some stats during that time seems 
acceptable. We can change that if we think it is important, but I would suggest 
to do that in another PR if needed.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Multi stage stats [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12704:
URL: https://github.com/apache/pinot/pull/12704#discussion_r1576200948


##
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##
@@ -276,6 +264,161 @@ protected BrokerResponse handleRequest(long requestId, 
String query, @Nullable S
 return brokerResponse;
   }
 
+  private void fillOldBrokerResponseStats(BrokerResponseNativeV2 
brokerResponse,
+  List queryStats, 
DispatchableSubPlan dispatchableSubPlan) {
+for (int i = 0; i < queryStats.size(); i++) {
+  MultiStageQueryStats.StageStats.Closed stageStats = queryStats.get(i);
+  if (stageStats == null) {
+brokerResponse.addStageStats(JsonUtils.newObjectNode());
+  } else {
+stageStats.forEach((type, stats) -> type.mergeInto(brokerResponse, 
stats));
+
+DispatchablePlanFragment dispatchablePlanFragment = 
dispatchableSubPlan.getQueryStageList().get(i);
+MultiStageStatsTreeBuilder treeBuilder = new 
MultiStageStatsTreeBuilder(stageStats);
+PlanNode fragmentRoot = 
dispatchablePlanFragment.getPlanFragment().getFragmentRoot();
+JsonNode node = fragmentRoot.visit(treeBuilder, null);
+brokerResponse.addStageStats(node);
+  }
+}
+  }
+
+  public static class MultiStageStatsTreeBuilder implements 
PlanNodeVisitor {
+private final MultiStageQueryStats.StageStats.Closed _stageStats;
+private int _index;
+private static final String CHILDREN_KEY = "children";
+
+public MultiStageStatsTreeBuilder(MultiStageQueryStats.StageStats.Closed 
stageStats) {
+  _stageStats = stageStats;
+  _index = stageStats.getLastOperatorIndex();
+}
+
+private ObjectNode selfNode(MultiStageOperator.Type type) {
+  ObjectNode json = JsonUtils.newObjectNode();
+  json.put("type", type.toString());
+  Iterator> statsIt = 
_stageStats.getOperatorStats(_index).asJson().fields();
+  while (statsIt.hasNext()) {
+Map.Entry entry = statsIt.next();
+json.set(entry.getKey(), entry.getValue());
+  }
+  return json;
+}
+
+private JsonNode recursiveCase(AbstractPlanNode node, 
MultiStageOperator.Type expectedType) {
+  MultiStageOperator.Type type = _stageStats.getOperatorType(_index);
+  if (type != expectedType) {
+if (type == MultiStageOperator.Type.LEAF) {
+  // Leaf nodes compile the plan node into a single operator and 
therefore return a single stat
+  return selfNode(type);
+}
+int childrenSize = node.getInputs().size();
+switch (childrenSize) {
+  case 0:
+LOGGER.warn("Skipping unexpected node {} when stat of type {} was 
found at index {}",
+node.getClass(), type, _index);
+return JsonUtils.newObjectNode();
+  case 1:
+LOGGER.warn("Skipping unexpected node {} when stat of type {} was 
found at index {}",
+node.getClass(), type, _index);
+return node.getInputs().get(0).visit(this, null);
+  default:
+throw new IllegalStateException("Expected operator type: " + 
expectedType + ", but got: " + type + " with "

Review Comment:
   I've modified the code to try to return something, even it doesn't make much 
sense.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Multi stage stats [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12704:
URL: https://github.com/apache/pinot/pull/12704#discussion_r1576190502


##
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##
@@ -276,6 +264,161 @@ protected BrokerResponse handleRequest(long requestId, 
String query, @Nullable S
 return brokerResponse;
   }
 
+  private void fillOldBrokerResponseStats(BrokerResponseNativeV2 
brokerResponse,
+  List queryStats, 
DispatchableSubPlan dispatchableSubPlan) {
+for (int i = 0; i < queryStats.size(); i++) {
+  MultiStageQueryStats.StageStats.Closed stageStats = queryStats.get(i);
+  if (stageStats == null) {
+brokerResponse.addStageStats(JsonUtils.newObjectNode());
+  } else {
+stageStats.forEach((type, stats) -> type.mergeInto(brokerResponse, 
stats));
+
+DispatchablePlanFragment dispatchablePlanFragment = 
dispatchableSubPlan.getQueryStageList().get(i);
+MultiStageStatsTreeBuilder treeBuilder = new 
MultiStageStatsTreeBuilder(stageStats);
+PlanNode fragmentRoot = 
dispatchablePlanFragment.getPlanFragment().getFragmentRoot();
+JsonNode node = fragmentRoot.visit(treeBuilder, null);
+brokerResponse.addStageStats(node);
+  }
+}
+  }
+
+  public static class MultiStageStatsTreeBuilder implements 
PlanNodeVisitor {
+private final MultiStageQueryStats.StageStats.Closed _stageStats;
+private int _index;
+private static final String CHILDREN_KEY = "children";
+
+public MultiStageStatsTreeBuilder(MultiStageQueryStats.StageStats.Closed 
stageStats) {
+  _stageStats = stageStats;
+  _index = stageStats.getLastOperatorIndex();
+}
+
+private ObjectNode selfNode(MultiStageOperator.Type type) {
+  ObjectNode json = JsonUtils.newObjectNode();
+  json.put("type", type.toString());
+  Iterator> statsIt = 
_stageStats.getOperatorStats(_index).asJson().fields();
+  while (statsIt.hasNext()) {
+Map.Entry entry = statsIt.next();
+json.set(entry.getKey(), entry.getValue());
+  }
+  return json;
+}
+
+private JsonNode recursiveCase(AbstractPlanNode node, 
MultiStageOperator.Type expectedType) {
+  MultiStageOperator.Type type = _stageStats.getOperatorType(_index);
+  if (type != expectedType) {

Review Comment:
   No, this is very rare. Mostly only happens on the stage 0 and in leaf stages.
   
   In stage 0 always have two nodes. One is a transform IIRC and the other is a 
receive node and we only have stats for the latter.
   
   In leaf stage, we may have several nodes, but the leaf stage compiles them 
into a different plan (as explained above).
   
   In can add some comments



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] Multi stage stats [pinot]

2024-04-23 Thread via GitHub


gortiz commented on code in PR #12704:
URL: https://github.com/apache/pinot/pull/12704#discussion_r1576184899


##
pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNative.java:
##
@@ -51,6 +59,7 @@
 "segmentStatistics", "traceInfo", "partialResult"
 })
 public class BrokerResponseNative implements BrokerResponse {

Review Comment:
   The main reason to modify this file is to have an easier way to merge the 
stats from V2 into V1. Basically BrokerResponseNative exports the V1 metrics 
into a different format. In order to make it easy to populate these stats from 
V2 (mainly from leaf stage) I've modified `BrokerResponseNative` to be store a 
StatMap inside. The difference is very small (mostly change setters and 
getters) but it ends up being closer to an ideal state where all these metrics 
are stored in the same way.
   
   I guess I can remove this change, but then 
`MultiStageOperator.Type.LEAF.mergeInto(BrokerResponseNativeV2 response, , 
StatMap map)`. I can take to follow that approach given it doesn't look you 
like the idea of touching this class.



-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot) branch dependabot/maven/org.jline-jline-3.25.1 deleted (was ebfc61722a)

2024-04-23 Thread github-bot
This is an automated email from the ASF dual-hosted git repository.

github-bot pushed a change to branch dependabot/maven/org.jline-jline-3.25.1
in repository https://gitbox.apache.org/repos/asf/pinot.git


 was ebfc61722a Bump org.jline:jline from 3.24.1 to 3.25.1

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.


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



Re: [PR] Bump org.jline:jline from 3.24.1 to 3.25.1 [pinot]

2024-04-23 Thread via GitHub


dependabot[bot] commented on PR #12875:
URL: https://github.com/apache/pinot/pull/12875#issuecomment-2072071073

   Superseded by #12991.


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot) branch dependabot/maven/aws.sdk.version-2.25.36 created (now 9e21da3528)

2024-04-23 Thread github-bot
This is an automated email from the ASF dual-hosted git repository.

github-bot pushed a change to branch dependabot/maven/aws.sdk.version-2.25.36
in repository https://gitbox.apache.org/repos/asf/pinot.git


  at 9e21da3528 Bump aws.sdk.version from 2.25.35 to 2.25.36

No new revisions were added by this update.


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



[PR] Bump aws.sdk.version from 2.25.35 to 2.25.36 [pinot]

2024-04-23 Thread via GitHub


dependabot[bot] opened a new pull request, #12990:
URL: https://github.com/apache/pinot/pull/12990

   Bumps `aws.sdk.version` from 2.25.35 to 2.25.36.
   Updates `software.amazon.awssdk:bom` from 2.25.35 to 2.25.36
   
   Updates `software.amazon.awssdk:apache-client` from 2.25.35 to 2.25.36
   
   
   Dependabot will resolve any conflicts with this PR as long as you don't 
alter it yourself. You can also trigger a rebase manually by commenting 
`@dependabot rebase`.
   
   [//]: # (dependabot-automerge-start)
   [//]: # (dependabot-automerge-end)
   
   ---
   
   
   Dependabot commands and options
   
   
   You can trigger Dependabot actions by commenting on this PR:
   - `@dependabot rebase` will rebase this PR
   - `@dependabot recreate` will recreate this PR, overwriting any edits that 
have been made to it
   - `@dependabot merge` will merge this PR after your CI passes on it
   - `@dependabot squash and merge` will squash and merge this PR after your CI 
passes on it
   - `@dependabot cancel merge` will cancel a previously requested merge and 
block automerging
   - `@dependabot reopen` will reopen this PR if it is closed
   - `@dependabot close` will close this PR and stop Dependabot recreating it. 
You can achieve the same result by closing it manually
   - `@dependabot show  ignore conditions` will show all of 
the ignore conditions of the specified dependency
   - `@dependabot ignore this major version` will close this PR and stop 
Dependabot creating any more for this major version (unless you reopen the PR 
or upgrade to it yourself)
   - `@dependabot ignore this minor version` will close this PR and stop 
Dependabot creating any more for this minor version (unless you reopen the PR 
or upgrade to it yourself)
   - `@dependabot ignore this dependency` will close this PR and stop 
Dependabot creating any more for this dependency (unless you reopen the PR or 
upgrade to it yourself)
   
   
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



(pinot) branch dependabot/maven/org.jline-jline-3.26.0 created (now 8bb3ae7d0e)

2024-04-23 Thread github-bot
This is an automated email from the ASF dual-hosted git repository.

github-bot pushed a change to branch dependabot/maven/org.jline-jline-3.26.0
in repository https://gitbox.apache.org/repos/asf/pinot.git


  at 8bb3ae7d0e Bump org.jline:jline from 3.24.1 to 3.26.0

No new revisions were added by this update.


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



(pinot) branch dependabot/maven/org.webjars-swagger-ui-5.17.0 created (now c2c795c58a)

2024-04-23 Thread github-bot
This is an automated email from the ASF dual-hosted git repository.

github-bot pushed a change to branch 
dependabot/maven/org.webjars-swagger-ui-5.17.0
in repository https://gitbox.apache.org/repos/asf/pinot.git


  at c2c795c58a Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0

No new revisions were added by this update.


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



Re: [PR] Bump org.jline:jline from 3.24.1 to 3.25.1 [pinot]

2024-04-23 Thread via GitHub


dependabot[bot] closed pull request #12875: Bump org.jline:jline from 3.24.1 to 
3.25.1
URL: https://github.com/apache/pinot/pull/12875


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[PR] Bump org.jline:jline from 3.24.1 to 3.26.0 [pinot]

2024-04-23 Thread via GitHub


dependabot[bot] opened a new pull request, #12991:
URL: https://github.com/apache/pinot/pull/12991

   Bumps [org.jline:jline](https://github.com/jline/jline3) from 3.24.1 to 
3.26.0.
   
   Release notes
   Sourced from https://github.com/jline/jline3/releases;>org.jline:jline's 
releases.
   
   JLine 3.26.0
   What's Changed
   
   Update GitHub Actions by https://github.com/eed3si9n;>@​eed3si9n in https://redirect.github.com/jline/jline3/pull/955;>jline/jline3#955
   In maven-wrapper.properties, sync maven version with pom.xml by https://github.com/dattasid;>@​dattasid in https://redirect.github.com/jline/jline3/pull/961;>jline/jline3#961
   Update maven to 3.9.6 (fixes https://redirect.github.com/jline/jline3/issues/944;>#944) by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/950;>jline/jline3#950
   Fixing the FfmTerminal to run on JDK 22 and on Linux. by https://github.com/lahodaj;>@​lahodaj in https://redirect.github.com/jline/jline3/pull/945;>jline/jline3#945
   Provide jdk8 jar by excluding the FFM support (fixes https://redirect.github.com/jline/jline3/issues/937;>#937) by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/949;>jline/jline3#949
   Exclude directories when locating tty, test 
etc by https://github.com/lrytz;>@​lrytz in https://redirect.github.com/jline/jline3/pull/948;>jline/jline3#948
   Return early from LineReaderImpl.doList if no possibilities or rows 
(fixes https://redirect.github.com/jline/jline3/issues/941;>#941) 
by https://github.com/VerKWer;>@​VerKWer in https://redirect.github.com/jline/jline3/pull/942;>jline/jline3#942
   Fix native image compilation (fixes https://redirect.github.com/jline/jline3/issues/954;>#954) by https://github.com/translatenix;>@​translatenix in https://redirect.github.com/jline/jline3/pull/959;>jline/jline3#959
   Use default sort order when custom sort order is identical by https://github.com/valepakh;>@​valepakh in https://redirect.github.com/jline/jline3/pull/957;>jline/jline3#957
   Disable Quick Edit Mode with mouse support by https://github.com/jvalkeal;>@​jvalkeal in https://redirect.github.com/jline/jline3/pull/965;>jline/jline3#965
   Don't change keybindings to emacs for dumb terminals by https://github.com/hvesalai;>@​hvesalai in https://redirect.github.com/jline/jline3/pull/953;>jline/jline3#953
   Merge ConsoleUI project into JLine (fixes https://redirect.github.com/jline/jline3/issues/841;>#841) by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/935;>jline/jline3#935
   Don't attempt line break at last line by https://github.com/jvalkeal;>@​jvalkeal in https://redirect.github.com/jline/jline3/pull/966;>jline/jline3#966
   Fix remote terminal on linux/bsd (fixes https://redirect.github.com/jline/jline3/issues/962;>#962) by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/968;>jline/jline3#968
   Warn when using a deprecated provider by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/967;>jline/jline3#967
   Improve status bar drawing mechanism (fixes https://redirect.github.com/jline/jline3/issues/939;>#939) by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/969;>jline/jline3#969
   Upgrade maven wrapper by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/972;>jline/jline3#972
   Switch the nativeSignals boolean to true by default by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/971;>jline/jline3#971
   Simplify graal demo by https://github.com/gnodet;>@​gnodet in https://redirect.github.com/jline/jline3/pull/970;>jline/jline3#970
   
   New Contributors
   
   https://github.com/eed3si9n;>@​eed3si9n made 
their first contribution in https://redirect.github.com/jline/jline3/pull/955;>jline/jline3#955
   https://github.com/lahodaj;>@​lahodaj made 
their first contribution in https://redirect.github.com/jline/jline3/pull/945;>jline/jline3#945
   https://github.com/lrytz;>@​lrytz made their 
first contribution in https://redirect.github.com/jline/jline3/pull/948;>jline/jline3#948
   https://github.com/VerKWer;>@​VerKWer made 
their first contribution in https://redirect.github.com/jline/jline3/pull/942;>jline/jline3#942
   https://github.com/translatenix;>@​translatenix made 
their first contribution in https://redirect.github.com/jline/jline3/pull/959;>jline/jline3#959
   https://github.com/valepakh;>@​valepakh made 
their first contribution in https://redirect.github.com/jline/jline3/pull/957;>jline/jline3#957
   https://github.com/jvalkeal;>@​jvalkeal made 
their first contribution in https://redirect.github.com/jline/jline3/pull/965;>jline/jline3#965
   
   Full Changelog: https://github.com/jline/jline3/compare/jline-parent-3.25.1...jline-parent-3.26.0;>https://github.com/jline/jline3/compare/jline-parent-3.25.1...jline-parent-3.26.0
   

[PR] Bump org.webjars:swagger-ui from 5.15.0 to 5.17.0 [pinot]

2024-04-23 Thread via GitHub


dependabot[bot] opened a new pull request, #12989:
URL: https://github.com/apache/pinot/pull/12989

   Bumps [org.webjars:swagger-ui](https://github.com/swagger-api/swagger-ui) 
from 5.15.0 to 5.17.0.
   
   Release notes
   Sourced from https://github.com/swagger-api/swagger-ui/releases;>org.webjars:swagger-ui's
 releases.
   
   Swagger UI v5.17.0 Released!
   https://github.com/swagger-api/swagger-ui/compare/v5.16.2...v5.17.0;>5.17.0
 (2024-04-22)
   Features
   
   config: expose config and make it overridable (https://redirect.github.com/swagger-api/swagger-ui/issues/9862;>#9862)
 (https://github.com/swagger-api/swagger-ui/commit/17d50a6ab56287d5c2f6385af46057631e2a81b0;>17d50a6)
   swagger-ui-react: rewrite into SSR compatible function 
component (https://redirect.github.com/swagger-api/swagger-ui/issues/9855;>#9855)
 (https://github.com/swagger-api/swagger-ui/commit/351191bc5745de5c46f34c33a6ef6c75d7fd5290;>351191b),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9243;>#9243
   
   Swagger UI v5.16.2 Released!
   https://github.com/swagger-api/swagger-ui/compare/v5.16.1...v5.16.2;>5.16.2
 (2024-04-19)
   Bug Fixes
   
   swagger-ui-react: avoid triggering implicit system 
rendering (https://redirect.github.com/swagger-api/swagger-ui/issues/9847;>#9847)
 (https://github.com/swagger-api/swagger-ui/commit/3a671c61d3efe8beddf3ff3ae180bae39c4870c9;>3a671c6),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9846;>#9846
   
   Swagger UI v5.16.1 Released!
   https://github.com/swagger-api/swagger-ui/compare/v5.16.0...v5.16.1;>5.16.1
 (2024-04-18)
   Bug Fixes
   
   config: add options defined during runtime (https://redirect.github.com/swagger-api/swagger-ui/issues/9845;>#9845)
 (https://github.com/swagger-api/swagger-ui/commit/60670d1bbd5cfad367e6ed145b1c871d19e72ffa;>60670d1),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9844;>#9844
   config: address getting configUrl from options (https://redirect.github.com/swagger-api/swagger-ui/issues/9840;>#9840)
 (https://github.com/swagger-api/swagger-ui/commit/dc9650b80bb3f1ff029eebbe3316cc7440d1f3b9;>dc9650b),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9819;>#9819
   config: sync SwaggerUI default options with 
swagger-ui-react (https://redirect.github.com/swagger-api/swagger-ui/issues/9843;>#9843)
 (https://github.com/swagger-api/swagger-ui/commit/bcd7388a2d6416973074ba0a04e3a1a1e4ef7991;>bcd7388),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9838;>#9838
   
   Swagger UI v5.16.0 Released!
   https://github.com/swagger-api/swagger-ui/compare/v5.15.2...v5.16.0;>5.16.0
 (2024-04-18)
   Bug Fixes
   
   config: cast configuration values into proper types (https://redirect.github.com/swagger-api/swagger-ui/issues/9829;>#9829)
 (https://github.com/swagger-api/swagger-ui/commit/7378821669bba8a90ab09abaaa74a8c4e22d0fa8;>7378821),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9808;>#9808
   config: fix precedence for system options (https://redirect.github.com/swagger-api/swagger-ui/issues/9831;>#9831)
 (https://github.com/swagger-api/swagger-ui/commit/2720d76365b8613bd2d1c46cda7ceacbc38d7b46;>2720d76),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9818;>#9818
   config: handle query config options (https://redirect.github.com/swagger-api/swagger-ui/issues/9824;>#9824)
 (https://github.com/swagger-api/swagger-ui/commit/692311105b087971b7b6ba312985db3595a75e83;>6923111),
 closes https://redirect.github.com/swagger-api/swagger-ui/issues/9807;>#9807
   resolve: fix infinite recursions and cycle detection in 
OpenAPI 3.1.0 resolution (https://redirect.github.com/swagger-api/swagger-ui/issues/9837;>#9837)
 (https://github.com/swagger-api/swagger-ui/commit/c452527cb22b8d2d077e8d01bfd2093a155c86c7;>c452527)
   
   Features
   
   expose build info via new versions plugin (https://redirect.github.com/swagger-api/swagger-ui/issues/9820;>#9820)
 (https://github.com/swagger-api/swagger-ui/commit/463cd35d31a1e5deeca5279e4e1b398ae316268b;>463cd35)
   request-snippets: use active class modifier for active 
snippet (https://redirect.github.com/swagger-api/swagger-ui/issues/9826;>#9826)
 (https://github.com/swagger-api/swagger-ui/commit/06e6dfac2146142c4d2e8acc63d169a736f7c66e;>06e6dfa)
   
   Swagger UI v5.15.2 Released!
   https://github.com/swagger-api/swagger-ui/compare/v5.15.1...v5.15.2;>5.15.2
 (2024-04-15)
   Bug Fixes
   
   
   ... (truncated)
   
   
   Commits
   
   https://github.com/swagger-api/swagger-ui/commit/84f66a6f168263e158cab8472cc2cbf2ca4c6242;>84f66a6
 chore(release): cut the v5.17.0 release
   https://github.com/swagger-api/swagger-ui/commit/17d50a6ab56287d5c2f6385af46057631e2a81b0;>17d50a6
 feat(config): expose config and make it overridable (https://redirect.github.com/swagger-api/swagger-ui/issues/9862;>#9862)
   https://github.com/swagger-api/swagger-ui/commit/351191bc5745de5c46f34c33a6ef6c75d7fd5290;>351191b
 

Re: [I] Upsert table backfill enhancement: support externally partitioned data [pinot]

2024-04-23 Thread via GitHub


tibrewalpratik17 commented on issue #12987:
URL: https://github.com/apache/pinot/issues/12987#issuecomment-2071920309

   > Provide partition id externally:
   Option 1: Provide partition id as http headers during segment upload
   Option 2: Provide partition id as part of uploaded segment metadata(not as 
columnPartitionMap) (metadata.properties)
   
   IMO if we go for option-2, then we should be consistent to add this / update 
this metadata for all present segments too. Option-1 is better in that aspect 
as we already pass a lot of info as headers during segment upload and use each 
header as more of a config.


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [I] Vector embeddings support in Pinot [pinot]

2024-04-23 Thread via GitHub


hpvd commented on issue #10919:
URL: https://github.com/apache/pinot/issues/10919#issuecomment-2071881575

   Release video: Apache Pinot 1.1 | Overview of Latest Features and Updates 
https://www.youtube.com/watch?v=wSwPtOajsGY
   talks also about vector index support: 
https://www.youtube.com/watch?v=wSwPtOajsGY=1m20s


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [I] COUNT(DISTINCT col1, col2) does not throw a syntax error and returns an incorrect result [pinot]

2024-04-23 Thread via GitHub


prajyotkcsu commented on issue #11503:
URL: https://github.com/apache/pinot/issues/11503#issuecomment-2071478815

   good observation!


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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