Skip to content

Commit

Permalink
Add memory optimized dimension table (#9802)
Browse files Browse the repository at this point in the history
* Add mem optimized dim table

* rename config and remove template variable

* Add dimensionTable config to table config builder and serializers

* fix linting

* WIP: close segments after memtable is done

* Do not close segments when data is not preLoaded

* Fix segment close logic

* closing segment data manager inside dimension table

* Fix linting

Co-authored-by: Kartik Khare <[email protected]>
  • Loading branch information
KKcorps and Kartik Khare authored Dec 2, 2022
1 parent 568d858 commit 4235420
Show file tree
Hide file tree
Showing 10 changed files with 420 additions and 40 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.commons.collections.MapUtils;
import org.apache.helix.zookeeper.datamodel.ZNRecord;
import org.apache.pinot.spi.config.table.DedupConfig;
import org.apache.pinot.spi.config.table.DimensionTableConfig;
import org.apache.pinot.spi.config.table.FieldConfig;
import org.apache.pinot.spi.config.table.IndexingConfig;
import org.apache.pinot.spi.config.table.QueryConfig;
Expand Down Expand Up @@ -139,6 +140,12 @@ public static TableConfig fromZNRecord(ZNRecord znRecord)
dedupConfig = JsonUtils.stringToObject(dedupConfigString, DedupConfig.class);
}

DimensionTableConfig dimensionTableConfig = null;
String dimensionTableConfigString = simpleFields.get(TableConfig.DIMENSION_TABLE_CONFIG_KEY);
if (dimensionTableConfigString != null) {
dimensionTableConfig = JsonUtils.stringToObject(dimensionTableConfigString, DimensionTableConfig.class);
}

IngestionConfig ingestionConfig = null;
String ingestionConfigString = simpleFields.get(TableConfig.INGESTION_CONFIG_KEY);
if (ingestionConfigString != null) {
Expand Down Expand Up @@ -175,7 +182,7 @@ public static TableConfig fromZNRecord(ZNRecord znRecord)

return new TableConfig(tableName, tableType, validationConfig, tenantConfig, indexingConfig, customConfig,
quotaConfig, taskConfig, routingConfig, queryConfig, instanceAssignmentConfigMap,
fieldConfigList, upsertConfig, dedupConfig, ingestionConfig, tierConfigList, isDimTable,
fieldConfigList, upsertConfig, dedupConfig, dimensionTableConfig, ingestionConfig, tierConfigList, isDimTable,
tunerConfigList, instancePartitionsMap, segmentAssignmentConfigMap);
}

Expand Down Expand Up @@ -227,6 +234,10 @@ public static ZNRecord toZNRecord(TableConfig tableConfig)
if (dedupConfig != null) {
simpleFields.put(TableConfig.DEDUP_CONFIG_KEY, JsonUtils.objectToString(dedupConfig));
}
DimensionTableConfig dimensionTableConfig = tableConfig.getDimensionTableConfig();
if (dimensionTableConfig != null) {
simpleFields.put(TableConfig.DIMENSION_TABLE_CONFIG_KEY, JsonUtils.objectToString(dimensionTableConfig));
}
IngestionConfig ingestionConfig = tableConfig.getIngestionConfig();
if (ingestionConfig != null) {
simpleFields.put(TableConfig.INGESTION_CONFIG_KEY, JsonUtils.objectToString(ingestionConfig));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,44 +18,20 @@
*/
package org.apache.pinot.core.data.manager.offline;

import java.util.HashMap;
import java.io.Closeable;
import java.util.List;
import java.util.Map;
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;


class DimensionTable {
public interface DimensionTable extends Closeable {

private final Map<PrimaryKey, GenericRow> _lookupTable;
private final Schema _tableSchema;
private final List<String> _primaryKeyColumns;
List<String> getPrimaryKeyColumns();

DimensionTable(Schema tableSchema, List<String> primaryKeyColumns) {
this(tableSchema, primaryKeyColumns, new HashMap<>());
}
GenericRow get(PrimaryKey pk);

DimensionTable(Schema tableSchema, List<String> primaryKeyColumns, Map<PrimaryKey, GenericRow> lookupTable) {
_lookupTable = lookupTable;
_tableSchema = tableSchema;
_primaryKeyColumns = primaryKeyColumns;
}
boolean isEmpty();

List<String> getPrimaryKeyColumns() {
return _primaryKeyColumns;
}

GenericRow get(PrimaryKey pk) {
return _lookupTable.get(pk);
}

boolean isEmpty() {
return _lookupTable.isEmpty();
}

FieldSpec getFieldSpecFor(String columnName) {
return _tableSchema.getFieldSpecFor(columnName);
}
FieldSpec getFieldSpecFor(String columnName);
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand All @@ -32,6 +33,8 @@
import org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader;
import org.apache.pinot.segment.spi.ImmutableSegment;
import org.apache.pinot.segment.spi.IndexSegment;
import org.apache.pinot.spi.config.table.DimensionTableConfig;
import org.apache.pinot.spi.config.table.TableConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.readers.GenericRow;
Expand Down Expand Up @@ -78,16 +81,32 @@ public static DimensionTableDataManager getInstanceByTableName(String tableNameW
AtomicReferenceFieldUpdater.newUpdater(DimensionTableDataManager.class, DimensionTable.class, "_dimensionTable");

private volatile DimensionTable _dimensionTable;
private boolean _disablePreload = false;

@Override
protected void doInit() {
super.doInit();
Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
Preconditions.checkState(schema != null, "Failed to find schema for dimension table: %s", _tableNameWithType);

List<String> primaryKeyColumns = schema.getPrimaryKeyColumns();
Preconditions.checkState(CollectionUtils.isNotEmpty(primaryKeyColumns),
"Primary key columns must be configured for dimension table: %s", _tableNameWithType);
_dimensionTable = new DimensionTable(schema, primaryKeyColumns);

TableConfig tableConfig = ZKMetadataProvider.getTableConfig(_propertyStore, _tableNameWithType);
if (tableConfig != null) {
DimensionTableConfig dimensionTableConfig = tableConfig.getDimensionTableConfig();
if (dimensionTableConfig != null) {
_disablePreload = dimensionTableConfig.isDisablePreload();
}
}

if (_disablePreload) {
_dimensionTable = new MemoryOptimizedDimensionTable(schema, primaryKeyColumns, Collections.emptyMap(),
Collections.emptyList(), this);
} else {
_dimensionTable = new FastLookupDimensionTable(schema, primaryKeyColumns, new HashMap<>());
}
}

@Override
Expand Down Expand Up @@ -117,6 +136,19 @@ public void removeSegment(String segmentName) {
}
}

@Override
protected void doShutdown() {
closeDimensionTable(_dimensionTable);
}

private void closeDimensionTable(DimensionTable dimensionTable) {
try {
dimensionTable.close();
} catch (Exception e) {
_logger.warn("Cannot close dimension table: {}", _tableNameWithType, e);
}
}

/**
* `loadLookupTable()` reads contents of the DimensionTable into _lookupTable HashMap for fast lookup.
*/
Expand All @@ -125,21 +157,28 @@ private void loadLookupTable() {
DimensionTable replacement;
do {
snapshot = _dimensionTable;
replacement = createDimensionTable();
if (_disablePreload) {
replacement = createMemOptimisedDimensionTable();
} else {
replacement = createFastLookupDimensionTable();
}
} while (!UPDATER.compareAndSet(this, snapshot, replacement));

closeDimensionTable(snapshot);
}

private DimensionTable createDimensionTable() {
private DimensionTable createFastLookupDimensionTable() {
Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
Preconditions.checkState(schema != null, "Failed to find schema for dimension table: %s", _tableNameWithType);

List<String> primaryKeyColumns = schema.getPrimaryKeyColumns();
Preconditions.checkState(CollectionUtils.isNotEmpty(primaryKeyColumns),
"Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Map<PrimaryKey, GenericRow> lookupTable = new HashMap<>();
List<SegmentDataManager> segmentManagers = acquireAllSegments();
List<SegmentDataManager> segmentDataManagers = acquireAllSegments();
try {
for (SegmentDataManager segmentManager : segmentManagers) {
for (SegmentDataManager segmentManager : segmentDataManagers) {
IndexSegment indexSegment = segmentManager.getSegment();
int numTotalDocs = indexSegment.getSegmentMetadata().getTotalDocs();
if (numTotalDocs > 0) {
Expand All @@ -156,14 +195,46 @@ private DimensionTable createDimensionTable() {
}
}
}
return new DimensionTable(schema, primaryKeyColumns, lookupTable);
return new FastLookupDimensionTable(schema, primaryKeyColumns, lookupTable);
} finally {
for (SegmentDataManager segmentManager : segmentManagers) {
for (SegmentDataManager segmentManager : segmentDataManagers) {
releaseSegment(segmentManager);
}
}
}

private DimensionTable createMemOptimisedDimensionTable() {
Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
Preconditions.checkState(schema != null, "Failed to find schema for dimension table: %s", _tableNameWithType);

List<String> primaryKeyColumns = schema.getPrimaryKeyColumns();
Preconditions.checkState(CollectionUtils.isNotEmpty(primaryKeyColumns),
"Primary key columns must be configured for dimension table: %s", _tableNameWithType);

Map<PrimaryKey, LookupRecordLocation> lookupTable = new HashMap<>();
List<SegmentDataManager> segmentDataManagers = acquireAllSegments();
for (SegmentDataManager segmentManager : segmentDataManagers) {
IndexSegment indexSegment = segmentManager.getSegment();
int numTotalDocs = indexSegment.getSegmentMetadata().getTotalDocs();
if (numTotalDocs > 0) {
try {
PinotSegmentRecordReader recordReader = new PinotSegmentRecordReader();
recordReader.init(indexSegment);
for (int i = 0; i < numTotalDocs; i++) {
GenericRow row = new GenericRow();
recordReader.getRecord(i, row);
lookupTable.put(row.getPrimaryKey(primaryKeyColumns), new LookupRecordLocation(recordReader, i));
}
} catch (Exception e) {
throw new RuntimeException(
"Caught exception while reading records from segment: " + indexSegment.getSegmentName());
}
}
}
return new MemoryOptimizedDimensionTable(schema, primaryKeyColumns, lookupTable,
segmentDataManagers, this);
}

public boolean isPopulated() {
return !_dimensionTable.isEmpty();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pinot.core.data.manager.offline;

import java.io.IOException;
import java.util.List;
import java.util.Map;
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;


class FastLookupDimensionTable implements DimensionTable {

private Map<PrimaryKey, GenericRow> _lookupTable;
private final Schema _tableSchema;
private final List<String> _primaryKeyColumns;

FastLookupDimensionTable(Schema tableSchema, List<String> primaryKeyColumns,
Map<PrimaryKey, GenericRow> lookupTable) {
_lookupTable = lookupTable;
_tableSchema = tableSchema;
_primaryKeyColumns = primaryKeyColumns;
}

@Override
public List<String> getPrimaryKeyColumns() {
return _primaryKeyColumns;
}

@Override
public GenericRow get(PrimaryKey pk) {
return _lookupTable.get(pk);
}

@Override
public boolean isEmpty() {
return _lookupTable.isEmpty();
}

@Override
public FieldSpec getFieldSpecFor(String columnName) {
return _tableSchema.getFieldSpecFor(columnName);
}

@Override
public void close()
throws IOException {
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/**
* 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 org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader;
import org.apache.pinot.spi.data.readers.GenericRow;


public class LookupRecordLocation {
private final PinotSegmentRecordReader _pinotSegmentRecordReader;
private final int _docId;

public LookupRecordLocation(PinotSegmentRecordReader pinotSegmentRecordReader, int docId) {
_pinotSegmentRecordReader = pinotSegmentRecordReader;
_docId = docId;
}

public PinotSegmentRecordReader getPinotSegmentRecordReader() {
return _pinotSegmentRecordReader;
}

public int getDocId() {
return _docId;
}

public GenericRow getRecord(GenericRow reuse) {
_pinotSegmentRecordReader.getRecord(_docId, reuse);
return reuse;
}
}
Loading

0 comments on commit 4235420

Please sign in to comment.