[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300792
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/TextMatchPredicateAstNode.java
 ##
 @@ -0,0 +1,96 @@
+/**
+ * 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.pql.parsers.pql2.ast;
+
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.FilterOperator;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.common.utils.request.FilterQueryTree;
+import org.apache.pinot.common.utils.request.HavingQueryTree;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.pql.parsers.Pql2CompilationException;
+import org.apache.pinot.pql.parsers.Pql2Compiler;
+
+
+public class TextMatchPredicateAstNode extends PredicateAstNode {
+
+  private static final String SEPERATOR = "\t\t";
+
+  @Override
+  public void addChild(AstNode childNode) {
+if (!Pql2Compiler.ENABLE_TEXT_MATCH) {
 
 Review comment:
   TEXT index is currently disabled (temporarily) until support for this is 
added in segment reload path. The feature will be enabled and this check will 
be removed in the immediate follow-up PR when we add changes for segment reload.
   
   A similar check is there in segment generator code path to prevent usage of 
this feature on existing tables 


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300929
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/Pql2AstListener.java
 ##
 @@ -429,4 +430,14 @@ public void enterOptions(PQL2Parser.OptionsContext ctx) {
   public void exitOptions(PQL2Parser.OptionsContext ctx) {
 popNode();
   }
+
+  @Override
+  public void enterTextMatchPredicate(@NotNull 
PQL2Parser.TextMatchPredicateContext ctx) {
 
 Review comment:
   Yes


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300821
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/Pql2Compiler.java
 ##
 @@ -72,7 +72,8 @@
   
Boolean.valueOf(System.getProperty("pinot.query.converter.fail_on_error", 
"false"));
   public static String ENABLE_DISTINCT_KEY = "pinot.distinct.enabled";
   public static boolean ENABLE_DISTINCT = 
Boolean.valueOf(System.getProperty(ENABLE_DISTINCT_KEY, "true"));
-
+  public static String ENABLE_TEXT_MATCH_KEY = "pinot.textmatch.enabled";
 
 Review comment:
   TEXT index is currently disabled (temporarily) until support for this is 
added in segment reload path. The feature will be enabled and this check will 
be removed in the immediate follow-up PR when we add changes for segment reload.
   
   A similar check is there in segment generator code path to prevent usage of 
this feature on existing tables 


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300621
 
 

 ##
 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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300512
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/segment/index/readers/InvertedIndexReader.java
 ##
 @@ -19,12 +19,20 @@
 package org.apache.pinot.core.segment.index.readers;
 
 import java.io.Closeable;
+import org.apache.pinot.core.common.Predicate;
 
 
 public interface InvertedIndexReader extends Closeable {
 
   /**
* Get the document ids for the given dictionary id.
+   * @param dictId dictionary ID
*/
   T getDocIds(int dictId);
+
+  /**
+   * Get the document ids after evaluating the given predicate
+   * @param predicate predicate
+   */
+  T getDocIds(Predicate predicate);
 
 Review comment:
   As discussed offline, this interface might help in extending it in the 
future when we implement native text index and add support for complex objects. 
   
   However, I have changed the API to getDocIds(Object value) instead of 
getDocIds(Predicate predicate). This makes it intuitive to use our inverted 
index readers for both dictionary ID based lookup and raw value (or search 
query) based lookup.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300610
 
 

 ##
 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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300571
 
 

 ##
 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:
   Done. Thanks for catching


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300486
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/segment/index/readers/text/LuceneTextIndexReader.java
 ##
 @@ -0,0 +1,139 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+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.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.pinot.core.common.Predicate;
+import org.apache.pinot.core.common.predicate.TextMatchPredicate;
+import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import 
org.apache.pinot.core.segment.creator.impl.inv.text.LuceneTextIndexCreator;
+import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is used to read/search the Lucene text index.
+ * When {@link 
org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader} loads the 
segment,
+ * it also loads (mmaps) the Lucene text index if the segment has TEXT 
column(s).
+ */
+public class LuceneTextIndexReader implements 
InvertedIndexReader {
+  private static final org.slf4j.Logger LOGGER = 
LoggerFactory.getLogger(LuceneTextIndexReader.class);
+
+  private final IndexReader _indexReader;
+  private final Directory _indexDirectory;
+  private final IndexSearcher _indexSearcher;
+  private final QueryParser _queryParser;
+  private final String _column;
+
+  /**
+   * As part of loading the segment in ImmutableSegmentLoader,
+   * we load the text index (per column if it exists) and store
+   * the reference in {@link 
org.apache.pinot.core.segment.index.column.PhysicalColumnIndexContainer}
+   * similar to how it is done for other types of indexes.
+   * @param column column name
+   * @param segmentIndexDir segment index directory
+   */
+  public LuceneTextIndexReader(String column, File segmentIndexDir) {
+_column = column;
+try {
+  File indexFile = new File(segmentIndexDir.getPath() + "/" + 
SegmentVersion.v3.name() + "/" + column + 
LuceneTextIndexCreator.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+  _indexDirectory = FSDirectory.open(indexFile.toPath());
+  _indexReader = DirectoryReader.open(_indexDirectory);
+  _indexSearcher = new IndexSearcher(_indexReader);
+  // Disable Lucene query result cache. While it helps a lot with 
performance for
+  // repeated queries, on the downside it cause heap issues.
+  _indexSearcher.setQueryCache(null);
+} catch (Exception e) {
+  LOGGER.error("Failed to instantiate Lucene text index reader for column 
{}, exception {}", column, e.getMessage());
+  throw new RuntimeException(e);
+}
+StandardAnalyzer analyzer = new StandardAnalyzer();
+_queryParser = new QueryParser(column, analyzer);
+  }
+
+  @Override
+  public LuceneSearchResult getDocIds(int dictId) {
+// This should not be called from anywhere. If it happens, there is a bug 
in the current implementation
+// and that's why we throw illegal state exception
+throw new IllegalStateException("Using dictionary ID is not supported on 
Lucene inverted index");
+  }
+
+  /**
+   * Called during filter operator execution
+   * by {@link org.apache.pinot.core.operator.filter.TextMatchFilterOperator}
+   * @param predicate text search predicate
+   * @return search results
+   */
+  @Override
+  public LuceneSearchResult getDocIds(Predicate predicate) {
+String searchQuery = ((TextMatchPredicate)predicate).getSearchQuery();
+MutableRoaringBitmap docIDs = new MutableRoaringBitmap();
+Collector docIDCollector = new 

[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300537
 
 

 ##
 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:
   Done


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300647
 
 

 ##
 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:
   Done


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300549
 
 

 ##
 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:
   If it is lazily initialized, the code knows if this object has to be added 
to global concurrent queue or not.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-14 Thread GitBox
siddharthteotia 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_r379300597
 
 

 ##
 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:
   done


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-13 Thread GitBox
siddharthteotia 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_r378815323
 
 

 ##
 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:
   Done


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-13 Thread GitBox
siddharthteotia 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_r378814017
 
 

 ##
 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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-13 Thread GitBox
siddharthteotia 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_r378813550
 
 

 ##
 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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-13 Thread GitBox
siddharthteotia 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_r378812786
 
 

 ##
 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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-13 Thread GitBox
siddharthteotia 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_r378811622
 
 

 ##
 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:
   Done


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-13 Thread GitBox
siddharthteotia 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_r378811575
 
 

 ##
 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:
   Not needed. A single instance is 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.
 
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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-02-12 Thread GitBox
siddharthteotia 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_r378524643
 
 

 ##
 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:
   Done. Please look at RealtimeLuceneIndexRefreshState and 
RealtimeLuceneIndexReaderRefreshThread


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368757789
 
 

 ##
 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:
   I missed removing it. Done 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.
 
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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368758007
 
 

 ##
 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:
   This list need not be concurrent since this is updated exactly once only 
when the MutableSegmentImpl is created. The overall queue of realtime segments 
is concurrent though


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368760225
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
 ##
 @@ -45,10 +55,13 @@
 import org.apache.pinot.core.realtime.impl.dictionary.MutableDictionaryFactory;
 import 
org.apache.pinot.core.realtime.impl.invertedindex.RealtimeInvertedIndexReader;
 import 
org.apache.pinot.core.realtime.impl.nullvalue.RealtimeNullValueVectorReaderWriter;
+import 
org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneTextIndexReader;
 
 Review comment:
   Discussed offline. Right now the special casing (typecasting is happening 
only once during segment destroy). A follow-up PR is needed as per this comment 
https://github.com/apache/incubator-pinot/pull/4993#discussion_r368136714 to 
clean up the RealtimeInvertedIndexReader interface.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368759880
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/dociditerators/LuceneIndexScanDocIdIterator.java
 ##
 @@ -0,0 +1,90 @@
+/**
+ * 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.operator.dociditerators;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.pinot.core.common.Constants;
+import org.apache.pinot.core.segment.creator.impl.V1Constants;
+import 
org.apache.pinot.core.segment.index.readers.text.LuceneIndexSearcherReferenceManager;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+import org.roaringbitmap.IntIterator;
+
+
+public class LuceneIndexScanDocIdIterator implements IndexBasedDocIdIterator {
+
+  private int _currentDocId = -1;
+  private int _startDocId;
+  private int _endDocId;
+  private final IntIterator _docIDIterator;
+  private final IndexSearcher _indexSearcher;
+  private final LuceneIndexSearcherReferenceManager _searcherReferenceManager;
+  private final int _numDocs;
+
+  public LuceneIndexScanDocIdIterator(LuceneTextIndexReader.LuceneSearchResult 
luceneSearchResult) {
+_docIDIterator = luceneSearchResult.getDocIDs().getIntIterator();
+_numDocs = luceneSearchResult.getDocIDs().getCardinality();
+_searcherReferenceManager = 
luceneSearchResult.getIndexSearcherReferenceManager();
+_indexSearcher = _searcherReferenceManager.getIndexSearcher();
+  }
+
+  public void setStartDocId(int startDocId) {
+_startDocId = startDocId;
+  }
+
+  public void setEndDocId(int endDocId) {
+_endDocId = endDocId;
+  }
+
+  @Override
+  public int next() {
+if (_currentDocId == Constants.EOF || !_docIDIterator.hasNext()) {
 
 Review comment:
   No longer needed since this file is removed.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368759932
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
 ##
 @@ -608,8 +671,27 @@ public void destroy() {
 // clear map now that index is closed to prevent accidental usage
 _indexReaderWriterMap.clear();
 
-for (RealtimeInvertedIndexReader index : _invertedIndexMap.values()) {
-  index.close();
+for (InvertedIndexReader index : _invertedIndexMap.values()) {
 
 Review comment:
   Addressed as per this comment -- 
https://github.com/apache/incubator-pinot/pull/4993#discussion_r368136714


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368760010
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
 ##
 @@ -118,6 +131,36 @@
   private volatile long _lastIndexedTimeMs = Long.MIN_VALUE;
   private volatile long _latestIngestionTimeMs = Long.MIN_VALUE;
 
+  private static final ScheduledExecutorService _scheduledExecutorService;
 
 Review comment:
   Addressed as per this comment -- 
https://github.com/apache/incubator-pinot/pull/4993#discussion_r368136714


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368759415
 
 

 ##
 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 don't follow -- it has to be static right? We can't create an instance of 
this per LLRealtimeSegmentDataManager which also has a corresponding instance 
of MutableSegmentImpl.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368758547
 
 

 ##
 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:
   Compiler will catch 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.
 
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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368758339
 
 

 ##
 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:
   Actually this check is not needed. Removed 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.
 
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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368758007
 
 

 ##
 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:
   This list need not be concurrent since this is updated only when the 
MutableSegmentImpl is created. The overall queue of realtime segments is 
concurrent.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368758087
 
 

 ##
 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 have added a TODO to handle the case of segment reload when text column is 
added to consuming 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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368757857
 
 

 ##
 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:
   Done


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368757847
 
 

 ##
 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:
   Yes, the follow-up will address 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.
 
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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-20 Thread GitBox
siddharthteotia 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_r368757789
 
 

 ##
 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:
   I missed removing 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.
 
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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368181968
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/TextMatchPredicateAstNode.java
 ##
 @@ -0,0 +1,82 @@
+/**
+ * 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.pql.parsers.pql2.ast;
+
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.FilterOperator;
+import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.common.utils.request.FilterQueryTree;
+import org.apache.pinot.common.utils.request.HavingQueryTree;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.pql.parsers.Pql2CompilationException;
+
+
+public class TextMatchPredicateAstNode extends PredicateAstNode {
+
+  private static final String SEPERATOR = "\t\t";
+
+  @Override
+  public void addChild(AstNode childNode) {
+if (childNode instanceof IdentifierAstNode) {
+  if (_identifier == null) {
+IdentifierAstNode node = (IdentifierAstNode) childNode;
+_identifier = node.getName();
+  } else {
+throw new Pql2CompilationException("TEXT_MATCH predicate has more than 
one identifier.");
+  }
+} else if (childNode instanceof FunctionCallAstNode) {
+  throw new Pql2CompilationException("TEXT_MATCH is not supported with 
function");
+} else {
+  super.addChild(childNode);
+}
+  }
+
+  @Override
+  public FilterQueryTree buildFilterQueryTree() {
+if (_identifier == null) {
+  throw new Pql2CompilationException("TEXT_MATCH predicate has no 
identifier");
+}
+
+List children = getChildren();
+Preconditions.checkState(children != null && children.size() == 1);
+AstNode child = children.get(0);
+Preconditions.checkState(child instanceof StringLiteralAstNode);
+String expr = ((StringLiteralAstNode)child).getValueAsString();
+FilterOperator filterOperator = FilterOperator.TEXT_MATCH;
+List value = Collections.singletonList(expr);
+return new FilterQueryTree(_identifier, value, filterOperator, null);
+  }
+
+  @Override
+  public Expression buildFilterExpression() {
 
 Review comment:
   When we build PinotQuery by going down the PQL compiler path:
   
   (1) Build AST.
   (2) Build BrokerRequest by doing root.updateBrokerRequest().
   (3) Build PinotQuery by doing astNode.updatePinotQuery() implemented by our 
AST nodes. Internally the WhereAstNode calls the buildFilterExpression() method 
implemented by each type of PredicateAstNode. FilterExpression is needed by 
PinotQuery.
   
   I have implemented this function.
   


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368180500
 
 

 ##
 File path: pinot-common/src/main/antlr4/org/apache/pinot/pql/parsers/PQL2.g4
 ##
 @@ -95,6 +96,9 @@ betweenClause:
 regexpLikeClause:
   REGEXP_LIKE '(' expression ',' literal ')';
 
+textMatchClause:
 
 Review comment:
   We need to extend the Calcite grammar and parser to make its compiler 
recognize TEXT_MATCH as a valid token. This way a SQL query on Pinot that has 
text_match() predicate(s) and goes down the Calcite compiler path to build 
PinotQuery will work perfectly


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368141436
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/dociditerators/LuceneIndexScanDocIdIterator.java
 ##
 @@ -0,0 +1,90 @@
+/**
+ * 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.operator.dociditerators;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.pinot.core.common.Constants;
+import org.apache.pinot.core.segment.creator.impl.V1Constants;
+import 
org.apache.pinot.core.segment.index.readers.text.LuceneIndexSearcherReferenceManager;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+import org.roaringbitmap.IntIterator;
+
+
+public class LuceneIndexScanDocIdIterator implements IndexBasedDocIdIterator {
+
+  private int _currentDocId = -1;
+  private int _startDocId;
+  private int _endDocId;
+  private final IntIterator _docIDIterator;
+  private final IndexSearcher _indexSearcher;
+  private final LuceneIndexSearcherReferenceManager _searcherReferenceManager;
+  private final int _numDocs;
+
+  public LuceneIndexScanDocIdIterator(LuceneTextIndexReader.LuceneSearchResult 
luceneSearchResult) {
+_docIDIterator = luceneSearchResult.getDocIDs().getIntIterator();
 
 Review comment:
   As a follow-up we can optimize this path to see if a construction of new 
bitmap is needed or not


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r367800344
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/dociditerators/LuceneIndexScanDocIdIterator.java
 ##
 @@ -0,0 +1,90 @@
+/**
+ * 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.operator.dociditerators;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.pinot.core.common.Constants;
+import org.apache.pinot.core.segment.creator.impl.V1Constants;
+import 
org.apache.pinot.core.segment.index.readers.text.LuceneIndexSearcherReferenceManager;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+import org.roaringbitmap.IntIterator;
+
+
+public class LuceneIndexScanDocIdIterator implements IndexBasedDocIdIterator {
+
+  private int _currentDocId = -1;
+  private int _startDocId;
+  private int _endDocId;
+  private final IntIterator _docIDIterator;
+  private final IndexSearcher _indexSearcher;
+  private final LuceneIndexSearcherReferenceManager _searcherReferenceManager;
+  private final int _numDocs;
+
+  public LuceneIndexScanDocIdIterator(LuceneTextIndexReader.LuceneSearchResult 
luceneSearchResult) {
+_docIDIterator = luceneSearchResult.getDocIDs().getIntIterator();
 
 Review comment:
   When Lucene returns search result... we get the docIDs collected by the 
collector in MutableRoaringBitmap.
   
   These are essentially Lucene internal docIDs (during index building, lucene 
creates separate index structures (it calls them segments) and they are merged 
as per its internal merge criteria so we can't rely on its internal docIds). In 
some cases, they can be same as Pinot docIDs but based on the index size, how 
many index segments it creates and how it merges etc can vary so relying on 
them permanently is very likely to break our code and they also advise against 
it.
   
   So the way to get the actual Pinot docIDs is to store them with each Lucene 
document
   
   Once the IndexSearcher finishes searching each document and all lucene 
docIDs are collected by the collector in the bitmap, we can do 
searcher.doc(lucene doc id), this will get the document and we can retrieve the 
actual docID from the document.
   
   Now why can't we do searcher.doc(lucene doc id) as part of collector? It 
suffers severe performance -- (something there documentation also says not to 
do this while the collector is being invoked).
   
   So that's why this bitmap with lucene docids is then iterated by 
LuceneIndexDocIdIterator and for each lucene doc ID, we get the actual docID 
and this is what is returned by the iterator for rest of the filter processing 
to continue. Once we have iterated completely, we also release the searcher via 
reference manager (NO-OP for offline search).
   
   Now if we were to use BitMapDocIdSet -- which directly works against the 
bitmap of docIDs, we have two options
   
   (1) Change the implementation of BitmapDocIdSet to take LuceneSearchResult 
in the constructor and do all of this there and in BitMapDocIdIterator
   OR
   (2) Once the call to textIndexReader.getDocIds(predicate) comes back into 
TextMatchFilterOperator, we iterate over the lucene docIDs there itself and get 
the Pinot docIDs (as described above) and collect them in a new bitmap. Pass 
this bitmap over to BitmapDocIdSet. Everything will work correctly but then we 
have to construct another bitmap which will impact performance and memory


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368135253
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/dociditerators/LuceneIndexScanDocIdIterator.java
 ##
 @@ -0,0 +1,90 @@
+/**
+ * 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.operator.dociditerators;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.pinot.core.common.Constants;
+import org.apache.pinot.core.segment.creator.impl.V1Constants;
+import 
org.apache.pinot.core.segment.index.readers.text.LuceneIndexSearcherReferenceManager;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+import org.roaringbitmap.IntIterator;
+
+
+public class LuceneIndexScanDocIdIterator implements IndexBasedDocIdIterator {
+
+  private int _currentDocId = -1;
+  private int _startDocId;
+  private int _endDocId;
+  private final IntIterator _docIDIterator;
+  private final IndexSearcher _indexSearcher;
+  private final LuceneIndexSearcherReferenceManager _searcherReferenceManager;
+  private final int _numDocs;
+
+  public LuceneIndexScanDocIdIterator(LuceneTextIndexReader.LuceneSearchResult 
luceneSearchResult) {
+_docIDIterator = luceneSearchResult.getDocIDs().getIntIterator();
 
 Review comment:
   Discussed offline. It is better to iterate over the lucene doc id set 
upfront and build the bitmap with actual pinot doc ids and then have rest of 
the filter processing work off that bimap since then we can do fast 
intersection for multi-predicate query which is either leveraging native Pinot 
inverted index or another text_match predicate.


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368135838
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/docidsets/LuceneIndexDocIdSet.java
 ##
 @@ -0,0 +1,84 @@
+/**
+ * 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.operator.docidsets;
+
+import org.apache.pinot.core.common.BlockDocIdIterator;
+import 
org.apache.pinot.core.operator.dociditerators.LuceneIndexScanDocIdIterator;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+
+
+public class LuceneIndexDocIdSet implements FilterBlockDocIdSet {
 
 Review comment:
   Removed this class after latest discussion


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368135722
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/TextMatchFilterOperator.java
 ##
 @@ -0,0 +1,67 @@
+/**
+ * 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.operator.filter;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.common.Predicate;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.LuceneIndexDocIdSet;
+import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluator;
+import 
org.apache.pinot.core.operator.filter.predicate.TextMatchPredicateEvaluatorFactory;
+import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+
+
+/**
+ * Filter operator for supporting the execution of text search
+ * queries: WHERE TEXT_MATCH(column_name, query_string)
+ */
+public class TextMatchFilterOperator extends BaseFilterOperator {
 
 Review comment:
   Done -- the operator stays the same (TextMatchFilterOperator) but it now 
outputs a BitMapDocIdSet


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368135253
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/dociditerators/LuceneIndexScanDocIdIterator.java
 ##
 @@ -0,0 +1,90 @@
+/**
+ * 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.operator.dociditerators;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.pinot.core.common.Constants;
+import org.apache.pinot.core.segment.creator.impl.V1Constants;
+import 
org.apache.pinot.core.segment.index.readers.text.LuceneIndexSearcherReferenceManager;
+import org.apache.pinot.core.segment.index.readers.text.LuceneTextIndexReader;
+import org.roaringbitmap.IntIterator;
+
+
+public class LuceneIndexScanDocIdIterator implements IndexBasedDocIdIterator {
+
+  private int _currentDocId = -1;
+  private int _startDocId;
+  private int _endDocId;
+  private final IntIterator _docIDIterator;
+  private final IndexSearcher _indexSearcher;
+  private final LuceneIndexSearcherReferenceManager _searcherReferenceManager;
+  private final int _numDocs;
+
+  public LuceneIndexScanDocIdIterator(LuceneTextIndexReader.LuceneSearchResult 
luceneSearchResult) {
+_docIDIterator = luceneSearchResult.getDocIDs().getIntIterator();
 
 Review comment:
   Discussed offline. It is better to iterate over the lucene doc id set 
upfront and build the bitmap with actual pinot doc ids and then have rest of 
the filter processing work off that bimap


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] siddharthteotia commented on a change in pull request #4993: Support Text column type in Pinot (both offline and realtime)

2020-01-17 Thread GitBox
siddharthteotia 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_r368134995
 
 

 ##
 File path: pinot-core/pom.xml
 ##
 @@ -198,5 +198,20 @@
   test-jar
   test
 
+
+  org.apache.lucene
+  lucene-core
+  8.2.0
 
 Review comment:
   Done


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