You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/12/11 05:42:30 UTC

[GitHub] [incubator-pinot] cbalci opened a new pull request #6346: Introduce DimensionTableDataManager

cbalci opened a new pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346


   ## Description
   Adding `DimensionTableDataManager` for managing data access for 'Dimension Tables'. It will be used by the upcoming `LookupTransformUDF` as outlined in [Lookup UDF Join In Pinot](https://docs.google.com/document/d/1InWmxbRqwcqIakzvoEWHLxtX4XR9H5L01256EbAUHV8/edit) document.
   
   This is a followup to the PR: [Adding offline dimension table creation and segment assignment](https://github.com/apache/incubator-pinot/pull/6286/files) and will be followed by a third PR to create the `LookupTransformFunction` soon. If you'd like to see a full end-to-end proof of concept implementation please take a look at [here](https://github.com/cbalci/incubator-pinot/pull/1).
   
   `DimensionTableDataManager` is implemented as an extension of `OfflineTableDataManager` since Dimension tables are modeled as Offline tables with a couple additional features. `DimensionTableDataManager` has a private constructor and its Instances are 'per-table' singletons which are created/accessed via static methods: `createInstanceByTableName` and `getInstanceByTableName`. This is to make it possible for the UDF functions to be able to access Dimension tables, without having to change the `TransformFunction` interface, which would have been very intrusive.
   
   In this implementation, `DimensionTableDataManager` simply loads the contents of a Dimension table into a HashMap on `addSegment` hook. Entries are keyed by table's PrimaryKey, and are available for querying via method `lookupRowByPrimaryKey`.
   
   A couple of known shortcomings:
   - Size of the table is not bounded: This will be addressed in a separate work as quota config enforcement at table creation flow.
   - Schema changes require server restart: This can be addressed in the next iteration of the feature, once basic feature is finalized.
   
   ## Testing
   
   - Unit tests are added
   - A `JoinQuickStart` is added for local manual testing.
   - A manual end-to-end test is done in the [POC implementation](https://github.com/cbalci/incubator-pinot/pull/1) by loading >1MM items on a trivial dim table.
   
   Please take a look
   
   ## Documentation
   * Will be added with the next PR, 'LookupTransformUDF'.
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544633888



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);

Review comment:
       Good suggestion. How about creating the try..finally block inside the for loop like:
   ```java
   for (SegmentDataManager segmentManager: segmentManagers) {
     try {
        //... read contents...
     } finally {
       releaseSegment(segmentManager);
     }
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544617107



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);

Review comment:
       We might want to put release in the finally block so that it is always released even with exceptions:
   ```suggestion
     try {
       for (SegmentDataManager segmentManager : segmentManagers) {
         ...
       }
     } finally {
       for (SegmentDataManager segmentManager : segmentManagers) {
         releaseSegment(segmentManager);
       }
     }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545495850



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);
+      }
+    } finally {
+      _lookupTableWriteLock.unlock();
+    }
+  }
+
+  public GenericRow lookupRowByPrimaryKey(PrimaryKey pk) {
+    _lookupTableReadLock.lock();
+    try {
+      return _lookupTable.get(pk);
+    } finally {
+      _lookupTableReadLock.unlock();
+    }
+  }
+
+  public FieldSpec getColumnFieldSpec(String columnName) {

Review comment:
       Hi Subbu! This is actually one of the two public methods which, the proposed LookupUDF will use. So it is not visible for testing only. To see how the UDF will interact, please take a look at the POC implementation [here](https://github.com/cbalci/incubator-pinot/pull/1/files#diff-b3dc29283ab6ee4ae023e05d276284c754d5e0261df038e70a770f1899456d6bR115).




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545508052



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);
+      }
+    } finally {
+      _lookupTableWriteLock.unlock();
+    }
+  }
+
+  public GenericRow lookupRowByPrimaryKey(PrimaryKey pk) {
+    _lookupTableReadLock.lock();
+    try {
+      return _lookupTable.get(pk);
+    } finally {
+      _lookupTableReadLock.unlock();
+    }
+  }
+
+  public FieldSpec getColumnFieldSpec(String columnName) {

Review comment:
       OK




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545494409



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.

Review comment:
       This is going to be in-heap, using the `_lookupTable` HashMap declared below.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543792128



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,163 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /*
+   * `createInstanceByTableName` should only be used by the `TableDataManagerProvider` and the returned instance

Review comment:
       Updated, thanks for the tip!




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546023953



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Review comment:
       Yeah, guess you're right, it looks a bit too defensive now. Removing.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545504817



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());

Review comment:
       Added

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());

Review comment:
       Added




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770340



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {

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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-748406879


   Great, I don't have any more suggestions. Feel free to merge.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (bab1342) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **increase** coverage by `7.67%`.
   > The diff coverage is `68.27%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   + Coverage   66.44%   74.12%   +7.67%     
   ==========================================
     Files        1075     1288     +213     
     Lines       54773    62141    +7368     
     Branches     8168     9014     +846     
   ==========================================
   + Hits        36396    46060    +9664     
   + Misses      15700    13104    -2596     
   - Partials     2677     2977     +300     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `45.50% <51.88%> (?)` | |
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `44.44% <0.00%> (-4.40%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `78.57% <ø> (+5.40%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../pinot/common/function/DateTimePatternHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRGF0ZVRpbWVQYXR0ZXJuSGFuZGxlci5qYXZh) | `83.33% <ø> (ø)` | |
   | [...ot/common/function/FunctionDefinitionRegistry.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25EZWZpbml0aW9uUmVnaXN0cnkuamF2YQ==) | `88.88% <ø> (+44.44%)` | :arrow_up: |
   | [...org/apache/pinot/common/function/FunctionInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbmZvLmphdmE=) | `100.00% <ø> (ø)` | |
   | ... and [1075 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...bab1342](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543778830



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,163 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /*
+   * `createInstanceByTableName` should only be used by the `TableDataManagerProvider` and the returned instance

Review comment:
       Use `@link` to reference classes and methods. The nice thing is that IDE can help you track it and refactor it.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,163 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /*
+   * `createInstanceByTableName` should only be used by the `TableDataManagerProvider` and the returned instance
+   * should be properly initialized via `TableDataManager::init` method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    DimensionTableDataManager instance = new DimensionTableDataManager();
+    _instances.put(tableNameWithType, instance);
+    return instance;
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+

Review comment:
       extra line

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
##########
@@ -53,6 +53,11 @@ public static TableDataManager getTableDataManager(@Nonnull TableDataManagerConf
     TableDataManager tableDataManager;
     switch (TableType.valueOf(tableDataManagerConfig.getTableDataManagerType())) {
       case OFFLINE:
+        if (tableDataManagerConfig.isDimTable()) {
+          tableDataManager =
+              DimensionTableDataManager.createInstanceByTableName(tableDataManagerConfig.getTableName());
+          break;
+        }

Review comment:
       nit: use `else` to be clearer




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544033062



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);

Review comment:
       Hmm, this is actually what I wanted to do with `putIfAbsent`. `computeIfAbsent` seems more succinct anyways, updating accordingly. 👍 
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (b47465c) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `21.39%`.
   > The diff coverage is `38.69%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #6346       +/-   ##
   ===========================================
   - Coverage   66.44%   45.05%   -21.40%     
   ===========================================
     Files        1075     1295      +220     
     Lines       54773    62582     +7809     
     Branches     8168     9102      +934     
   ===========================================
   - Hits        36396    28198     -8198     
   - Misses      15700    32052    +16352     
   + Partials     2677     2332      -345     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `45.05% <38.69%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...org/apache/pinot/broker/queryquota/HitCounter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9IaXRDb3VudGVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/broker/queryquota/MaxHitRateTracker.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9NYXhIaXRSYXRlVHJhY2tlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ache/pinot/broker/queryquota/QueryQuotaEntity.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9RdWVyeVF1b3RhRW50aXR5LmphdmE=) | `0.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ceselector/StrictReplicaGroupInstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL1N0cmljdFJlcGxpY2FHcm91cEluc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...roker/routing/segmentpruner/TimeSegmentPruner.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL1RpbWVTZWdtZW50UHJ1bmVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...roker/routing/segmentpruner/interval/Interval.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL2ludGVydmFsL0ludGVydmFsLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...r/routing/segmentpruner/interval/IntervalTree.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL2ludGVydmFsL0ludGVydmFsVHJlZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [1295 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [003442a...f719a84](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543796794



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);

Review comment:
       Updated as suggested, however now this method is technically non thread-safe. It doesn't look like this method will ever be called from multiple threads, but let me know if you think this is something we should address.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io commented on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (bab1342) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.09%`.
   > The diff coverage is `45.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.35%   -1.10%     
   ==========================================
     Files        1075     1288     +213     
     Lines       54773    62141    +7368     
     Branches     8168     9014     +846     
   ==========================================
   + Hits        36396    40614    +4218     
   - Misses      15700    18632    +2932     
   - Partials     2677     2895     +218     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1135 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...bab1342](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770718



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());

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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-748403508


   Thanks for the reviews @Jackie-Jiang @yupeng9 @mcvsubbu @chenboat , please let me know if you have any other recommendations. Otherwise I'd like to move forward with landing this and prepare the next PR in for the feature: Transform UDF.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (71c70ee) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.02%`.
   > The diff coverage is `44.92%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.42%   -1.03%     
   ==========================================
     Files        1075     1295     +220     
     Lines       54773    62572    +7799     
     Branches     8168     9097     +929     
   ==========================================
   + Hits        36396    40937    +4541     
   - Misses      15700    18702    +3002     
   - Partials     2677     2933     +256     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.42% <44.92%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1151 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...22b1549](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546177020



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/SegmentDataManager.java
##########
@@ -29,7 +29,7 @@
   private int _referenceCount = 1;
 
   @VisibleForTesting
-  synchronized int getReferenceCount() {
+  public synchronized int getReferenceCount() {

Review comment:
       Yes, we are using for testing only, to assert that the DimensionTable correctly releases segments after loading.
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (56e0a15) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **increase** coverage by `7.67%`.
   > The diff coverage is `68.27%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   + Coverage   66.44%   74.12%   +7.67%     
   ==========================================
     Files        1075     1292     +217     
     Lines       54773    62217    +7444     
     Branches     8168     9021     +853     
   ==========================================
   + Hits        36396    46118    +9722     
   + Misses      15700    13122    -2578     
   - Partials     2677     2977     +300     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `45.48% <51.88%> (?)` | |
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `44.44% <0.00%> (-4.40%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `78.57% <ø> (+5.40%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../pinot/common/function/DateTimePatternHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRGF0ZVRpbWVQYXR0ZXJuSGFuZGxlci5qYXZh) | `83.33% <ø> (ø)` | |
   | [...ot/common/function/FunctionDefinitionRegistry.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25EZWZpbml0aW9uUmVnaXN0cnkuamF2YQ==) | `88.88% <ø> (+44.44%)` | :arrow_up: |
   | [...org/apache/pinot/common/function/FunctionInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbmZvLmphdmE=) | `100.00% <ø> (ø)` | |
   | ... and [1080 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...c8fd8bf](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543798682



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,163 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    DimensionTableDataManager instance = new DimensionTableDataManager();
+    _instances.put(tableNameWithType, instance);
+    return instance;
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {

Review comment:
       (Critical) release these segment data managers after reading them




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546013112



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Review comment:
       Yeah, I think you're right, it looks a bit too defensive now. Removing.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546176081



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully added segment {} and loaded lookup table: {}", indexDir.getName(), getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment {} and reloaded lookup table: {}", segmentName, getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String
+          .format("Error reloading lookup table after segment remove ({}) for table: {}", segmentName, getTableName()),
+          e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      try {
+        for (SegmentDataManager segmentManager : segmentManagers) {

Review comment:
       OK yes, we can address it later.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543771591



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {

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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770167



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {

Review comment:
       Applied your suggestion, however I think the older way was safer in the case where loading of the segment is interrupted with an exception in segment reader. Now, if an exception happens, older content is lost. Let me know if you think this is some edge case worth thinking about.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543771269



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        _lookupTable.clear();
+        return;
+      }
+
+      List<File> indexDirs = new ArrayList<>();
+      for (SegmentDataManager segmentManager: segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        System.out.println(indexSegment.getSegmentName());
+        indexDirs.add(indexSegment.getSegmentMetadata().getIndexDir());
+      }
+      MultiplePinotSegmentRecordReader reader = new MultiplePinotSegmentRecordReader(indexDirs);

Review comment:
       Good suggestion, ended up cleaning the code a bit, thanks.

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManagerTest.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import java.io.File;
+import java.net.URL;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixManager;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentCreationDriverFactory;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.core.segment.index.loader.LoaderTest;
+import org.apache.pinot.segments.v1.creator.SegmentTestUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeSuite;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class DimensionTableDataManagerTest {
+  private static final String TABLE_NAME = "dimBaseballTeams";
+  private static final File INDEX_DIR = new File(LoaderTest.class.getName());
+  private static final String AVRO_DATA_PATH = "data/dimBaseballTeams.avro";
+
+  private File _indexDir;
+  private IndexLoadingConfig _indexLoadingConfig;
+
+  @BeforeSuite
+  public void setUp()
+      throws Exception {
+    // prepare segment data
+    URL resourceUrl = getClass().getClassLoader().getResource(AVRO_DATA_PATH);
+    Assert.assertNotNull(resourceUrl);
+    File avroFile = new File(resourceUrl.getFile());
+
+    // create segment
+    SegmentGeneratorConfig segmentGeneratorConfig =
+        SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, INDEX_DIR, TABLE_NAME);
+    segmentGeneratorConfig.setSegmentVersion(SegmentVersion.v3);
+    SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+    _indexDir = new File(INDEX_DIR, driver.getSegmentName());
+
+    _indexLoadingConfig = new IndexLoadingConfig();
+    _indexLoadingConfig.setReadMode(ReadMode.mmap);
+    _indexLoadingConfig.setSegmentVersion(SegmentVersion.v3);
+  }
+
+  @AfterSuite

Review comment:
       Done

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManagerTest.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import java.io.File;
+import java.net.URL;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixManager;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentCreationDriverFactory;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.core.segment.index.loader.LoaderTest;
+import org.apache.pinot.segments.v1.creator.SegmentTestUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeSuite;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class DimensionTableDataManagerTest {
+  private static final String TABLE_NAME = "dimBaseballTeams";
+  private static final File INDEX_DIR = new File(LoaderTest.class.getName());
+  private static final String AVRO_DATA_PATH = "data/dimBaseballTeams.avro";
+
+  private File _indexDir;
+  private IndexLoadingConfig _indexLoadingConfig;
+
+  @BeforeSuite

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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770437



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());

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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546013112



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Review comment:
       Yeah, I think you're right, it looks a bit too defensive now. Removing.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770578



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (56e0a15) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.09%`.
   > The diff coverage is `45.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.35%   -1.10%     
   ==========================================
     Files        1075     1292     +217     
     Lines       54773    62217    +7444     
     Branches     8168     9021     +853     
   ==========================================
   + Hits        36396    40661    +4265     
   - Misses      15700    18654    +2954     
   - Partials     2677     2902     +225     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1140 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...c8fd8bf](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544034995



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {

Review comment:
       Agreed that the previous approach doesn't save a lot either. Leaving as is, thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544021074



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,163 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    DimensionTableDataManager instance = new DimensionTableDataManager();
+    _instances.put(tableNameWithType, instance);
+    return instance;
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {

Review comment:
       Good catch, 👍. Updated to release the segment after usage. Also added an assertion to the unit test to make sure reference count is preserved.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (f5579ca) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.08%`.
   > The diff coverage is `45.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.35%   -1.09%     
   ==========================================
     Files        1075     1292     +217     
     Lines       54773    62217    +7444     
     Branches     8168     9021     +853     
   ==========================================
   + Hits        36396    40665    +4269     
   - Misses      15700    18654    +2954     
   - Partials     2677     2898     +221     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1140 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...353d70d](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] chenboat commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
chenboat commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544727058



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.

Review comment:
       is this table going to use heap or non-heap 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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (a94ab18) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.05%`.
   > The diff coverage is `45.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.39%   -1.06%     
   ==========================================
     Files        1075     1292     +217     
     Lines       54773    62215    +7442     
     Branches     8168     9020     +852     
   ==========================================
   + Hits        36396    40685    +4289     
   - Misses      15700    18634    +2934     
   - Partials     2677     2896     +219     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.39% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1141 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...56e0a15](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r544713506



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());

Review comment:
       nit:
   ```suggestion
         _logger.info("Successfully added segment {} and loaded lookup table: {}", getTableName());
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);
+      }
+    } finally {
+      _lookupTableWriteLock.unlock();
+    }
+  }
+
+  public GenericRow lookupRowByPrimaryKey(PrimaryKey pk) {
+    _lookupTableReadLock.lock();
+    try {
+      return _lookupTable.get(pk);
+    } finally {
+      _lookupTableReadLock.unlock();
+    }
+  }
+
+  public FieldSpec getColumnFieldSpec(String columnName) {

Review comment:
       Please annotate as visiblefortesting.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());

Review comment:
       Good to add segment name in the log

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Review comment:
       Please move this check to the table creation code in the controller.
   Also ensure that the other checks are there as well:
   (1) table type must be OFFLINE
   (2) table ingestion should be REFRESH




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770827



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        _lookupTable.clear();
+        return;
+      }
+
+      List<File> indexDirs = new ArrayList<>();
+      for (SegmentDataManager segmentManager: segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        System.out.println(indexSegment.getSegmentName());

Review comment:
       👍 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu merged pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu merged pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543798021



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {

Review comment:
       I think it is the same as the loading happens after you clear the map even in the old code. If it throws exception, segment will get into ERROR state, and we'll have to wait until the next segment triggers this method again.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543792267



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
##########
@@ -53,6 +53,11 @@ public static TableDataManager getTableDataManager(@Nonnull TableDataManagerConf
     TableDataManager tableDataManager;
     switch (TableType.valueOf(tableDataManagerConfig.getTableDataManagerType())) {
       case OFFLINE:
+        if (tableDataManagerConfig.isDimTable()) {
+          tableDataManager =
+              DimensionTableDataManager.createInstanceByTableName(tableDataManagerConfig.getTableName());
+          break;
+        }

Review comment:
       Updated




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543800298



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);

Review comment:
       It is still thread-safe unless multiple threads are trying to get the instance for the same table. But to be completely thread-safe, you may use:
   return _instances.computeIfAbsent(tableName, k -> new DimensionTableDataManager());
   
   The difference is that this method won't always create a new instance of the data manager.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770167



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {

Review comment:
       Applied your suggestion, however I think the older way was safer in case where loading of the segment is interrupted with an exception in segment reader. Now, if an exception happens, older content is lost. Let me know if you thing this is some edge case worth thinking about.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546176282



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/SegmentDataManager.java
##########
@@ -29,7 +29,7 @@
   private int _referenceCount = 1;
 
   @VisibleForTesting
-  synchronized int getReferenceCount() {
+  public synchronized int getReferenceCount() {

Review comment:
       I hope this API is still only for testing. The API does not make sense in production since the ref count could change any time after you get it. Please confirm.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r542962860



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {

Review comment:
       For clarity
   ```suggestion
     public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {

Review comment:
       For clarity
   ```suggestion
     public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        _lookupTable.clear();
+        return;
+      }
+
+      List<File> indexDirs = new ArrayList<>();
+      for (SegmentDataManager segmentManager: segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        System.out.println(indexSegment.getSegmentName());
+        indexDirs.add(indexSegment.getSegmentMetadata().getIndexDir());
+      }
+      MultiplePinotSegmentRecordReader reader = new MultiplePinotSegmentRecordReader(indexDirs);

Review comment:
       No need to use `MultiplePinotSegmentRecordReader` here, use `PinotSegmentRecordReader` instead on each segment. Use try-with-resource to close the reader after the reading

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManagerTest.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import java.io.File;
+import java.net.URL;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixManager;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentCreationDriverFactory;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.core.segment.index.loader.LoaderTest;
+import org.apache.pinot.segments.v1.creator.SegmentTestUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeSuite;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class DimensionTableDataManagerTest {
+  private static final String TABLE_NAME = "dimBaseballTeams";
+  private static final File INDEX_DIR = new File(LoaderTest.class.getName());
+  private static final String AVRO_DATA_PATH = "data/dimBaseballTeams.avro";
+
+  private File _indexDir;
+  private IndexLoadingConfig _indexLoadingConfig;
+
+  @BeforeSuite
+  public void setUp()
+      throws Exception {
+    // prepare segment data
+    URL resourceUrl = getClass().getClassLoader().getResource(AVRO_DATA_PATH);
+    Assert.assertNotNull(resourceUrl);
+    File avroFile = new File(resourceUrl.getFile());
+
+    // create segment
+    SegmentGeneratorConfig segmentGeneratorConfig =
+        SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, INDEX_DIR, TABLE_NAME);
+    segmentGeneratorConfig.setSegmentVersion(SegmentVersion.v3);
+    SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+    _indexDir = new File(INDEX_DIR, driver.getSegmentName());
+
+    _indexLoadingConfig = new IndexLoadingConfig();
+    _indexLoadingConfig.setReadMode(ReadMode.mmap);
+    _indexLoadingConfig.setSegmentVersion(SegmentVersion.v3);
+  }
+
+  @AfterSuite

Review comment:
       ```suggestion
     @AfterClass
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);

Review comment:
       (nit) reformat

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());

Review comment:
       (nit)
   ```suggestion
         _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);

Review comment:
       Always create a new one instead of reusing the existing one (there should be no existing one normally)
   ```suggestion
       DimensionTableDataManager instance = new DimensionTableDataManager();
       _instances.put(tableNameWithType, instance);
       return instance;
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {

Review comment:
       To simplify the logic, clear the lookup table first, then load it

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}

Review comment:
       (nit) 
   ```suggestion
     private DimensionTableDataManager() {
     }
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());

Review comment:
       Wrap it into a `RuntimeException` and throw it

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());

Review comment:
       (nit)
   ```suggestion
         _logger.info("Successfully loaded lookup table: {}", getTableName());
   ```

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManagerTest.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import java.io.File;
+import java.net.URL;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixManager;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentCreationDriverFactory;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.core.segment.index.loader.LoaderTest;
+import org.apache.pinot.segments.v1.creator.SegmentTestUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeSuite;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class DimensionTableDataManagerTest {
+  private static final String TABLE_NAME = "dimBaseballTeams";
+  private static final File INDEX_DIR = new File(LoaderTest.class.getName());
+  private static final String AVRO_DATA_PATH = "data/dimBaseballTeams.avro";
+
+  private File _indexDir;
+  private IndexLoadingConfig _indexLoadingConfig;
+
+  @BeforeSuite

Review comment:
       ```suggestion
     @BeforeClass
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      _logger.error("Error reloading lookup table after segment remove for table {}", getTableName());
+    }
+  }
+
+  /*
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable() throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        _lookupTable.clear();
+        return;
+      }
+
+      List<File> indexDirs = new ArrayList<>();
+      for (SegmentDataManager segmentManager: segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        System.out.println(indexSegment.getSegmentName());

Review comment:
       Remove




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543792223



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,163 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /*
+   * `createInstanceByTableName` should only be used by the `TableDataManagerProvider` and the returned instance
+   * should be properly initialized via `TableDataManager::init` method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    DimensionTableDataManager instance = new DimensionTableDataManager();
+    _instances.put(tableNameWithType, instance);
+    return instance;
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+

Review comment:
       Cleaned




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545508499



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.

Review comment:
       I think @chenboat  is asking if the `loadMode` setting is going to be `HEAP` or `MMAP`. Perhaps the answer is that it does not matter, since the contents are instantiated into the hash table.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (c8fd8bf) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.09%`.
   > The diff coverage is `45.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.35%   -1.10%     
   ==========================================
     Files        1075     1292     +217     
     Lines       54773    62216    +7443     
     Branches     8168     9021     +853     
   ==========================================
   + Hits        36396    40662    +4266     
   - Misses      15700    18653    +2953     
   - Partials     2677     2901     +224     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1140 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...c8fd8bf](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-749187775


   Thanks folks!
   
   @Jackie-Jiang can you please merge the branch so I can move to the next step?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (f719a84) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `1.03%`.
   > The diff coverage is `56.80%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   - Coverage   66.44%   65.41%   -1.04%     
   ==========================================
     Files        1075     1295     +220     
     Lines       54773    62582    +7809     
     Branches     8168     9102     +934     
   ==========================================
   + Hits        36396    40937    +4541     
   - Misses      15700    18730    +3030     
   - Partials     2677     2915     +238     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.41% <56.80%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <0.00%> (+9.52%)` | :arrow_up: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <0.00%> (-13.29%)` | :arrow_down: |
   | [...inot/client/JsonAsyncHttpPinotClientTransport.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0pzb25Bc3luY0h0dHBQaW5vdENsaWVudFRyYW5zcG9ydC5qYXZh) | `10.90% <0.00%> (-51.10%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `73.80% <ø> (+0.63%)` | :arrow_up: |
   | [...common/config/tuner/NoOpTableTableConfigTuner.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL3R1bmVyL05vT3BUYWJsZVRhYmxlQ29uZmlnVHVuZXIuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ot/common/config/tuner/RealTimeAutoIndexTuner.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL3R1bmVyL1JlYWxUaW1lQXV0b0luZGV4VHVuZXIuamF2YQ==) | `100.00% <ø> (ø)` | |
   | ... and [1147 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [003442a...f719a84](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (71c70ee) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **increase** coverage by `7.58%`.
   > The diff coverage is `68.09%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   + Coverage   66.44%   74.03%   +7.58%     
   ==========================================
     Files        1075     1295     +220     
     Lines       54773    62572    +7799     
     Branches     8168     9097     +929     
   ==========================================
   + Hits        36396    46326    +9930     
   + Misses      15700    13234    -2466     
   - Partials     2677     3012     +335     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `45.02% <51.69%> (?)` | |
   | unittests | `65.42% <44.92%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `44.44% <0.00%> (-4.40%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `78.57% <ø> (+5.40%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../pinot/common/function/DateTimePatternHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRGF0ZVRpbWVQYXR0ZXJuSGFuZGxlci5qYXZh) | `83.33% <ø> (ø)` | |
   | [...ot/common/function/FunctionDefinitionRegistry.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25EZWZpbml0aW9uUmVnaXN0cnkuamF2YQ==) | `88.88% <ø> (+44.44%)` | :arrow_up: |
   | [...org/apache/pinot/common/function/FunctionInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbmZvLmphdmE=) | `100.00% <ø> (ø)` | |
   | ... and [1091 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...22b1549](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545314869



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);

Review comment:
       This won't work because if the loading throws exception, it will only release one segment manager




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545540784



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Review comment:
       Good points Subbu. This is actually something @yupeng9 also mentioned in the original POC review, but seems like I forgot to add in this PR.
   Now, I added validations to `TableConfigUtils` to capture 'TableType', 'IngestionType' and 'PrimaryKey' requirements for Dimension tables. Please take a look.
   Also, I would prefer to keep the 'PrimaryKey' precondition here, since I think it documents the constraint very well and is better than doing a null check deep down. Let me know what you think.
   Thanks




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r546031542



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully added segment {} and loaded lookup table: {}", indexDir.getName(), getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment {} and reloaded lookup table: {}", segmentName, getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String
+          .format("Error reloading lookup table after segment remove ({}) for table: {}", segmentName, getTableName()),
+          e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      try {
+        for (SegmentDataManager segmentManager : segmentManagers) {

Review comment:
       Right, I listed the table size issue in the description as one of the things I'd like to addresses subsequently.
   
   What I'm thinking is putting a 'default maximum' value for Dimension table size quota, say something like 200MB. But I also think this value should be configurable via server config (by the Pinot admin, not the user). This should allow use cases for larger tables if you have the hardware resources to handle them. I think restricting the table to be a single segment is somewhat arbitrary and doesn't help us that much. Happy to hear you and @Jackie-Jiang 's thoughts 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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545493923



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table: {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error reloading lookup table after segment remove for table: {}", getTableName()), e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      for (SegmentDataManager segmentManager : segmentManagers) {
+        IndexSegment indexSegment = segmentManager.getSegment();
+        try (PinotSegmentRecordReader reader =
+            new PinotSegmentRecordReader(indexSegment.getSegmentMetadata().getIndexDir())) {
+          while (reader.hasNext()) {
+            GenericRow row = reader.next();
+            _lookupTable.put(row.getPrimaryKey(_primaryKeyColumns), row);
+          }
+        }
+        releaseSegment(segmentManager);

Review comment:
       Oh, you're right, missed that completely. Updating as suggested.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r545972540



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Review comment:
       If we already check at the time of adding the table config, then it should never be null here, right?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via {@link #getInstanceByTableName} static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {
+  }
+
+  /**
+   * `createInstanceByTableName` should only be used by the {@link TableDataManagerProvider} and the returned
+   * instance should be properly initialized via {@link #init} method before using.
+   */
+  public static DimensionTableDataManager createInstanceByTableName(String tableNameWithType) {
+    return _instances.computeIfAbsent(tableNameWithType, k -> new DimensionTableDataManager());
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableNameWithType) {
+    return _instances.get(tableNameWithType);
+  }
+
+  /**
+   * Instance properties/methods
+   */
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully added segment {} and loaded lookup table: {}", indexDir.getName(), getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Error loading lookup table: %s", getTableName()), e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment {} and reloaded lookup table: {}", segmentName, getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(String
+          .format("Error reloading lookup table after segment remove ({}) for table: {}", segmentName, getTableName()),
+          e);
+    }
+  }
+
+  /**
+   * `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
+   */
+  private void loadLookupTable()
+      throws Exception {
+    _lookupTableWriteLock.lock();
+    try {
+      _lookupTable.clear();
+      List<SegmentDataManager> segmentManagers = acquireAllSegments();
+      if (segmentManagers.size() == 0) {
+        return;
+      }
+
+      try {
+        for (SegmentDataManager segmentManager : segmentManagers) {

Review comment:
       Since we want that a lookup table should be small, it may be worth asserting that the table has to be in one segment?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io edited a comment on pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#issuecomment-743002120


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=h1) Report
   > Merging [#6346](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=desc) (f5579ca) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **increase** coverage by `7.61%`.
   > The diff coverage is `68.14%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6346/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6346      +/-   ##
   ==========================================
   + Coverage   66.44%   74.06%   +7.61%     
   ==========================================
     Files        1075     1292     +217     
     Lines       54773    62217    +7444     
     Branches     8168     9021     +853     
   ==========================================
   + Hits        36396    46081    +9685     
   + Misses      15700    13159    -2541     
   - Partials     2677     2977     +300     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `45.46% <51.75%> (?)` | |
   | unittests | `65.35% <45.12%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `44.44% <0.00%> (-4.40%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `78.57% <ø> (+5.40%)` | :arrow_up: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | [...pinot/common/function/AggregationFunctionType.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vQWdncmVnYXRpb25GdW5jdGlvblR5cGUuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../pinot/common/function/DateTimePatternHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRGF0ZVRpbWVQYXR0ZXJuSGFuZGxlci5qYXZh) | `83.33% <ø> (ø)` | |
   | [...ot/common/function/FunctionDefinitionRegistry.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25EZWZpbml0aW9uUmVnaXN0cnkuamF2YQ==) | `88.88% <ø> (+44.44%)` | :arrow_up: |
   | [...org/apache/pinot/common/function/FunctionInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbmZvLmphdmE=) | `100.00% <ø> (ø)` | |
   | ... and [1080 more](https://codecov.io/gh/apache/incubator-pinot/pull/6346/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=footer). Last update [64d1054...353d70d](https://codecov.io/gh/apache/incubator-pinot/pull/6346?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] cbalci commented on a change in pull request #6346: Introduce DimensionTableDataManager

Posted by GitBox <gi...@apache.org>.
cbalci commented on a change in pull request #6346:
URL: https://github.com/apache/incubator-pinot/pull/6346#discussion_r543770638



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/DimensionTableDataManager.java
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.data.manager.offline;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.readers.MultiplePinotSegmentRecordReader;
+import org.apache.pinot.core.indexsegment.IndexSegment;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+
+/**
+ * Dimension Table is a special type of OFFLINE table which is assigned to all servers
+ * in a tenant and is used to execute a LOOKUP Transform Function. DimensionTableDataManager
+ * loads the contents into a HashMap for faster access thus the size should be small
+ * enough to easily fit in memory.
+ *
+ * DimensionTableDataManager uses Registry of Singletons pattern to store one instance per table
+ * which can be accessed via `getInstanceByTableName` static method.
+ */
+@ThreadSafe
+public class DimensionTableDataManager extends OfflineTableDataManager {
+  // Storing singletons per table in a HashMap
+  private static final Map<String, DimensionTableDataManager> _instances = new ConcurrentHashMap<>();
+
+  private DimensionTableDataManager() {}
+
+  public static DimensionTableDataManager createInstanceByTableName(String tableName) {
+    _instances.putIfAbsent(tableName, new DimensionTableDataManager());
+    return _instances.get(tableName);
+  }
+
+  public static DimensionTableDataManager getInstanceByTableName(String tableName) {
+    return _instances.get(tableName);
+  }
+
+  /*
+   * DimensionTableDataManager Instance Properties/Methods
+   */
+
+  private final ReadWriteLock _rwl = new ReentrantReadWriteLock();
+  private final Lock _lookupTableReadLock = _rwl.readLock();
+  private final Lock _lookupTableWriteLock = _rwl.writeLock();
+
+  // _lookupTable is a HashMap used for storing/serving records for a table keyed by table PK
+  @GuardedBy("_rwl")
+  private final Map<PrimaryKey, GenericRow> _lookupTable = new HashMap<>();
+  private Schema _tableSchema;
+  private List<String> _primaryKeyColumns;
+
+  @Override
+  protected void doInit() {
+    super.doInit();
+
+    _tableSchema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+    Preconditions.checkState(_tableSchema != null, "Failed to find schema for table: %s", _tableNameWithType);
+    _primaryKeyColumns = _tableSchema.getPrimaryKeyColumns();
+    Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
+        "Primary key columns must be configured for dimension table: %s", _tableNameWithType);
+  }
+
+  @Override
+  public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
+      throws Exception {
+    super.addSegment(indexDir, indexLoadingConfig);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully loaded lookup table for {}", getTableName());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Error loading lookup table: %s", getTableName()),e);
+    }
+  }
+
+  @Override
+  public void removeSegment(String segmentName) {
+    super.removeSegment(segmentName);
+    try {
+      loadLookupTable();
+      _logger.info("Successfully removed segment and reloaded lookup table for {}", getTableName());

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:
users@infra.apache.org



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