[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379157224 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; Review comment: should also signal condition variable so that we get shutdown signal 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379144319 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +124,8 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private RealtimeLuceneReadersForRealtimeSegment _realtimeLuceneReadersForRealtimeSegment; Review comment: Also, does it make sense to make this a final and create it right here? It is only memory, and we can add it to the refresh thead if we find that are there any columns that have lucene indexing. You can skip some if checks in the code below 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379143199 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +124,8 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private RealtimeLuceneReadersForRealtimeSegment _realtimeLuceneReadersForRealtimeSegment; Review comment: You can choose to do this in another PR, but does it make sense to call the class MutableLuceneReaders? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379148300 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; + } + + @Override + public void run() { +while (_stopped) { + _mutex.lock(); + while (_luceneRealtimeReaders.isEmpty()) { +try { + // During instantiation of MutableSegmentImpl, we will signal on this condition variable once Review comment: as many times as there are mutable segments 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379155888 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; + } + + @Override + public void run() { +while (_stopped) { + _mutex.lock(); + while (_luceneRealtimeReaders.isEmpty()) { +try { + // During instantiation of MutableSegmentImpl, we will signal on this condition variable once + // one or more realtime lucene readers (one per column) belonging to the MutableSegment + // are added to the global queue managed by this thread. The thread that signals will + // grab this mutex and signal on the condition variable. + // + // This refresh thread will be woken up (and grab the mutex automatically as per the + // implementation of await) and check if the queue is non-empty. It will then proceed to + // poll the queue and refresh the realtime index readers for the polled segment. + // + // The mutex and condition-variable semantics take care of the scenario when on + // a given Pinot server, there is no realtime segment with text index enabled. In such + // cases, there is no need for this thread to wake up simply after every few seconds/minutes + // only to find that there is nothing to be refreshed. The thread should simply be + // off CPU until signalled specifically. This also covers the situation where initially + // there were few realtime segments of a table with text index. Later if they got + //
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379151451 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; + } + + @Override + public void run() { +while (_stopped) { + _mutex.lock(); + while (_luceneRealtimeReaders.isEmpty()) { +try { + // During instantiation of MutableSegmentImpl, we will signal on this condition variable once + // one or more realtime lucene readers (one per column) belonging to the MutableSegment + // are added to the global queue managed by this thread. The thread that signals will + // grab this mutex and signal on the condition variable. + // + // This refresh thread will be woken up (and grab the mutex automatically as per the + // implementation of await) and check if the queue is non-empty. It will then proceed to + // poll the queue and refresh the realtime index readers for the polled segment. + // + // The mutex and condition-variable semantics take care of the scenario when on + // a given Pinot server, there is no realtime segment with text index enabled. In such + // cases, there is no need for this thread to wake up simply after every few seconds/minutes + // only to find that there is nothing to be refreshed. The thread should simply be + // off CPU until signalled specifically. This also covers the situation where initially + // there were few realtime segments of a table with text index. Later if they got + //
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379148012 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; + } + + @Override + public void run() { +while (_stopped) { Review comment: ```suggestion while (!_stopped) { ``` 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379150641 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; + } + + @Override + public void run() { +while (_stopped) { + _mutex.lock(); + while (_luceneRealtimeReaders.isEmpty()) { +try { + // During instantiation of MutableSegmentImpl, we will signal on this condition variable once + // one or more realtime lucene readers (one per column) belonging to the MutableSegment + // are added to the global queue managed by this thread. The thread that signals will + // grab this mutex and signal on the condition variable. + // + // This refresh thread will be woken up (and grab the mutex automatically as per the + // implementation of await) and check if the queue is non-empty. It will then proceed to + // poll the queue and refresh the realtime index readers for the polled segment. + // + // The mutex and condition-variable semantics take care of the scenario when on + // a given Pinot server, there is no realtime segment with text index enabled. In such + // cases, there is no need for this thread to wake up simply after every few seconds/minutes + // only to find that there is nothing to be refreshed. The thread should simply be + // off CPU until signalled specifically. This also covers the situation where initially + // there were few realtime segments of a table with text index. Later if they got + //
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r379157705 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,133 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulates a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + private final Lock _mutex; + private final Condition _conditionVariable; + + private volatile boolean _stopped = false; + + RealtimeLuceneIndexReaderRefreshThread( + ConcurrentLinkedQueue luceneRealtimeReaders, + Lock mutex, Condition conditionVariable) { +_luceneRealtimeReaders = luceneRealtimeReaders; +_mutex = mutex; +_conditionVariable = conditionVariable; + } + + void setStopped() { +_stopped = true; + } + + @Override + public void run() { +while (_stopped) { + _mutex.lock(); + while (_luceneRealtimeReaders.isEmpty()) { +try { + // During instantiation of MutableSegmentImpl, we will signal on this condition variable once + // one or more realtime lucene readers (one per column) belonging to the MutableSegment + // are added to the global queue managed by this thread. The thread that signals will + // grab this mutex and signal on the condition variable. + // + // This refresh thread will be woken up (and grab the mutex automatically as per the + // implementation of await) and check if the queue is non-empty. It will then proceed to + // poll the queue and refresh the realtime index readers for the polled segment. + // + // The mutex and condition-variable semantics take care of the scenario when on + // a given Pinot server, there is no realtime segment with text index enabled. In such + // cases, there is no need for this thread to wake up simply after every few seconds/minutes + // only to find that there is nothing to be refreshed. The thread should simply be + // off CPU until signalled specifically. This also covers the situation where initially + // there were few realtime segments of a table with text index. Later if they got + //
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378576556 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexRefreshState.java ## @@ -0,0 +1,98 @@ +package org.apache.pinot.core.realtime.impl.invertedindex; + +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + + +/** + * This class manages the realtime lucene index readers. Creates a global + * queue with all the realtime segment lucene index readers across + * all tables and manages their refresh using {@link RealtimeLuceneIndexReaderRefreshThread} + */ +public class RealtimeLuceneIndexRefreshState { + private static RealtimeLuceneIndexRefreshState _singletonInstance; + private static ScheduledExecutorService _scheduledExecutorService; + private static ConcurrentLinkedQueue _luceneRealtimeReaders; + + private RealtimeLuceneIndexRefreshState() { +// This constructor is called by getInstance() exactly once. Since +// getInstance() is invoked by MutableSegmentImpl only if it encounters +// a TEXT index column, we ensure that background refresh thread is not created +// if there is no column with TEXT index enabled +_scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); +// TODO: eventually we should explore partitioning this queue on per table basis +_luceneRealtimeReaders = new ConcurrentLinkedQueue<>(); +_scheduledExecutorService.scheduleWithFixedDelay(new RealtimeLuceneIndexReaderRefreshThread(_luceneRealtimeReaders), +RealtimeLuceneIndexReaderRefreshThread.INITIAL_DELAY_MS_DEFAULT, RealtimeLuceneIndexReaderRefreshThread.DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT, TimeUnit.MILLISECONDS); + } + + public static RealtimeLuceneIndexRefreshState getInstance() { +if (_singletonInstance == null) { + synchronized (RealtimeLuceneIndexRefreshState.class) { +if (_singletonInstance == null) { + _singletonInstance = new RealtimeLuceneIndexRefreshState(); +} + } +} +return _singletonInstance; + } + + public void addRealtimeReadersToQueue(RealtimeLuceneReadersForRealtimeSegment readersForRealtimeSegment) { +_luceneRealtimeReaders.offer(readersForRealtimeSegment); + } + + /** + * Since the text index is maintained per TEXT column (similar to other Pinot indexes), Review comment: please change the comment, we dont have TEXT columns anymore. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378578210 ## File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/readers/text/LuceneIndexSearcherReferenceManager.java ## @@ -0,0 +1,57 @@ +/** + * 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.core.segment.index.readers.text; + +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.SearcherManager; + + +/** + * A reference manager to help release the Lucene text index reader/searcher. + * This is used only in the context of realtime search with Lucene. To avoid + * introducing a release() API on the index reader interface, this wrapper is + * created over SearcherManager and returned to the caller. The caller can + * then appropriately call the release once the searcher associated with + * SearcherManager is no longer needed. + */ +public class LuceneIndexSearcherReferenceManager { + + private final SearcherManager _searcherManager; + private final IndexSearcher _indexSearcher; + + public LuceneIndexSearcherReferenceManager(SearcherManager searcherManager, IndexSearcher indexSearcher) { +_searcherManager = searcherManager; +_indexSearcher = indexSearcher; + } + + public void release() { +try { + // Only used for realtime search with Lucene + if (_searcherManager != null) { +_searcherManager.release(_indexSearcher); + } +} catch (Exception e) { + throw new RuntimeException("Error: failed while releasing a previously acquired searcher. " + e); Review comment: Please remove the word "Error". You can log an error log. Also, e.getMessage()? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378577762 ## File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/inv/text/LuceneTextIndexCreator.java ## @@ -0,0 +1,149 @@ +/** + * 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.core.segment.creator.impl.inv.text; + +import java.io.File; +import java.io.IOException; +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.apache.pinot.core.segment.creator.InvertedIndexCreator; + + +/** + * This is used to create Lucene based text index. + * Used for both offline from {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * and realtime from {@link org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneTextIndexReader} + */ +public class LuceneTextIndexCreator implements InvertedIndexCreator { + // TODO: make buffer size configurable choosing a default value based on the heap usage results in design doc + private static final int LUCENE_INDEX_MAX_BUFFER_SIZE_MB = 500; + public static final String LUCENE_INDEX_DOC_ID_COLUMN_NAME = "DocID"; + public static final String LUCENE_TEXT_INDEX_FILE_EXTENSION = ".lucene.index"; + + private final String _textColumn; + private final Directory _indexDirectory; + private final IndexWriter _indexWriter; + + /** + * Called by {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * when building an offline segment. Similar to how it creates per column + * dictionary, forward and inverted index, a text index is also created + * if text search is enabled on a column. + * @param column column name + * @param segmentIndexDir segment index directory + * @param commit true if the index should be committed (at the end after all documents have + * been added), false if index should not be committed + * Note on commit: + * Once {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * finishes indexing all documents/rows for the segment, we need to commit and close + * the Lucene index which will internally persist the index on disk, do the necessary + * resource cleanup etc. We commit during {@link InvertedIndexCreator#seal()} + * and close during {@link InvertedIndexCreator#close()}. + * This lucene index writer is used by both offline and realtime (both during + * indexing in-memory MutableSegment and later during conversion to offline). + * Since realtime segment conversion is again going to go through the offline + * indexing path and will do everything (indexing, commit, close etc), there is + * no need to commit the index from the realtime side. So when the realtime segment + * is destroyed (which is after the realtime segment has been committed and converted + * to offline), we close this lucene index writer to release resources but don't commit. + * This is the reason to have commit flag part of the constructor. + */ + public LuceneTextIndexCreator(String column, File segmentIndexDir, boolean commit) { +_textColumn = column; +try { + File indexFile = new File(segmentIndexDir.getPath() + "/" + _textColumn + LUCENE_TEXT_INDEX_FILE_EXTENSION); + _indexDirectory = FSDirectory.open(indexFile.toPath()); + StandardAnalyzer standardAnalyzer = new StandardAnalyzer(); + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(standardAnalyzer); +
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378575897 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java ## @@ -0,0 +1,105 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.lucene.search.SearcherManager; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState.RealtimeLuceneReadersForRealtimeSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type {@link RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulate a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshThread implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + public static final int INITIAL_DELAY_MS_DEFAULT = 1000; + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; + + private final ConcurrentLinkedQueue _luceneRealtimeReaders; + + public RealtimeLuceneIndexReaderRefreshThread(ConcurrentLinkedQueue luceneRealtimeReaders) { +_luceneRealtimeReaders = luceneRealtimeReaders; + } + + @Override + public void run() { +RealtimeLuceneReadersForRealtimeSegment realtimeReadersForSegment = _luceneRealtimeReaders.poll(); +if (realtimeReadersForSegment != null) { + String segmentName = realtimeReadersForSegment.getSegmentName(); + if (!realtimeReadersForSegment.isSegmentAboutToBeDestroyed()) { +// If segmentAboutToBeDestroyed is false, it implies MutableSegmentImpl +// code hasn't yet started to destroy the segment. So the refresh task +// can proceed for now. +boolean refreshed = true; +if (realtimeReadersForSegment.getLock().tryLock()) { + // got the lock + try { +List realtimeLuceneReaders = realtimeReadersForSegment.getRealtimeLuceneReaders(); +for (RealtimeLuceneTextIndexReader realtimeReader : realtimeLuceneReaders) { + if (realtimeReadersForSegment.isSegmentAboutToBeDestroyed()) { +// Check this here to know if MutableSegmentImpl wants to start destroying the +// realtime segment but is unable to proceed since we hold the lock. So we will stop +// in the middle, abort the refresh and let MutableSegmentImpl proceed with destroying +// the segment. +refreshed = false; +break; + } + SearcherManager searcherManager = realtimeReader.getSearcherManager(); + try { +searcherManager.maybeRefresh(); + } catch (Exception e) { +LOGGER.warn("Caught exception {} while refreshing realtime lucene reader for segment: {}", e, segmentName); Review comment: One too many `{}`. Also, e should be the last argument
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378577699 ## File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/inv/text/LuceneTextIndexCreator.java ## @@ -0,0 +1,149 @@ +/** + * 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.core.segment.creator.impl.inv.text; + +import java.io.File; +import java.io.IOException; +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.apache.pinot.core.segment.creator.InvertedIndexCreator; + + +/** + * This is used to create Lucene based text index. + * Used for both offline from {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * and realtime from {@link org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneTextIndexReader} + */ +public class LuceneTextIndexCreator implements InvertedIndexCreator { + // TODO: make buffer size configurable choosing a default value based on the heap usage results in design doc + private static final int LUCENE_INDEX_MAX_BUFFER_SIZE_MB = 500; + public static final String LUCENE_INDEX_DOC_ID_COLUMN_NAME = "DocID"; + public static final String LUCENE_TEXT_INDEX_FILE_EXTENSION = ".lucene.index"; + + private final String _textColumn; + private final Directory _indexDirectory; + private final IndexWriter _indexWriter; + + /** + * Called by {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * when building an offline segment. Similar to how it creates per column + * dictionary, forward and inverted index, a text index is also created + * if text search is enabled on a column. + * @param column column name + * @param segmentIndexDir segment index directory + * @param commit true if the index should be committed (at the end after all documents have + * been added), false if index should not be committed + * Note on commit: + * Once {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * finishes indexing all documents/rows for the segment, we need to commit and close + * the Lucene index which will internally persist the index on disk, do the necessary + * resource cleanup etc. We commit during {@link InvertedIndexCreator#seal()} + * and close during {@link InvertedIndexCreator#close()}. + * This lucene index writer is used by both offline and realtime (both during + * indexing in-memory MutableSegment and later during conversion to offline). + * Since realtime segment conversion is again going to go through the offline + * indexing path and will do everything (indexing, commit, close etc), there is + * no need to commit the index from the realtime side. So when the realtime segment + * is destroyed (which is after the realtime segment has been committed and converted + * to offline), we close this lucene index writer to release resources but don't commit. + * This is the reason to have commit flag part of the constructor. + */ + public LuceneTextIndexCreator(String column, File segmentIndexDir, boolean commit) { +_textColumn = column; +try { + File indexFile = new File(segmentIndexDir.getPath() + "/" + _textColumn + LUCENE_TEXT_INDEX_FILE_EXTENSION); + _indexDirectory = FSDirectory.open(indexFile.toPath()); + StandardAnalyzer standardAnalyzer = new StandardAnalyzer(); + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(standardAnalyzer); +
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378577159 ## File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/inv/text/LuceneTextIndexCreator.java ## @@ -0,0 +1,149 @@ +/** + * 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.core.segment.creator.impl.inv.text; + +import java.io.File; +import java.io.IOException; +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.apache.pinot.core.segment.creator.InvertedIndexCreator; + + +/** + * This is used to create Lucene based text index. + * Used for both offline from {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * and realtime from {@link org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneTextIndexReader} + */ +public class LuceneTextIndexCreator implements InvertedIndexCreator { + // TODO: make buffer size configurable choosing a default value based on the heap usage results in design doc + private static final int LUCENE_INDEX_MAX_BUFFER_SIZE_MB = 500; + public static final String LUCENE_INDEX_DOC_ID_COLUMN_NAME = "DocID"; + public static final String LUCENE_TEXT_INDEX_FILE_EXTENSION = ".lucene.index"; + + private final String _textColumn; + private final Directory _indexDirectory; + private final IndexWriter _indexWriter; + + /** + * Called by {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * when building an offline segment. Similar to how it creates per column + * dictionary, forward and inverted index, a text index is also created + * if text search is enabled on a column. + * @param column column name + * @param segmentIndexDir segment index directory + * @param commit true if the index should be committed (at the end after all documents have + * been added), false if index should not be committed + * Note on commit: + * Once {@link org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator} + * finishes indexing all documents/rows for the segment, we need to commit and close + * the Lucene index which will internally persist the index on disk, do the necessary + * resource cleanup etc. We commit during {@link InvertedIndexCreator#seal()} + * and close during {@link InvertedIndexCreator#close()}. + * This lucene index writer is used by both offline and realtime (both during + * indexing in-memory MutableSegment and later during conversion to offline). + * Since realtime segment conversion is again going to go through the offline + * indexing path and will do everything (indexing, commit, close etc), there is + * no need to commit the index from the realtime side. So when the realtime segment + * is destroyed (which is after the realtime segment has been committed and converted + * to offline), we close this lucene index writer to release resources but don't commit. + * This is the reason to have commit flag part of the constructor. + */ + public LuceneTextIndexCreator(String column, File segmentIndexDir, boolean commit) { +_textColumn = column; +try { + File indexFile = new File(segmentIndexDir.getPath() + "/" + _textColumn + LUCENE_TEXT_INDEX_FILE_EXTENSION); + _indexDirectory = FSDirectory.open(indexFile.toPath()); + StandardAnalyzer standardAnalyzer = new StandardAnalyzer(); + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(standardAnalyzer); +
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r378529761 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +124,8 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private final Map _segmentToRealtimeLuceneReadersMap; Review comment: Why do we need a map here? There is one MutableSegmentImpl per segment, so it looks like we just need one instance of RealtimeLuceneReadersForRealtimeSegment. Or, is this a reader per column (in which case the key should be column name instead of segment name) 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r369169551 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +131,54 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private static final ScheduledExecutorService _scheduledExecutorService; + private static final ConcurrentLinkedQueue _luceneRealtimeReaders; + private static final ConcurrentHashMap _segmentToRealtimeLuceneReadersMap; + + static { Review comment: As discussed, it needs to be either an instance per table or a singleton instance across all tables. Best to create a class and add functionality to it. We can wire it in later depending on what else we decide (e.g. per table config to control the knob of consistency vs overhead) 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r369153368 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +131,54 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private static final ScheduledExecutorService _scheduledExecutorService; + private static final ConcurrentLinkedQueue _luceneRealtimeReaders; + private static final ConcurrentHashMap _segmentToRealtimeLuceneReadersMap; + + static { Review comment: I misspoke. It should be in TableDataManager. To be specific, RealtimeTableDataManager. thanks 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r368334405 ## File path: pinot-core/src/main/java/org/apache/pinot/core/common/predicate/TextMatchPredicate.java ## @@ -0,0 +1,45 @@ +/** + * 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.core.common.predicate; + +import com.google.common.base.Preconditions; +import java.util.Arrays; +import java.util.List; +import org.apache.pinot.core.common.Predicate; + + +public class TextMatchPredicate extends Predicate { + String _searchQuery; + + public TextMatchPredicate(String lhs, List rhs) { +super(lhs, Predicate.Type.TEXT_MATCH, rhs); +Preconditions.checkArgument(rhs.size() == 1); Review comment: Is there a compilation check that this is validated? If not, then throwing a runtime error on a query that is invalid in the first place seems to be wrong. We will end up returning 5xx instead of 4xx on the query, 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r368336179 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneTextIndexReader.java ## @@ -0,0 +1,117 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.io.File; +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.queryparser.classic.QueryParser; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.SearcherManager; +import org.apache.pinot.core.common.Predicate; +import org.apache.pinot.core.common.predicate.TextMatchPredicate; +import org.apache.pinot.core.segment.creator.impl.inv.text.LuceneTextIndexCreator; +import org.apache.pinot.core.segment.index.readers.InvertedIndexReader; +import org.apache.pinot.core.segment.index.readers.text.LuceneDocIdCollector; +import org.apache.pinot.core.segment.index.readers.text.LuceneIndexSearcherReferenceManager; +import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader; +import org.roaringbitmap.buffer.MutableRoaringBitmap; + + +/** + * Lucene text index reader supporting near realtime search. An instance of this + * is created per consuming segment by {@link org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl}. + * Internally it uses {@link LuceneTextIndexCreator} for adding documents to the lucene index + * as and when they are indexed by the consuming segment. + */ +public class RealtimeLuceneTextIndexReader implements InvertedIndexReader { + private final QueryParser _queryParser; + private final LuceneTextIndexCreator _indexCreator; + private volatile SearcherManager _searcherManager; Review comment: why is this volatile? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r368335582 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshTask.java ## @@ -0,0 +1,106 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.lucene.search.SearcherManager; +import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type + * {@link org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl.RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulate a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshTask implements Runnable { Review comment: Suggest rename to `RealtimeLuceneIndexReaderRefreshThread` since "task" has a different meaning in pinot (see minion) 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r368334759 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +131,54 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private static final ScheduledExecutorService _scheduledExecutorService; + private static final ConcurrentLinkedQueue _luceneRealtimeReaders; + private static final ConcurrentHashMap _segmentToRealtimeLuceneReadersMap; + + static { Review comment: This code should go into the segment data manager instead of a static 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r368335707 ## File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshTask.java ## @@ -0,0 +1,106 @@ +/** + * 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.core.realtime.impl.invertedindex; + +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.lucene.search.SearcherManager; +import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl; +import org.apache.pinot.core.segment.creator.impl.SegmentColumnarIndexCreator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Background thread to refresh the realtime lucene index readers for supporting + * near-realtime text search. The task maintains a queue of realtime segments. + * This queue is global (across all realtime segments of all realtime/hybrid tables). + * + * Each element in the queue is of type + * {@link org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl.RealtimeLuceneReadersForRealtimeSegment}. + * It encapsulate a lock and all the realtime lucene readers for the particular realtime segment. + * Since text index is also create on a per column basis, there will be as many realtime lucene + * readers as the number of columns with text search enabled. + * + * Between each successive execution of the task, there is a fixed delay (regardless of how long + * each execution took). When the task wakes up, it pick the RealtimeLuceneReadersForRealtimeSegment + * from the head of queue, refresh it's readers and adds this at the tail of queue. + */ +public class RealtimeLuceneIndexReaderRefreshTask implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentColumnarIndexCreator.class); + public static final int INITIAL_DELAY_MS_DEFAULT = 1000; + // TODO: make this configurable and choose a higher default value + public static final int DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT = 10; Review comment: default should not be this value. In fact, the default should be to disable the task. Ideally, the thread should be scheduled only when there is at least one text column added 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org
[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)
mcvsubbu commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime) URL: https://github.com/apache/incubator-pinot/pull/4993#discussion_r368335026 ## File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java ## @@ -118,6 +131,54 @@ private volatile long _lastIndexedTimeMs = Long.MIN_VALUE; private volatile long _latestIngestionTimeMs = Long.MIN_VALUE; + private static final ScheduledExecutorService _scheduledExecutorService; + private static final ConcurrentLinkedQueue _luceneRealtimeReaders; + private static final ConcurrentHashMap _segmentToRealtimeLuceneReadersMap; + + static { +_scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); +_luceneRealtimeReaders = new ConcurrentLinkedQueue<>(); +_segmentToRealtimeLuceneReadersMap = new ConcurrentHashMap<>(); +_scheduledExecutorService.scheduleWithFixedDelay(new RealtimeLuceneIndexReaderRefreshTask(_luceneRealtimeReaders), +RealtimeLuceneIndexReaderRefreshTask.INITIAL_DELAY_MS_DEFAULT, RealtimeLuceneIndexReaderRefreshTask.DELAY_BETWEEN_SUCCESSIVE_EXECUTION_MS_DEFAULT, TimeUnit.MILLISECONDS); + } + + /** + * Since the text index is maintained per TEXT column (similar to other Pinot indexes), + * there could be multiple lucene indexes for a given segment and therefore there can be + * multiple realtime lucene readers (one for each index/column) for the particular + * realtime segment. + */ + public static class RealtimeLuceneReadersForRealtimeSegment { +private final String segmentName; +private final Lock lock; +private volatile boolean segmentAboutToBeDestroyed; +private final List realtimeLuceneReaders; Review comment: I guess we need to handle the case when a text column is added to a consuming segment and a reload is issued. See PR 4954 in the brew. Lists are not thread-safe, but it depends on how we handle this. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org