From 0e316390fa4219bb8d91b12b41ac7e71903566cf Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Wed, 29 Nov 2023 16:04:26 +0530 Subject: [PATCH 01/30] Enricher interface --- pinot-segment-local/pom.xml | 4 + .../local/recordenricher/RecordEnricher.java | 28 ++++++ .../RecordEnricherPipeline.java | 56 +++++++++++ .../clp/CLPEncodingEnricher.java | 99 +++++++++++++++++++ .../impl/SegmentIndexCreationDriverImpl.java | 5 + .../table/ingestion/EnrichmentConfig.java | 49 +++++++++ .../table/ingestion/IngestionConfig.java | 14 +++ pom.xml | 1 + 8 files changed, 256 insertions(+) create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java create mode 100644 pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java diff --git a/pinot-segment-local/pom.xml b/pinot-segment-local/pom.xml index 5b2e5e24c7e2..8d81adafb060 100644 --- a/pinot-segment-local/pom.xml +++ b/pinot-segment-local/pom.xml @@ -152,5 +152,9 @@ test-jar test + + com.yscope.clp + clp-ffi + diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java new file mode 100644 index 000000000000..4a0badc5b179 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java @@ -0,0 +1,28 @@ +/** + * 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.segment.local.recordenricher; + +import java.util.Map; +import org.apache.pinot.spi.data.readers.GenericRow; + + +public abstract class RecordEnricher { + abstract public void init(Map enricherProps); + abstract public void enrich(GenericRow record); +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java new file mode 100644 index 000000000000..fd769113d298 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java @@ -0,0 +1,56 @@ +/** + * 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.segment.local.recordenricher; + +import java.util.ArrayList; +import java.util.List; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.EnrichmentConfig; +import org.apache.pinot.spi.data.readers.GenericRow; + + +public class RecordEnricherPipeline { + private final List _enrichers = new ArrayList<>(); + + public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { + RecordEnricherPipeline pipeline = new RecordEnricherPipeline(); + for (EnrichmentConfig enrichmentConfig : tableConfig.getIngestionConfig().getEnrichmentConfigs()) { + try { + RecordEnricher enricher = + (RecordEnricher) Class.forName(enrichmentConfig.getEnricherClassName()).newInstance(); + enricher.init(enrichmentConfig.getEnricherProps()); + pipeline.add(enricher); + } catch (Exception e) { + throw new RuntimeException("Failed to instantiate record enricher: " + enrichmentConfig.getEnricherClassName(), + e); + } + } + return pipeline; + } + + public void add(RecordEnricher enricher) { + _enrichers.add(enricher); + } + + public void run(GenericRow record) { + for (RecordEnricher enricher : _enrichers) { + enricher.enrich(record); + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java new file mode 100644 index 000000000000..025de2c68e49 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java @@ -0,0 +1,99 @@ +package org.apache.pinot.segment.local.recordenricher.clp; + +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.segment.local.recordenricher.RecordEnricher; +import org.apache.pinot.spi.data.readers.GenericRow; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageEncoder; +import org.apache.pinot.sql.parsers.rewriter.CLPDecodeRewriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * 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. + */ +public class CLPEncodingEnricher extends RecordEnricher { + public static final String FIELDS_FOR_CLP_ENCODING_CONFIG_KEY = "fieldsForClpEncoding"; + public static final String FIELDS_FOR_CLP_ENCODING_SEPARATOR = ","; + private static final Logger LOGGER = LoggerFactory.getLogger(CLPEncodingEnricher.class); + + private Set _fields; + private EncodedMessage _clpEncodedMessage; + private MessageEncoder _clpMessageEncoder; + + @Override + public void init(Map enricherProperties) { + String concatenatedFieldNames = enricherProperties.get(FIELDS_FOR_CLP_ENCODING_CONFIG_KEY); + if (StringUtils.isEmpty(concatenatedFieldNames)) { + throw new IllegalArgumentException("Missing required property: " + FIELDS_FOR_CLP_ENCODING_CONFIG_KEY); + } else { + _fields = Set.of(concatenatedFieldNames.split(FIELDS_FOR_CLP_ENCODING_SEPARATOR)); + } + + _clpEncodedMessage = new EncodedMessage(); + _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + + @Override + public void enrich(GenericRow record) { + try { + for (String field : _fields) { + Object value = record.getValue(field); + if (value != null) { + enrichWithClpEncodedFields(field, value, record); + } + } + } catch (Exception e) { + LOGGER.error("Failed to enrich record: {}", record); + } + } + + private void enrichWithClpEncodedFields(String key, Object value, GenericRow to) { + String logtype = null; + Object[] dictVars = null; + Object[] encodedVars = null; + if (null != value) { + if (!(value instanceof String)) { + LOGGER.error("Can't encode value of type {} with CLP. name: '{}', value: '{}'", + value.getClass().getSimpleName(), key, value); + } else { + String valueAsString = (String) value; + try { + _clpMessageEncoder.encodeMessage(valueAsString, _clpEncodedMessage); + logtype = _clpEncodedMessage.getLogTypeAsString(); + encodedVars = _clpEncodedMessage.getEncodedVarsAsBoxedLongs(); + dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + } catch (IOException e) { + LOGGER.error("Can't encode field with CLP. name: '{}', value: '{}', error: {}", key, valueAsString, + e.getMessage()); + } + } + } + + to.putValue(key + CLPDecodeRewriter.LOGTYPE_COLUMN_SUFFIX, logtype); + to.putValue(key + CLPDecodeRewriter.DICTIONARY_VARS_COLUMN_SUFFIX, dictVars); + to.putValue(key + CLPDecodeRewriter.ENCODED_VARS_COLUMN_SUFFIX, encodedVars); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java index 129bc35ccfef..bcde7f3d716f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java @@ -36,6 +36,7 @@ import javax.annotation.Nullable; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.io.FileUtils; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.recordtransformer.ComplexTypeTransformer; import org.apache.pinot.segment.local.recordtransformer.RecordTransformer; import org.apache.pinot.segment.local.segment.creator.RecordReaderSegmentCreationDataSource; @@ -100,6 +101,7 @@ public class SegmentIndexCreationDriverImpl implements SegmentIndexCreationDrive private SegmentCreator _indexCreator; private SegmentIndexCreationInfo _segmentIndexCreationInfo; private Schema _dataSchema; + private RecordEnricherPipeline _recordEnricherPipeline; private TransformPipeline _transformPipeline; private IngestionSchemaValidator _ingestionSchemaValidator; private int _totalDocs = 0; @@ -168,6 +170,7 @@ public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSo } public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSource, + RecordEnricherPipeline enricherPipeline, TransformPipeline transformPipeline) throws Exception { _config = config; @@ -178,6 +181,7 @@ public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSo if (config.isFailOnEmptySegment()) { Preconditions.checkState(_recordReader.hasNext(), "No record in data source"); } + _recordEnricherPipeline = enricherPipeline; _transformPipeline = transformPipeline; // Use the same transform pipeline if the data source is backed by a record reader if (dataSource instanceof RecordReaderSegmentCreationDataSource) { @@ -244,6 +248,7 @@ public void build() // Should not be needed anymore. // Add row to indexes + _recordEnricherPipeline.run(decodedRow); _transformPipeline.processRow(decodedRow, reusedResult); recordReadStopTime = System.nanoTime(); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java new file mode 100644 index 000000000000..a0089cea8224 --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java @@ -0,0 +1,49 @@ +/** + * 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.spi.config.table.ingestion; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import java.util.Map; +import org.apache.pinot.spi.config.BaseJsonConfig; + + +public class EnrichmentConfig extends BaseJsonConfig { + @JsonPropertyDescription("Enricher class name") + private final String _enricherClassName; + + @JsonPropertyDescription("Enricher properties") + private final Map _enricherProps; + + @JsonCreator + public EnrichmentConfig(@JsonProperty("enricherClassName") String className, + @JsonProperty("transformFunction") Map enricherProps) { + _enricherClassName = className; + _enricherProps = enricherProps; + } + + public String getEnricherClassName() { + return _enricherClassName; + } + + public Map getEnricherProps() { + return _enricherProps; + } +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java index 86ff9cec6f83..5af9cdcc5076 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java @@ -39,6 +39,9 @@ public class IngestionConfig extends BaseJsonConfig { @JsonPropertyDescription("Config related to filtering records during ingestion") private FilterConfig _filterConfig; + @JsonPropertyDescription("Config related to enriching records during ingestion") + private List _enrichmentConfigs; + @JsonPropertyDescription("Configs related to record transformation functions applied during ingestion") private List _transformConfigs; @@ -63,12 +66,14 @@ public class IngestionConfig extends BaseJsonConfig { @Deprecated public IngestionConfig(@Nullable BatchIngestionConfig batchIngestionConfig, @Nullable StreamIngestionConfig streamIngestionConfig, @Nullable FilterConfig filterConfig, + @Nullable List enrichmentConfigs, @Nullable List transformConfigs, @Nullable ComplexTypeConfig complexTypeConfig, @Nullable SchemaConformingTransformerConfig schemaConformingTransformerConfig, @Nullable List aggregationConfigs) { _batchIngestionConfig = batchIngestionConfig; _streamIngestionConfig = streamIngestionConfig; _filterConfig = filterConfig; + _enrichmentConfigs = enrichmentConfigs; _transformConfigs = transformConfigs; _complexTypeConfig = complexTypeConfig; _schemaConformingTransformerConfig = schemaConformingTransformerConfig; @@ -93,6 +98,11 @@ public FilterConfig getFilterConfig() { return _filterConfig; } + @Nullable + public List getEnrichmentConfigs() { + return _enrichmentConfigs; + } + @Nullable public List getTransformConfigs() { return _transformConfigs; @@ -137,6 +147,10 @@ public void setFilterConfig(FilterConfig filterConfig) { _filterConfig = filterConfig; } + public void setEnrichmentConfigs(List enrichmentConfigs) { + _enrichmentConfigs = enrichmentConfigs; + } + public void setTransformConfigs(List transformConfigs) { _transformConfigs = transformConfigs; } diff --git a/pom.xml b/pom.xml index 9358dd21b6a5..d864ec7e07de 100644 --- a/pom.xml +++ b/pom.xml @@ -27,6 +27,7 @@ org.apache apache 21 + org.apache.pinot From c9fd2017d43e2703997432998cf6bbd140011748 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 18 Dec 2023 10:26:36 +0530 Subject: [PATCH 02/30] Support in realtime ingestion --- .../realtime/RealtimeSegmentDataManager.java | 4 +++ .../RecordEnricherPipeline.java | 7 +++-- .../clp/CLPEncodingEnricher.java | 31 +++++++++---------- .../impl/SegmentIndexCreationDriverImpl.java | 6 ++-- .../segment/local/utils/IngestionUtils.java | 3 +- 5 files changed, 30 insertions(+), 21 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 0f880c5f22fa..7c90b5f8e900 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -55,6 +55,7 @@ import org.apache.pinot.segment.local.realtime.converter.ColumnIndicesForRealtimeTable; import org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.segment.creator.TransformPipeline; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.upsert.PartitionUpsertMetadataManager; @@ -272,6 +273,7 @@ public void deleteSegmentFile() { private final int _partitionGroupId; private final PartitionGroupConsumptionStatus _partitionGroupConsumptionStatus; final String _clientId; + private final RecordEnricherPipeline _recordEnricherPipeline; private final TransformPipeline _transformPipeline; private PartitionGroupConsumer _partitionGroupConsumer = null; private StreamMetadataProvider _partitionMetadataProvider = null; @@ -573,6 +575,7 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi _numRowsErrored++; } else { try { + _recordEnricherPipeline.run(decodedRow.getResult()); _transformPipeline.processRow(decodedRow.getResult(), reusedResult); } catch (Exception e) { _numRowsErrored++; @@ -1476,6 +1479,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf new SegmentErrorInfo(now(), "Failed to initialize the StreamMessageDecoder", e)); throw e; } + _recordEnricherPipeline = RecordEnricherPipeline.fromTableConfig(tableConfig); _transformPipeline = new TransformPipeline(tableConfig, schema); // Acquire semaphore to create stream consumers try { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java index fd769113d298..52cac80d9fc2 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java @@ -30,10 +30,13 @@ public class RecordEnricherPipeline { public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { RecordEnricherPipeline pipeline = new RecordEnricherPipeline(); + if (null == tableConfig.getIngestionConfig() || null == tableConfig.getIngestionConfig().getEnrichmentConfigs()) { + return pipeline; + } + for (EnrichmentConfig enrichmentConfig : tableConfig.getIngestionConfig().getEnrichmentConfigs()) { try { - RecordEnricher enricher = - (RecordEnricher) Class.forName(enrichmentConfig.getEnricherClassName()).newInstance(); + RecordEnricher enricher = (RecordEnricher) Class.forName(enrichmentConfig.getEnricherClassName()).newInstance(); enricher.init(enrichmentConfig.getEnricherProps()); pipeline.add(enricher); } catch (Exception e) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java index 025de2c68e49..5ea599ca9fa6 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java @@ -1,19 +1,3 @@ -package org.apache.pinot.segment.local.recordenricher.clp; - -import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; -import java.io.IOException; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.segment.local.recordenricher.RecordEnricher; -import org.apache.pinot.spi.data.readers.GenericRow; -import com.yscope.clp.compressorfrontend.EncodedMessage; -import com.yscope.clp.compressorfrontend.MessageEncoder; -import org.apache.pinot.sql.parsers.rewriter.CLPDecodeRewriter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -32,6 +16,21 @@ * specific language governing permissions and limitations * under the License. */ +package org.apache.pinot.segment.local.recordenricher.clp; + +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageEncoder; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.segment.local.recordenricher.RecordEnricher; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.sql.parsers.rewriter.CLPDecodeRewriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class CLPEncodingEnricher extends RecordEnricher { public static final String FIELDS_FOR_CLP_ENCODING_CONFIG_KEY = "fieldsForClpEncoding"; public static final String FIELDS_FOR_CLP_ENCODING_SEPARATOR = ","; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java index bcde7f3d716f..f285cac5a9f8 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java @@ -159,14 +159,16 @@ public RecordReader getRecordReader() { public void init(SegmentGeneratorConfig config, RecordReader recordReader) throws Exception { SegmentCreationDataSource dataSource = new RecordReaderSegmentCreationDataSource(recordReader); - init(config, dataSource, new TransformPipeline(config.getTableConfig(), config.getSchema())); + init(config, dataSource, RecordEnricherPipeline.fromTableConfig(config.getTableConfig()), + new TransformPipeline(config.getTableConfig(), config.getSchema())); } @Deprecated public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSource, RecordTransformer recordTransformer, @Nullable ComplexTypeTransformer complexTypeTransformer) throws Exception { - init(config, dataSource, new TransformPipeline(recordTransformer, complexTypeTransformer)); + init(config, dataSource, RecordEnricherPipeline.fromTableConfig(config.getTableConfig()), + new TransformPipeline(recordTransformer, complexTypeTransformer)); } public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSource, diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java index 315bd5becff3..792adc168c0c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java @@ -309,7 +309,8 @@ private static void registerPinotFS(String fileURIScheme, String fsClass, PinotC public static Set getFieldsForRecordExtractor(@Nullable IngestionConfig ingestionConfig, Schema schema) { Set fieldsForRecordExtractor = new HashSet<>(); - if (null != ingestionConfig && null != ingestionConfig.getSchemaConformingTransformerConfig()) { + if (null != ingestionConfig && (null != ingestionConfig.getSchemaConformingTransformerConfig() + || null != ingestionConfig.getEnrichmentConfigs())) { // The SchemaConformingTransformer requires that all fields are extracted, indicated by returning an empty set // here. Compared to extracting the fields specified below, extracting all fields should be a superset. return fieldsForRecordExtractor; From d2b2328d5d2a8a7eb2580e2c8a2892033328d6a7 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 18 Dec 2023 13:03:26 +0530 Subject: [PATCH 03/30] Fix offline segment builder --- .../framework/SegmentProcessorFramework.java | 2 ++ .../realtime/converter/RealtimeSegmentConverter.java | 4 +++- .../local/recordenricher/RecordEnricherPipeline.java | 6 +++++- .../creator/RecordReaderSegmentCreationDataSource.java | 9 +++++++++ .../creator/impl/SegmentIndexCreationDriverImpl.java | 1 + .../spi/config/table/ingestion/EnrichmentConfig.java | 10 +++++----- 6 files changed, 25 insertions(+), 7 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/framework/SegmentProcessorFramework.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/framework/SegmentProcessorFramework.java index fb5a08ff5607..df60e9d834bf 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/framework/SegmentProcessorFramework.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/framework/SegmentProcessorFramework.java @@ -33,6 +33,7 @@ import org.apache.pinot.core.segment.processing.mapper.SegmentMapper; import org.apache.pinot.core.segment.processing.reducer.Reducer; import org.apache.pinot.core.segment.processing.reducer.ReducerFactory; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.recordtransformer.RecordTransformer; import org.apache.pinot.segment.local.segment.creator.RecordReaderSegmentCreationDataSource; import org.apache.pinot.segment.local.segment.creator.TransformPipeline; @@ -214,6 +215,7 @@ private List doProcess() GenericRowFileRecordReader recordReaderForRange = recordReader.getRecordReaderForRange(startRowId, endRowId); SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); driver.init(generatorConfig, new RecordReaderSegmentCreationDataSource(recordReaderForRange), + RecordEnricherPipeline.getPassThroughPipeline(), TransformPipeline.getPassThroughPipeline()); driver.build(); outputSegmentDirs.add(driver.getOutputDirectory()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java index a6e2d067b63c..ec13df2d65f7 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java @@ -26,6 +26,7 @@ import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; import org.apache.pinot.segment.local.realtime.converter.stats.RealtimeSegmentSegmentCreationDataSource; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.segment.creator.TransformPipeline; import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.index.text.TextIndexConfigBuilder; @@ -125,7 +126,8 @@ public void build(@Nullable SegmentVersion segmentVersion, ServerMetrics serverM recordReader.init(_realtimeSegmentImpl, sortedDocIds); RealtimeSegmentSegmentCreationDataSource dataSource = new RealtimeSegmentSegmentCreationDataSource(_realtimeSegmentImpl, recordReader); - driver.init(genConfig, dataSource, TransformPipeline.getPassThroughPipeline()); + driver.init(genConfig, dataSource, RecordEnricherPipeline.getPassThroughPipeline(), + TransformPipeline.getPassThroughPipeline()); if (!_enableColumnMajor) { driver.build(); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java index 52cac80d9fc2..dff85dd73960 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java @@ -28,6 +28,10 @@ public class RecordEnricherPipeline { private final List _enrichers = new ArrayList<>(); + public static RecordEnricherPipeline getPassThroughPipeline() { + return new RecordEnricherPipeline(); + } + public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { RecordEnricherPipeline pipeline = new RecordEnricherPipeline(); if (null == tableConfig.getIngestionConfig() || null == tableConfig.getIngestionConfig().getEnrichmentConfigs()) { @@ -37,7 +41,7 @@ public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { for (EnrichmentConfig enrichmentConfig : tableConfig.getIngestionConfig().getEnrichmentConfigs()) { try { RecordEnricher enricher = (RecordEnricher) Class.forName(enrichmentConfig.getEnricherClassName()).newInstance(); - enricher.init(enrichmentConfig.getEnricherProps()); + enricher.init(enrichmentConfig.getProperties()); pipeline.add(enricher); } catch (Exception e) { throw new RuntimeException("Failed to instantiate record enricher: " + enrichmentConfig.getEnricherClassName(), diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/RecordReaderSegmentCreationDataSource.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/RecordReaderSegmentCreationDataSource.java index 411c040cdd6e..fe4bbeb1067e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/RecordReaderSegmentCreationDataSource.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/RecordReaderSegmentCreationDataSource.java @@ -19,6 +19,7 @@ package org.apache.pinot.segment.local.segment.creator; import org.apache.pinot.common.Utils; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.segment.creator.impl.stats.SegmentPreIndexStatsCollectorImpl; import org.apache.pinot.segment.spi.creator.SegmentCreationDataSource; import org.apache.pinot.segment.spi.creator.SegmentPreIndexStatsCollector; @@ -38,12 +39,17 @@ public class RecordReaderSegmentCreationDataSource implements SegmentCreationDat private static final Logger LOGGER = LoggerFactory.getLogger(RecordReaderSegmentCreationDataSource.class); private final RecordReader _recordReader; + private RecordEnricherPipeline _recordEnricherPipeline; private TransformPipeline _transformPipeline; public RecordReaderSegmentCreationDataSource(RecordReader recordReader) { _recordReader = recordReader; } + public void setRecordEnricherPipeline(RecordEnricherPipeline recordEnricherPipeline) { + _recordEnricherPipeline = recordEnricherPipeline; + } + public void setTransformPipeline(TransformPipeline transformPipeline) { _transformPipeline = transformPipeline; } @@ -51,6 +57,8 @@ public void setTransformPipeline(TransformPipeline transformPipeline) { @Override public SegmentPreIndexStatsCollector gatherStats(StatsCollectorConfig statsCollectorConfig) { try { + RecordEnricherPipeline recordEnricherPipeline = _recordEnricherPipeline != null ? _recordEnricherPipeline + : RecordEnricherPipeline.fromTableConfig(statsCollectorConfig.getTableConfig()); TransformPipeline transformPipeline = _transformPipeline != null ? _transformPipeline : new TransformPipeline(statsCollectorConfig.getTableConfig(), statsCollectorConfig.getSchema()); @@ -64,6 +72,7 @@ public SegmentPreIndexStatsCollector gatherStats(StatsCollectorConfig statsColle reuse.clear(); reuse = _recordReader.next(reuse); + recordEnricherPipeline.run(reuse); transformPipeline.processRow(reuse, reusedResult); for (GenericRow row : reusedResult.getTransformedRows()) { collector.collectRow(row); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java index f285cac5a9f8..d64fb38af513 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java @@ -188,6 +188,7 @@ public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSo // Use the same transform pipeline if the data source is backed by a record reader if (dataSource instanceof RecordReaderSegmentCreationDataSource) { ((RecordReaderSegmentCreationDataSource) dataSource).setTransformPipeline(transformPipeline); + ((RecordReaderSegmentCreationDataSource) dataSource).setRecordEnricherPipeline(enricherPipeline); } // Initialize stats collection diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java index a0089cea8224..6e13ff494acf 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java @@ -30,20 +30,20 @@ public class EnrichmentConfig extends BaseJsonConfig { private final String _enricherClassName; @JsonPropertyDescription("Enricher properties") - private final Map _enricherProps; + private final Map _properties; @JsonCreator public EnrichmentConfig(@JsonProperty("enricherClassName") String className, - @JsonProperty("transformFunction") Map enricherProps) { + @JsonProperty("properties") Map properties) { _enricherClassName = className; - _enricherProps = enricherProps; + _properties = properties; } public String getEnricherClassName() { return _enricherClassName; } - public Map getEnricherProps() { - return _enricherProps; + public Map getProperties() { + return _properties; } } From 262605a0e4b0b1854cddcba4531cecd43092c714 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Thu, 21 Dec 2023 11:08:21 +0530 Subject: [PATCH 04/30] Add enricher to mapper --- .../pinot/core/segment/processing/mapper/SegmentMapper.java | 4 ++++ .../java/org/apache/pinot/queries/TransformQueriesTest.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java index b883d775cc30..fdfb46407b99 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java @@ -37,6 +37,7 @@ import org.apache.pinot.core.segment.processing.timehandler.TimeHandler; import org.apache.pinot.core.segment.processing.timehandler.TimeHandlerFactory; import org.apache.pinot.core.segment.processing.utils.SegmentProcessorUtils; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer; import org.apache.pinot.segment.local.recordtransformer.RecordTransformer; import org.apache.pinot.segment.local.utils.IngestionUtils; @@ -72,6 +73,7 @@ public class SegmentMapper { private final boolean _includeNullFields; private final int _numSortFields; + private final RecordEnricherPipeline _recordEnricherPipeline; private final CompositeTransformer _recordTransformer; private final TimeHandler _timeHandler; private final Partitioner[] _partitioners; @@ -93,6 +95,7 @@ public SegmentMapper(List recordReaderFileConfigs, _fieldSpecs = pair.getLeft(); _numSortFields = pair.getRight(); _includeNullFields = tableConfig.getIndexingConfig().isNullHandlingEnabled(); + _recordEnricherPipeline = RecordEnricherPipeline.fromTableConfig(tableConfig); _recordTransformer = CompositeTransformer.composeAllTransformers(_customRecordTransformers, tableConfig, schema); _timeHandler = TimeHandlerFactory.getTimeHandler(processorConfig); List partitionerConfigs = processorConfig.getPartitionerConfigs(); @@ -164,6 +167,7 @@ private void mapAndTransformRow(RecordReader recordReader, GenericRow reuse, observer.accept(String.format("Doing map phase on data from RecordReader (%d out of %d)", count, totalCount)); while (recordReader.hasNext()) { reuse = recordReader.next(reuse); + _recordEnricherPipeline.run(reuse); // TODO: Add ComplexTypeTransformer here. Currently it is not idempotent so cannot add it diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java index 572080e44e5c..1f04d16d3b1e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java @@ -132,7 +132,7 @@ protected void buildSegment() TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTimeColumnName(TIME) - .setIngestionConfig(new IngestionConfig(null, null, null, + .setIngestionConfig(new IngestionConfig(null, null, null, null, Arrays.asList(new TransformConfig(M1_V2, "Groovy({INT_COL1_V3 == null || " + "INT_COL1_V3 == Integer.MIN_VALUE ? INT_COL1 : INT_COL1_V3 }, INT_COL1, INT_COL1_V3)")), null, null, null)) From fdcdbb73f58e5488e174ecdaf741b122171f7882 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Fri, 22 Dec 2023 12:42:42 +0530 Subject: [PATCH 05/30] Add columnsToExtract config --- .../local/recordenricher/RecordEnricher.java | 2 ++ .../RecordEnricherPipeline.java | 21 +++++++++++++++---- .../clp/CLPEncodingEnricher.java | 4 ++++ .../segment/local/utils/IngestionUtils.java | 12 +++++++---- .../table/ingestion/EnrichmentConfig.java | 2 ++ 5 files changed, 33 insertions(+), 8 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java index 4a0badc5b179..7fd82dc23921 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricher.java @@ -19,10 +19,12 @@ package org.apache.pinot.segment.local.recordenricher; import java.util.Map; +import java.util.Set; import org.apache.pinot.spi.data.readers.GenericRow; public abstract class RecordEnricher { abstract public void init(Map enricherProps); + abstract public Set getInputColumns(); abstract public void enrich(GenericRow record); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java index dff85dd73960..2b4e6e12a976 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java @@ -19,26 +19,30 @@ package org.apache.pinot.segment.local.recordenricher; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.ingestion.EnrichmentConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; import org.apache.pinot.spi.data.readers.GenericRow; public class RecordEnricherPipeline { private final List _enrichers = new ArrayList<>(); + private final Set _columnsToExtract = new HashSet<>(); public static RecordEnricherPipeline getPassThroughPipeline() { return new RecordEnricherPipeline(); } - public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { + public static RecordEnricherPipeline fromIngestionConfig(IngestionConfig ingestionConfig) { RecordEnricherPipeline pipeline = new RecordEnricherPipeline(); - if (null == tableConfig.getIngestionConfig() || null == tableConfig.getIngestionConfig().getEnrichmentConfigs()) { + if (null == ingestionConfig || null == ingestionConfig.getEnrichmentConfigs()) { return pipeline; } - - for (EnrichmentConfig enrichmentConfig : tableConfig.getIngestionConfig().getEnrichmentConfigs()) { + List enrichmentConfigs = ingestionConfig.getEnrichmentConfigs(); + for (EnrichmentConfig enrichmentConfig : enrichmentConfigs) { try { RecordEnricher enricher = (RecordEnricher) Class.forName(enrichmentConfig.getEnricherClassName()).newInstance(); enricher.init(enrichmentConfig.getProperties()); @@ -51,8 +55,17 @@ public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { return pipeline; } + public static RecordEnricherPipeline fromTableConfig(TableConfig tableConfig) { + return fromIngestionConfig(tableConfig.getIngestionConfig()); + } + + public Set getColumnsToExtract() { + return _columnsToExtract; + } + public void add(RecordEnricher enricher) { _enrichers.add(enricher); + _columnsToExtract.addAll(enricher.getInputColumns()); } public void run(GenericRow record) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java index 5ea599ca9fa6..6253068a1b24 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/clp/CLPEncodingEnricher.java @@ -54,6 +54,10 @@ public void init(Map enricherProperties) { BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); } + @Override + public Set getInputColumns() { + return _fields; + } @Override public void enrich(GenericRow record) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java index 792adc168c0c..3f31d772f600 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java @@ -34,6 +34,7 @@ import org.apache.pinot.common.request.context.RequestContextUtils; import org.apache.pinot.segment.local.function.FunctionEvaluator; import org.apache.pinot.segment.local.function.FunctionEvaluatorFactory; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.recordtransformer.ComplexTypeTransformer; import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; @@ -309,8 +310,7 @@ private static void registerPinotFS(String fileURIScheme, String fsClass, PinotC public static Set getFieldsForRecordExtractor(@Nullable IngestionConfig ingestionConfig, Schema schema) { Set fieldsForRecordExtractor = new HashSet<>(); - if (null != ingestionConfig && (null != ingestionConfig.getSchemaConformingTransformerConfig() - || null != ingestionConfig.getEnrichmentConfigs())) { + if (null != ingestionConfig && null != ingestionConfig.getSchemaConformingTransformerConfig()) { // The SchemaConformingTransformer requires that all fields are extracted, indicated by returning an empty set // here. Compared to extracting the fields specified below, extracting all fields should be a superset. return fieldsForRecordExtractor; @@ -380,14 +380,18 @@ private static void extractFieldsFromIngestionConfig(@Nullable IngestionConfig i expressionContext.getColumns(fields); } } + + fields.addAll(RecordEnricherPipeline.fromIngestionConfig(ingestionConfig).getColumnsToExtract()); + List transformConfigs = ingestionConfig.getTransformConfigs(); if (transformConfigs != null) { for (TransformConfig transformConfig : transformConfigs) { FunctionEvaluator expressionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformConfig.getTransformFunction()); fields.addAll(expressionEvaluator.getArguments()); - fields.add(transformConfig - .getColumnName()); // add the column itself too, so that if it is already transformed, we won't + fields.add( + transformConfig.getColumnName()); // add the column itself too, so that if it is already transformed, + // we won't // transform again } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java index 6e13ff494acf..f02d79522578 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/EnrichmentConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import java.util.List; import java.util.Map; import org.apache.pinot.spi.config.BaseJsonConfig; @@ -34,6 +35,7 @@ public class EnrichmentConfig extends BaseJsonConfig { @JsonCreator public EnrichmentConfig(@JsonProperty("enricherClassName") String className, + @JsonProperty("columnsToExtract") List columnsToExtract, @JsonProperty("properties") Map properties) { _enricherClassName = className; _properties = properties; From b4e2083b175134f442bca5f2fce28af09310dfcf Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 8 Jan 2024 16:13:24 +0530 Subject: [PATCH 06/30] Complete enricher pipeline usage --- .../apache/pinot/connector/flink/sink/FlinkSegmentWriter.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java b/pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java index 8815bbf37a20..525f4f045105 100644 --- a/pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java +++ b/pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java @@ -40,6 +40,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.pinot.common.utils.TarGzCompressionUtils; import org.apache.pinot.core.util.SegmentProcessorAvroUtils; +import org.apache.pinot.segment.local.recordenricher.RecordEnricherPipeline; import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer; import org.apache.pinot.segment.local.recordtransformer.RecordTransformer; import org.apache.pinot.segment.local.utils.IngestionUtils; @@ -77,6 +78,7 @@ public class FlinkSegmentWriter implements SegmentWriter { private String _outputDirURI; private Schema _schema; private Set _fieldsToRead; + private RecordEnricherPipeline _recordEnricherPipeline; private RecordTransformer _recordTransformer; private File _stagingDir; @@ -137,6 +139,7 @@ public void init(TableConfig tableConfig, Schema schema, Map bat _schema = schema; _fieldsToRead = _schema.getColumnNames(); + _recordEnricherPipeline = RecordEnricherPipeline.fromTableConfig(_tableConfig); _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema); _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema); _reusableRecord = new GenericData.Record(_avroSchema); @@ -172,6 +175,7 @@ private void resetBuffer() public void collect(GenericRow row) throws IOException { long startTime = System.currentTimeMillis(); + _recordEnricherPipeline.run(row); GenericRow transform = _recordTransformer.transform(row); SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead); _rowCount++; From f57eb3da6c53d50fc1ab1a19084639769c1e52a0 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Wed, 10 Jan 2024 17:09:45 +0530 Subject: [PATCH 07/30] Add CLP as a compression type --- .../local/io/util/VarLengthValueWriter.java | 25 ++- .../writer/impl/CLPForwardIndexWriterV1.java | 183 ++++++++++++++++++ .../SingleValueVarByteRawIndexCreator.java | 22 ++- .../AbstractColumnStatisticsCollector.java | 3 + .../StringColumnPreIndexStatsCollector.java | 23 +++ .../forward/ForwardIndexCreatorFactory.java | 7 +- .../spi/compression/ChunkCompressionType.java | 2 +- .../spi/creator/ColumnIndexCreationInfo.java | 4 + .../spi/creator/IndexCreationContext.java | 41 ++-- .../spi/creator/StatsCollectorConfig.java | 15 ++ .../segment/spi/index/ForwardIndexConfig.java | 3 + .../segment/spi/memory/PinotByteBuffer.java | 2 +- .../pinot/spi/config/table/FieldConfig.java | 1 + .../DictionaryToRawIndexConverter.java | 2 +- 14 files changed, 303 insertions(+), 30 deletions(-) create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java index df61cacc0e3e..778a2c541d55 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java @@ -93,6 +93,18 @@ public class VarLengthValueWriter implements Closeable { private final ByteBuffer _offsetBuffer; private final ByteBuffer _valueBuffer; + public VarLengthValueWriter(ByteBuffer offsetBuffer, int numValues) { + _offsetBuffer = offsetBuffer; + _offsetBuffer.put(MAGIC_BYTES); + _offsetBuffer.putInt(VERSION); + _offsetBuffer.putInt(numValues); + _offsetBuffer.putInt(HEADER_LENGTH); + + _valueBuffer = _offsetBuffer.duplicate(); + _valueBuffer.position(HEADER_LENGTH + (numValues + 1) * Integer.BYTES); + _fileChannel = null; + } + public VarLengthValueWriter(File outputFile, int numValues) throws IOException { _fileChannel = new RandomAccessFile(outputFile, "rw").getChannel(); @@ -124,11 +136,14 @@ public void close() throws IOException { int fileLength = _valueBuffer.position(); _offsetBuffer.putInt(fileLength); - _fileChannel.truncate(fileLength); - _fileChannel.close(); - if (CleanerUtil.UNMAP_SUPPORTED) { - CleanerUtil.BufferCleaner cleaner = CleanerUtil.getCleaner(); - cleaner.freeBuffer(_offsetBuffer); + + if (_fileChannel != null) { + _fileChannel.truncate(fileLength); + _fileChannel.close(); + if (CleanerUtil.UNMAP_SUPPORTED) { + CleanerUtil.BufferCleaner cleaner = CleanerUtil.getCleaner(); + cleaner.freeBuffer(_offsetBuffer); + } } } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java new file mode 100644 index 000000000000..f5763cb2d2fc --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -0,0 +1,183 @@ +/** + * 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.segment.local.io.writer.impl; + +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageEncoder; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import org.apache.pinot.segment.local.io.util.FixedBitIntReaderWriter; +import org.apache.pinot.segment.local.io.util.PinotDataBitSet; +import org.apache.pinot.segment.local.io.util.VarLengthValueWriter; +import org.apache.pinot.segment.spi.creator.ColumnStatistics; +import org.apache.pinot.segment.spi.memory.PinotByteBuffer; + + +/** + * HEADER + * version + * _LOGTYPE_DICTIONARY_OFFSET + * column_LOGTYPE_DICTIONARY buffer + */ + +public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { + // version (int, 4) + logType dict offset (int, 4) + logType fwd index offset (int, 4) + + // dictVar dict offset (int, 4) + dictVar fwd index offset (int, 4) + + private static final int HEADER_SIZE = 20; + private final FileChannel _dataFile; + private final ByteBuffer _header; + private final ByteBuffer _fileBuffer; + private final EncodedMessage _clpEncodedMessage; + private final MessageEncoder _clpMessageEncoder; + private final Set _logTypes = new TreeSet<>(); + private final Set _dictVars = new TreeSet<>(); + private List _logs; + + public CLPForwardIndexWriterV1(File file, int numDocs, ColumnStatistics columnStatistics) + throws IOException { + _dataFile = new RandomAccessFile(file, "rw").getChannel(); + _fileBuffer = _dataFile.map(FileChannel.MapMode.READ_WRITE, 0, Integer.MAX_VALUE); + _header = _fileBuffer.duplicate(); + _header.position(0); + _header.limit(HEADER_SIZE); + + _header.putInt(0); // version + _header.putInt(HEADER_SIZE); // logType dict offset + + _clpEncodedMessage = new EncodedMessage(); + _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + @Override + public void putBigDecimal(BigDecimal value) { + throw new UnsupportedOperationException("String only"); + } + + @Override + public void putString(String value) { + String logtype; + String[] dictVars; + + try { + _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); + logtype = _clpEncodedMessage.getLogTypeAsString(); + dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to encode message: " + value, e); + } + + // TODO: move this to stats collector so we won't need to store the whole logline + // collect logType in set + _logTypes.add(logtype); + // collect dictVars in set + _dictVars.addAll(Arrays.asList(dictVars)); + // collect encodedVars in set + _logs.add(value); + } + + @Override + public void putBytes(byte[] value) { + throw new UnsupportedOperationException("String only"); + } + + @Override + public void putStringMV(String[] values) { + throw new UnsupportedOperationException("String only"); + } + + @Override + public void putBytesMV(byte[][] values) { + throw new UnsupportedOperationException("String only"); + } + + @Override + public void close() + throws IOException { + // Build dictionary for logType + Object2IntOpenHashMap logTypeDict = new Object2IntOpenHashMap<>(_logTypes.size()); + int logTypeDictId = 0; + byte[][] sortedStringBytes = new byte[_logTypes.size()][]; + for (String logType : _logTypes) { + sortedStringBytes[logTypeDictId] = logType.getBytes(StandardCharsets.UTF_8); + logTypeDict.put(logType, logTypeDictId); + logTypeDictId++; + } + + ByteBuffer logTypeDictBuffer = _fileBuffer.duplicate(); + logTypeDictBuffer.position(HEADER_SIZE); + try (VarLengthValueWriter writer = new VarLengthValueWriter(logTypeDictBuffer, _logTypes.size())) { + for (byte[] value : sortedStringBytes) { + writer.add(value); + } + } + _header.putInt(HEADER_SIZE + logTypeDictBuffer.position()); // dictVar dictionary start offset + + // Build dictionary for dictVars + Object2IntOpenHashMap dictVarsDict = new Object2IntOpenHashMap<>(_dictVars.size()); + byte[][] sortedDictIds = new byte[_dictVars.size()][]; + int dictVarsDictId = 0; + for (String dictVar : _dictVars) { + sortedDictIds[dictVarsDictId] = dictVar.getBytes(StandardCharsets.UTF_8); + dictVarsDict.put(dictVar, dictVarsDictId); + dictVarsDictId++; + } + + ByteBuffer dictEncodedDictBuffer = _fileBuffer.duplicate(); + dictEncodedDictBuffer.position(HEADER_SIZE + logTypeDictBuffer.position()); + try (VarLengthValueWriter writer = new VarLengthValueWriter(dictEncodedDictBuffer, _dictVars.size())) { + for (byte[] value : sortedDictIds) { + writer.add(value); + } + } + + _header.putInt(HEADER_SIZE + logTypeDictBuffer.position() + + dictEncodedDictBuffer.position()); // encoded vars index start offset + + // fwd index for logType + ByteBuffer logTypeFwdIndexBuffer = _fileBuffer.duplicate(); + logTypeFwdIndexBuffer.position(HEADER_SIZE + logTypeDictBuffer.position() + dictEncodedDictBuffer.position()); + + FixedBitIntReaderWriter fixedBitIntReaderWriter = + new FixedBitIntReaderWriter(new PinotByteBuffer(logTypeFwdIndexBuffer, true, false), _logs.size(), + PinotDataBitSet.getNumBitsPerValue(_logTypes.size() - 1)); + + // fwd index for dictVars + ByteBuffer dictVarsFwdIndexBuffer = _fileBuffer.duplicate(); + dictVarsFwdIndexBuffer.position(HEADER_SIZE + logTypeDictBuffer.position() + dictEncodedDictBuffer.position() + + PinotDataBitSet.getNumBitsPerValue(_logTypes.size() - 1) * _logs.size()); + FixedBitMVForwardIndexWriter dictVarsFwdIndexWriter = + new FixedBitMVForwardIndexWriter(new PinotByteBuffer(dictVarsFwdIndexBuffer, true, false), _logs.size(), + PinotDataBitSet.getNumBitsPerValue(_dictVars.size() - 1)); + + // Write header + _header.putInt(0, logTypeDict.size()); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java index 9e3658802f57..4bbbb4c9c81b 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java @@ -22,11 +22,13 @@ import java.io.File; import java.io.IOException; import java.math.BigDecimal; +import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkWriter; import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; +import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.index.ForwardIndexConfig; import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator; import org.apache.pinot.spi.data.FieldSpec.DataType; @@ -57,7 +59,7 @@ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType int totalDocs, DataType valueType, int maxLength) throws IOException { this(baseIndexDir, compressionType, column, totalDocs, valueType, maxLength, false, - ForwardIndexConfig.DEFAULT_RAW_WRITER_VERSION); + ForwardIndexConfig.DEFAULT_RAW_WRITER_VERSION, null); } /** @@ -73,14 +75,20 @@ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType * @throws IOException */ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType compressionType, String column, - int totalDocs, DataType valueType, int maxLength, boolean deriveNumDocsPerChunk, int writerVersion) + int totalDocs, DataType valueType, int maxLength, boolean deriveNumDocsPerChunk, int writerVersion, + ColumnStatistics columnStatistics) throws IOException { File file = new File(baseIndexDir, column + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION); - int numDocsPerChunk = deriveNumDocsPerChunk ? getNumDocsPerChunk(maxLength) : DEFAULT_NUM_DOCS_PER_CHUNK; - _indexWriter = writerVersion < VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexWriter(file, - compressionType, totalDocs, numDocsPerChunk, maxLength, writerVersion) - : new VarByteChunkForwardIndexWriterV4(file, compressionType, TARGET_MAX_CHUNK_SIZE); - _valueType = valueType; + if (compressionType == ChunkCompressionType.CLP) { + _indexWriter = new CLPForwardIndexWriterV1(file, totalDocs, columnStatistics); + _valueType = valueType; + } else { + int numDocsPerChunk = deriveNumDocsPerChunk ? getNumDocsPerChunk(maxLength) : DEFAULT_NUM_DOCS_PER_CHUNK; + _indexWriter = writerVersion < VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexWriter(file, + compressionType, totalDocs, numDocsPerChunk, maxLength, writerVersion) + : new VarByteChunkForwardIndexWriterV4(file, compressionType, TARGET_MAX_CHUNK_SIZE); + _valueType = valueType; + } } @VisibleForTesting diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java index 5af1e1499965..41704b9bd420 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java @@ -27,6 +27,7 @@ import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.data.FieldSpec; @@ -44,6 +45,7 @@ public abstract class AbstractColumnStatisticsCollector implements ColumnStatistics { protected static final int INITIAL_HASH_SET_SIZE = 1000; protected final FieldSpec _fieldSpec; + protected final FieldConfig _fieldConfig; private final Map _partitionFunctionConfig; private final PartitionFunction _partitionFunction; @@ -57,6 +59,7 @@ public abstract class AbstractColumnStatisticsCollector implements ColumnStatist public AbstractColumnStatisticsCollector(String column, StatsCollectorConfig statsCollectorConfig) { _fieldSpec = statsCollectorConfig.getFieldSpecForColumn(column); + _fieldConfig = statsCollectorConfig.getFieldConfigForColumn(column); Preconditions.checkArgument(_fieldSpec != null, "Failed to find column: %s", column); if (!_fieldSpec.isSingleValueField()) { _sorted = false; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java index b896ac30ff9c..c7eb85801423 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java @@ -21,21 +21,44 @@ import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; import java.util.Arrays; import java.util.Set; +import org.apache.pinot.segment.local.recordenricher.clp.CLPEncodingEnricher; import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; +import org.apache.pinot.segment.spi.index.ForwardIndexConfig; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.spi.config.table.FieldConfig; import static java.nio.charset.StandardCharsets.UTF_8; public class StringColumnPreIndexStatsCollector extends AbstractColumnStatisticsCollector { + public static class CLPStats { + private String[] _sortedLogTypeValues; + private String[] _sortedDictVarValues; + private final CLPEncodingEnricher + + public void collect(String logLine) { + + } + + public void close() { + + } + } private Set _values = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); private int _minLength = Integer.MAX_VALUE; private int _maxLength = 0; private int _maxRowLength = 0; private String[] _sortedValues; private boolean _sealed = false; + private CLPStats _clpStats; public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { super(column, statsCollectorConfig); + if (_fieldConfig != null) { + if (_fieldConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { + _clpStats = new CLPStats(); + } + } } @Override diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java index ded5000b0c28..503df68dc8ae 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java @@ -31,6 +31,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.compression.DictIdCompressionType; +import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.creator.IndexCreationContext; import org.apache.pinot.segment.spi.index.ForwardIndexConfig; import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator; @@ -77,7 +78,7 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex int writerVersion = indexConfig.getRawIndexWriterVersion(); if (fieldSpec.isSingleValueField()) { return getRawIndexCreatorForSVColumn(indexDir, chunkCompressionType, columnName, storedType, numTotalDocs, - context.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion); + context.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion, context.getColumnStatistics()); } else { return getRawIndexCreatorForMVColumn(indexDir, chunkCompressionType, columnName, storedType, numTotalDocs, context.getMaxNumberOfMultiValueElements(), deriveNumDocsPerChunk, writerVersion, @@ -92,7 +93,7 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex */ public static ForwardIndexCreator getRawIndexCreatorForSVColumn(File indexDir, ChunkCompressionType compressionType, String column, DataType storedType, int numTotalDocs, int lengthOfLongestEntry, boolean deriveNumDocsPerChunk, - int writerVersion) + int writerVersion, ColumnStatistics columnStatistics) throws IOException { switch (storedType) { case INT: @@ -105,7 +106,7 @@ public static ForwardIndexCreator getRawIndexCreatorForSVColumn(File indexDir, C case STRING: case BYTES: return new SingleValueVarByteRawIndexCreator(indexDir, compressionType, column, numTotalDocs, storedType, - lengthOfLongestEntry, deriveNumDocsPerChunk, writerVersion); + lengthOfLongestEntry, deriveNumDocsPerChunk, writerVersion, columnStatistics); default: throw new IllegalStateException("Unsupported stored type: " + storedType); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/compression/ChunkCompressionType.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/compression/ChunkCompressionType.java index 97d7057d03c7..c10fc0de3aa0 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/compression/ChunkCompressionType.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/compression/ChunkCompressionType.java @@ -19,7 +19,7 @@ package org.apache.pinot.segment.spi.compression; public enum ChunkCompressionType { - PASS_THROUGH(0), SNAPPY(1), ZSTANDARD(2), LZ4(3), LZ4_LENGTH_PREFIXED(4); + PASS_THROUGH(0), SNAPPY(1), ZSTANDARD(2), LZ4(3), LZ4_LENGTH_PREFIXED(4), CLP(5); private static final ChunkCompressionType[] VALUES = values(); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/ColumnIndexCreationInfo.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/ColumnIndexCreationInfo.java index af0232d661bc..9e47502098ef 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/ColumnIndexCreationInfo.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/ColumnIndexCreationInfo.java @@ -43,6 +43,10 @@ public ColumnIndexCreationInfo(ColumnStatistics columnStatistics, boolean create _defaultNullValue = defaultNullValue; } + public ColumnStatistics getColumnStatistics() { + return _columnStatistics; + } + public boolean isCreateDictionary() { return _createDictionary; } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java index 5f4dd2cd66a5..2ddfe5499d50 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java @@ -22,12 +22,14 @@ import java.util.Objects; import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.spi.config.table.IndexConfig; import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.data.FieldSpec; /** - * Provides parameters for constructing indexes via {@link IndexType#createIndexCreator(IndexCreationContext, Object)}. + * Provides parameters for constructing indexes via + * {@link IndexType#createIndexCreator(IndexCreationContext, IndexConfig)}. * The responsibility for ensuring that the correct parameters for a particular * index type lies with the caller. */ @@ -88,7 +90,10 @@ public interface IndexCreationContext { */ boolean isTextCommitOnClose(); + ColumnStatistics getColumnStatistics(); + final class Builder { + private ColumnStatistics _columnStatistics; private File _indexDir; private int _lengthOfLongestEntry; private int _maxNumberOfMultiValueElements; @@ -109,16 +114,22 @@ final class Builder { private boolean _textCommitOnClose; public Builder withColumnIndexCreationInfo(ColumnIndexCreationInfo columnIndexCreationInfo) { - return withLengthOfLongestEntry(columnIndexCreationInfo.getLengthOfLongestEntry()) - .withMaxNumberOfMultiValueElements(columnIndexCreationInfo.getMaxNumberOfMultiValueElements()) + return withLengthOfLongestEntry( + columnIndexCreationInfo.getLengthOfLongestEntry()).withMaxNumberOfMultiValueElements( + columnIndexCreationInfo.getMaxNumberOfMultiValueElements()) .withMaxRowLengthInBytes(columnIndexCreationInfo.getMaxRowLengthInBytes()) .withMinValue((Comparable) columnIndexCreationInfo.getMin()) .withMaxValue((Comparable) columnIndexCreationInfo.getMax()) .withTotalNumberOfEntries(columnIndexCreationInfo.getTotalNumberOfEntries()) .withSortedUniqueElementsArray(columnIndexCreationInfo.getSortedUniqueElementsArray()) + .withColumnStatistics(columnIndexCreationInfo.getColumnStatistics()) .withCardinality(columnIndexCreationInfo.getDistinctValueCount()) - .withFixedLength(columnIndexCreationInfo.isFixedLength()) - .sorted(columnIndexCreationInfo.isSorted()); + .withFixedLength(columnIndexCreationInfo.isFixedLength()).sorted(columnIndexCreationInfo.isSorted()); + } + + public Builder withColumnStatistics(ColumnStatistics columnStatistics) { + _columnStatistics = columnStatistics; + return this; } public Builder withIndexDir(File indexDir) { @@ -222,7 +233,7 @@ public Common build() { return new Common(Objects.requireNonNull(_indexDir), _lengthOfLongestEntry, _maxNumberOfMultiValueElements, _maxRowLengthInBytes, _onHeap, Objects.requireNonNull(_fieldSpec), _sorted, _cardinality, _totalNumberOfEntries, _totalDocs, _hasDictionary, _minValue, _maxValue, _forwardIndexDisabled, - _sortedUniqueElementsArray, _optimizedDictionary, _fixedLength, _textCommitOnClose); + _sortedUniqueElementsArray, _optimizedDictionary, _fixedLength, _textCommitOnClose, _columnStatistics); } public Builder withSortedUniqueElementsArray(Object sortedUniqueElementsArray) { @@ -255,13 +266,13 @@ final class Common implements IndexCreationContext { private final boolean _optimizeDictionary; private final boolean _fixedLength; private final boolean _textCommitOnClose; + private final ColumnStatistics _columnStatistics; - public Common(File indexDir, int lengthOfLongestEntry, - int maxNumberOfMultiValueElements, int maxRowLengthInBytes, boolean onHeap, - FieldSpec fieldSpec, boolean sorted, int cardinality, int totalNumberOfEntries, - int totalDocs, boolean hasDictionary, Comparable minValue, Comparable maxValue, - boolean forwardIndexDisabled, Object sortedUniqueElementsArray, boolean optimizeDictionary, - boolean fixedLength, boolean textCommitOnClose) { + public Common(File indexDir, int lengthOfLongestEntry, int maxNumberOfMultiValueElements, int maxRowLengthInBytes, + boolean onHeap, FieldSpec fieldSpec, boolean sorted, int cardinality, int totalNumberOfEntries, int totalDocs, + boolean hasDictionary, Comparable minValue, Comparable maxValue, boolean forwardIndexDisabled, + Object sortedUniqueElementsArray, boolean optimizeDictionary, boolean fixedLength, boolean textCommitOnClose, + ColumnStatistics columnStatistics) { _indexDir = indexDir; _lengthOfLongestEntry = lengthOfLongestEntry; _maxNumberOfMultiValueElements = maxNumberOfMultiValueElements; @@ -280,6 +291,7 @@ public Common(File indexDir, int lengthOfLongestEntry, _optimizeDictionary = optimizeDictionary; _fixedLength = fixedLength; _textCommitOnClose = textCommitOnClose; + _columnStatistics = columnStatistics; } public FieldSpec getFieldSpec() { @@ -360,5 +372,10 @@ public boolean isFixedLength() { public boolean isTextCommitOnClose() { return _textCommitOnClose; } + + @Override + public ColumnStatistics getColumnStatistics() { + return _columnStatistics; + } } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java index f868eea15a41..f8c48650dbfe 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java @@ -19,8 +19,10 @@ package org.apache.pinot.segment.spi.creator; import com.google.common.base.Preconditions; +import java.util.HashMap; import java.util.Map; import javax.annotation.Nullable; +import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.SegmentPartitionConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.FieldSpec; @@ -36,6 +38,7 @@ public class StatsCollectorConfig { private final TableConfig _tableConfig; private final Schema _schema; private final SegmentPartitionConfig _segmentPartitionConfig; + private final Map _columnFieldConfigMap; /** * Constructor for the class. @@ -49,6 +52,13 @@ public StatsCollectorConfig(TableConfig tableConfig, Schema schema, _tableConfig = tableConfig; _schema = schema; _segmentPartitionConfig = segmentPartitionConfig; + _columnFieldConfigMap = new HashMap<>(); + if (tableConfig.getFieldConfigList() != null) { + for (FieldConfig fieldConfig : tableConfig.getFieldConfigList()) { + _columnFieldConfigMap.put(fieldConfig.getName(), fieldConfig); + } + } + } @Nullable @@ -89,4 +99,9 @@ public Schema getSchema() { public TableConfig getTableConfig() { return _tableConfig; } + + @Nullable + public FieldConfig getFieldConfigForColumn(String column) { + return _columnFieldConfigMap.get(column); + } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java index fcdbbe4fe0af..b56d91170903 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java @@ -156,6 +156,9 @@ public Builder withCompressionCodec(CompressionCodec compressionCodec) { case LZ4: _chunkCompressionType = ChunkCompressionType.LZ4; break; + case CLP: + _chunkCompressionType = ChunkCompressionType.CLP; + break; case MV_ENTRY_DICT: _dictIdCompressionType = DictIdCompressionType.MV_ENTRY_DICT; break; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java index 1cb357ea442b..aeee3ff98a58 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java @@ -60,7 +60,7 @@ static PinotByteBuffer mapFile(File file, boolean readOnly, long offset, int siz } } - private PinotByteBuffer(ByteBuffer buffer, boolean closeable, boolean flushable) { + public PinotByteBuffer(ByteBuffer buffer, boolean closeable, boolean flushable) { super(closeable); _buffer = buffer; _flushable = flushable; diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java index e1b554e54a26..070a9550dc13 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java @@ -126,6 +126,7 @@ public enum CompressionCodec { SNAPPY(true, false), ZSTANDARD(true, false), LZ4(true, false), + CLP(true, false), // For MV dictionary encoded forward index, add a second level dictionary encoding for the multi-value entries MV_ENTRY_DICT(false, true); diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java b/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java index 7be99a0392f4..a08d00f313e5 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java @@ -318,7 +318,7 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme try (ForwardIndexCreator rawIndexCreator = ForwardIndexCreatorFactory .getRawIndexCreatorForSVColumn(newSegment, compressionType, column, storedType, numDocs, lengthOfLongestEntry, - false, ForwardIndexConfig.DEFAULT_RAW_WRITER_VERSION); + false, ForwardIndexConfig.DEFAULT_RAW_WRITER_VERSION, null); ForwardIndexReaderContext readerContext = forwardIndexReader.createContext()) { switch (storedType) { case INT: From c408cde2ce248478ac1c4ea58ee037f90909a30a Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Thu, 11 Jan 2024 13:24:00 +0530 Subject: [PATCH 08/30] Finish the flow to add CLP index --- .../writer/impl/CLPForwardIndexWriterV1.java | 148 ++++++++---------- .../impl/FixedBitMVForwardIndexWriter.java | 2 +- .../impl/FixedBitSVForwardIndexWriter.java | 2 +- .../SingleValueVarByteRawIndexCreator.java | 2 +- .../StringColumnPreIndexStatsCollector.java | 114 +++++++++++--- .../segment/creator/DictionariesTest.java | 21 +++ 6 files changed, 176 insertions(+), 113 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index f5763cb2d2fc..d6f0c3dd3cfb 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -21,23 +21,20 @@ import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; import com.yscope.clp.compressorfrontend.EncodedMessage; import com.yscope.clp.compressorfrontend.MessageEncoder; -import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; -import org.apache.pinot.segment.local.io.util.FixedBitIntReaderWriter; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; -import org.apache.pinot.segment.local.io.util.VarLengthValueWriter; +import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator; +import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator; +import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; +import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.creator.ColumnStatistics; -import org.apache.pinot.segment.spi.memory.PinotByteBuffer; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; /** @@ -50,26 +47,51 @@ public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { // version (int, 4) + logType dict offset (int, 4) + logType fwd index offset (int, 4) + // dictVar dict offset (int, 4) + dictVar fwd index offset (int, 4) + - private static final int HEADER_SIZE = 20; + private final String _column; + private final File _baseIndexDir; private final FileChannel _dataFile; - private final ByteBuffer _header; private final ByteBuffer _fileBuffer; private final EncodedMessage _clpEncodedMessage; private final MessageEncoder _clpMessageEncoder; - private final Set _logTypes = new TreeSet<>(); - private final Set _dictVars = new TreeSet<>(); - private List _logs; - - public CLPForwardIndexWriterV1(File file, int numDocs, ColumnStatistics columnStatistics) + private final StringColumnPreIndexStatsCollector.CLPStats _clpStats; + private final SegmentDictionaryCreator _logTypeDictCreator; + private final SegmentDictionaryCreator _dictVarsDictCreator; + private final FixedBitSVForwardIndexWriter _logTypeFwdIndexWriter; + private final FixedBitMVForwardIndexWriter _dictVarsFwdIndexWriter; + private final MultiValueFixedByteRawIndexCreator _encodedVarsFwdIndexWriter; + + public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, int numDocs, + ColumnStatistics columnStatistics) throws IOException { - _dataFile = new RandomAccessFile(file, "rw").getChannel(); + _column = column; + _baseIndexDir = baseIndexDir; + _dataFile = new RandomAccessFile(indexFile, "rw").getChannel(); _fileBuffer = _dataFile.map(FileChannel.MapMode.READ_WRITE, 0, Integer.MAX_VALUE); - _header = _fileBuffer.duplicate(); - _header.position(0); - _header.limit(HEADER_SIZE); - _header.putInt(0); // version - _header.putInt(HEADER_SIZE); // logType dict offset + StringColumnPreIndexStatsCollector statsCollector = (StringColumnPreIndexStatsCollector) columnStatistics; + _clpStats = statsCollector.getClpStats(); + _logTypeDictCreator = new SegmentDictionaryCreator( + new DimensionFieldSpec(_column + "_clp_logtype.dict", FieldSpec.DataType.STRING, true), _baseIndexDir, true); + _logTypeDictCreator.build(_clpStats.getSortedLogTypeValues()); + + _dictVarsDictCreator = new SegmentDictionaryCreator( + new DimensionFieldSpec(_column + "_clp_dictvars.dict", FieldSpec.DataType.STRING, false), _baseIndexDir, true); + _dictVarsDictCreator.build(_clpStats.getSortedDictVarValues()); + + File logTypeFwdIndexFile = new File(_baseIndexDir, column + "_clp_logtype.fwd"); + _logTypeFwdIndexWriter = new FixedBitSVForwardIndexWriter(logTypeFwdIndexFile, numDocs, + PinotDataBitSet.getNumBitsPerValue(_clpStats.getSortedLogTypeValues().length - 1)); + + File dictVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_dictvars.fwd"); + _dictVarsFwdIndexWriter = + new FixedBitMVForwardIndexWriter(dictVarsFwdIndexFile, numDocs, _clpStats.getTotalNumberOfDictVars(), + PinotDataBitSet.getNumBitsPerValue(_clpStats.getSortedDictVarValues().length - 1)); + + _encodedVarsFwdIndexWriter = + new MultiValueFixedByteRawIndexCreator(_baseIndexDir, ChunkCompressionType.PASS_THROUGH, + column + "_clp_encodedvars.fwd", numDocs, FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), + false, -1); + _clpStats.clear(); _clpEncodedMessage = new EncodedMessage(); _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, @@ -85,22 +107,32 @@ public void putBigDecimal(BigDecimal value) { public void putString(String value) { String logtype; String[] dictVars; + Long[] encodedVars; try { _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); logtype = _clpEncodedMessage.getLogTypeAsString(); dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + encodedVars = _clpEncodedMessage.getEncodedVarsAsBoxedLongs(); } catch (IOException e) { throw new IllegalArgumentException("Failed to encode message: " + value, e); } - // TODO: move this to stats collector so we won't need to store the whole logline - // collect logType in set - _logTypes.add(logtype); - // collect dictVars in set - _dictVars.addAll(Arrays.asList(dictVars)); - // collect encodedVars in set - _logs.add(value); + addCLPFields(logtype, dictVars, encodedVars); + } + + private void addCLPFields(String logtype, String[] dictVars, Long[] encodedVars) { + int logTypeDictId = _logTypeDictCreator.indexOfSV(logtype); + int[] dictVarDictIds = _dictVarsDictCreator.indexOfMV(dictVars); + + _logTypeFwdIndexWriter.putDictId(logTypeDictId); + _dictVarsFwdIndexWriter.putDictIds(dictVarDictIds); + + long[] encodedVarsUnboxed = new long[encodedVars.length]; + for (int i = 0; i < encodedVars.length; i++) { + encodedVarsUnboxed[i] = encodedVars[i].longValue(); + } + _encodedVarsFwdIndexWriter.putLongMV(encodedVarsUnboxed); } @Override @@ -121,63 +153,5 @@ public void putBytesMV(byte[][] values) { @Override public void close() throws IOException { - // Build dictionary for logType - Object2IntOpenHashMap logTypeDict = new Object2IntOpenHashMap<>(_logTypes.size()); - int logTypeDictId = 0; - byte[][] sortedStringBytes = new byte[_logTypes.size()][]; - for (String logType : _logTypes) { - sortedStringBytes[logTypeDictId] = logType.getBytes(StandardCharsets.UTF_8); - logTypeDict.put(logType, logTypeDictId); - logTypeDictId++; - } - - ByteBuffer logTypeDictBuffer = _fileBuffer.duplicate(); - logTypeDictBuffer.position(HEADER_SIZE); - try (VarLengthValueWriter writer = new VarLengthValueWriter(logTypeDictBuffer, _logTypes.size())) { - for (byte[] value : sortedStringBytes) { - writer.add(value); - } - } - _header.putInt(HEADER_SIZE + logTypeDictBuffer.position()); // dictVar dictionary start offset - - // Build dictionary for dictVars - Object2IntOpenHashMap dictVarsDict = new Object2IntOpenHashMap<>(_dictVars.size()); - byte[][] sortedDictIds = new byte[_dictVars.size()][]; - int dictVarsDictId = 0; - for (String dictVar : _dictVars) { - sortedDictIds[dictVarsDictId] = dictVar.getBytes(StandardCharsets.UTF_8); - dictVarsDict.put(dictVar, dictVarsDictId); - dictVarsDictId++; - } - - ByteBuffer dictEncodedDictBuffer = _fileBuffer.duplicate(); - dictEncodedDictBuffer.position(HEADER_SIZE + logTypeDictBuffer.position()); - try (VarLengthValueWriter writer = new VarLengthValueWriter(dictEncodedDictBuffer, _dictVars.size())) { - for (byte[] value : sortedDictIds) { - writer.add(value); - } - } - - _header.putInt(HEADER_SIZE + logTypeDictBuffer.position() - + dictEncodedDictBuffer.position()); // encoded vars index start offset - - // fwd index for logType - ByteBuffer logTypeFwdIndexBuffer = _fileBuffer.duplicate(); - logTypeFwdIndexBuffer.position(HEADER_SIZE + logTypeDictBuffer.position() + dictEncodedDictBuffer.position()); - - FixedBitIntReaderWriter fixedBitIntReaderWriter = - new FixedBitIntReaderWriter(new PinotByteBuffer(logTypeFwdIndexBuffer, true, false), _logs.size(), - PinotDataBitSet.getNumBitsPerValue(_logTypes.size() - 1)); - - // fwd index for dictVars - ByteBuffer dictVarsFwdIndexBuffer = _fileBuffer.duplicate(); - dictVarsFwdIndexBuffer.position(HEADER_SIZE + logTypeDictBuffer.position() + dictEncodedDictBuffer.position() - + PinotDataBitSet.getNumBitsPerValue(_logTypes.size() - 1) * _logs.size()); - FixedBitMVForwardIndexWriter dictVarsFwdIndexWriter = - new FixedBitMVForwardIndexWriter(new PinotByteBuffer(dictVarsFwdIndexBuffer, true, false), _logs.size(), - PinotDataBitSet.getNumBitsPerValue(_dictVars.size() - 1)); - - // Write header - _header.putInt(0, logTypeDict.size()); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitMVForwardIndexWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitMVForwardIndexWriter.java index 650a75e28ff3..fd46a843ee22 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitMVForwardIndexWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitMVForwardIndexWriter.java @@ -75,7 +75,7 @@ public class FixedBitMVForwardIndexWriter implements Closeable { private int _nextDocId = 0; public FixedBitMVForwardIndexWriter(File file, int numDocs, int totalNumValues, int numBitsPerValue) - throws Exception { + throws IOException { float averageValuesPerDoc = totalNumValues / numDocs; _docsPerChunk = (int) (Math.ceil(PREFERRED_NUM_VALUES_PER_CHUNK / averageValuesPerDoc)); _numChunks = (numDocs + _docsPerChunk - 1) / _docsPerChunk; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitSVForwardIndexWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitSVForwardIndexWriter.java index b7c8c55c5b2e..f6712b38bfcc 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitSVForwardIndexWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/FixedBitSVForwardIndexWriter.java @@ -37,7 +37,7 @@ public class FixedBitSVForwardIndexWriter implements Closeable { private int _nextDocId = 0; public FixedBitSVForwardIndexWriter(File file, int numDocs, int numBitsPerValue) - throws Exception { + throws IOException { // Convert to long in order to avoid int overflow long length = ((long) numDocs * numBitsPerValue + Byte.SIZE - 1) / Byte.SIZE; // Backward-compatible: index file is always big-endian diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java index 4bbbb4c9c81b..0bc9655d84d5 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java @@ -80,7 +80,7 @@ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType throws IOException { File file = new File(baseIndexDir, column + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION); if (compressionType == ChunkCompressionType.CLP) { - _indexWriter = new CLPForwardIndexWriterV1(file, totalDocs, columnStatistics); + _indexWriter = new CLPForwardIndexWriterV1(baseIndexDir, file, column, totalDocs, columnStatistics); _valueType = valueType; } else { int numDocsPerChunk = deriveNumDocsPerChunk ? getNumDocsPerChunk(maxLength) : DEFAULT_NUM_DOCS_PER_CHUNK; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java index c7eb85801423..3c44d7eb6512 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java @@ -18,30 +18,24 @@ */ package org.apache.pinot.segment.local.segment.creator.impl.stats; +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageEncoder; import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; +import java.io.IOException; import java.util.Arrays; import java.util.Set; -import org.apache.pinot.segment.local.recordenricher.clp.CLPEncodingEnricher; import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; -import org.apache.pinot.segment.spi.index.ForwardIndexConfig; -import org.apache.pinot.segment.spi.index.StandardIndexes; import org.apache.pinot.spi.config.table.FieldConfig; import static java.nio.charset.StandardCharsets.UTF_8; public class StringColumnPreIndexStatsCollector extends AbstractColumnStatisticsCollector { - public static class CLPStats { - private String[] _sortedLogTypeValues; - private String[] _sortedDictVarValues; - private final CLPEncodingEnricher - - public void collect(String logLine) { - - } - - public void close() { - + public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { + super(column, statsCollectorConfig); + if (_fieldConfig != null && _fieldConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { + _clpStats = new CLPStats(); } } private Set _values = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); @@ -52,15 +46,6 @@ public void close() { private boolean _sealed = false; private CLPStats _clpStats; - public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { - super(column, statsCollectorConfig); - if (_fieldConfig != null) { - if (_fieldConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { - _clpStats = new CLPStats(); - } - } - } - @Override public void collect(Object entry) { assert !_sealed; @@ -71,6 +56,9 @@ public void collect(Object entry) { for (Object obj : values) { String value = (String) obj; _values.add(value); + if (_clpStats != null) { + _clpStats.collect(value); + } int length = value.getBytes(UTF_8).length; _minLength = Math.min(_minLength, length); @@ -84,6 +72,9 @@ public void collect(Object entry) { } else { String value = (String) entry; addressSorted(value); + if (_clpStats != null) { + _clpStats.collect(value); + } if (_values.add(value)) { if (isPartitionEnabled()) { updatePartition(value); @@ -97,6 +88,10 @@ public void collect(Object entry) { } } + public CLPStats getClpStats() { + return _clpStats; + } + @Override public String getMinValue() { if (_sealed) { @@ -148,7 +143,80 @@ public void seal() { _sortedValues = _values.toArray(new String[0]); _values = null; Arrays.sort(_sortedValues); + if (_clpStats != null) { + _clpStats.seal(); + } _sealed = true; } } + + public static class CLPStats { + int _totalNumberOfDictVars = 0; + private String[] _sortedLogTypeValues; + int _totalNumberOfEncodedVars = 0; + private String[] _sortedDictVarValues; + + public int getMaxNumberOfEncodedVars() { + return _maxNumberOfEncodedVars; + } + + int _maxNumberOfEncodedVars = 0; + private Set _logTypes = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); + private Set _dictVars = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); + private final EncodedMessage _clpEncodedMessage; + private final MessageEncoder _clpMessageEncoder; + public CLPStats() { + _clpEncodedMessage = new EncodedMessage(); + _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + public int getTotalNumberOfDictVars() { + return _totalNumberOfDictVars; + } + + public int getTotalNumberOfEncodedVars() { + return _totalNumberOfEncodedVars; + } + + public void collect(String value) { + String logType; + String[] dictVars; + try { + _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); + logType = _clpEncodedMessage.getLogTypeAsString(); + dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to encode message: " + value, e); + } + _logTypes.add(logType); + _dictVars.addAll(Arrays.asList(dictVars)); + _totalNumberOfDictVars += dictVars.length; + _totalNumberOfEncodedVars += _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length; + _maxNumberOfEncodedVars = + Math.max(_maxNumberOfEncodedVars, _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length); + } + + public void seal() { + _sortedLogTypeValues = _logTypes.toArray(new String[0]); + _logTypes = null; + Arrays.sort(_sortedLogTypeValues); + _sortedDictVarValues = _dictVars.toArray(new String[0]); + _dictVars = null; + Arrays.sort(_sortedDictVarValues); + } + + public void clear() { + _sortedLogTypeValues = null; + _sortedDictVarValues = null; + } + + public String[] getSortedLogTypeValues() { + return _sortedLogTypeValues; + } + + public String[] getSortedDictVarValues() { + return _sortedDictVarValues; + } + } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java index 69bc85551c75..5dccfcde5554 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java @@ -20,9 +20,12 @@ import java.io.File; import java.math.BigDecimal; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -54,6 +57,7 @@ import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; import org.apache.pinot.segment.spi.index.reader.Dictionary; +import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.DimensionFieldSpec; import org.apache.pinot.spi.data.FieldSpec; @@ -530,4 +534,21 @@ private AbstractColumnStatisticsCollector buildStatsCollector(String column, Dat throw new IllegalArgumentException("Illegal data type for stats builder: " + dataType); } } + + @Test + public void clpTest() { + Schema schema = new Schema(); + schema.addField(new DimensionFieldSpec("column1", DataType.STRING, true)); + List fieldConfigList = new ArrayList<>(); + fieldConfigList.add(new FieldConfig("column1", FieldConfig.EncodingType.RAW, Collections.EMPTY_LIST, + FieldConfig.CompressionCodec.CLP, Collections.EMPTY_MAP)); + _tableConfig.setFieldConfigList(fieldConfigList); + StatsCollectorConfig statsCollectorConfig = new StatsCollectorConfig(_tableConfig, schema, null); + StringColumnPreIndexStatsCollector statsCollector = new StringColumnPreIndexStatsCollector("column1", statsCollectorConfig); + statsCollector.collect("2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); + statsCollector.collect("2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + + statsCollector.seal(); + System.out.println(statsCollector.getClpStats()); + } } From 97977917283ce6e45c0463eeabd715a3afdbda05 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Thu, 11 Jan 2024 17:23:01 +0530 Subject: [PATCH 09/30] Add reader --- .../writer/impl/CLPForwardIndexWriterV1.java | 78 +++++++++-- .../impl/SegmentDictionaryCreator.java | 7 + .../MultiValueFixedByteRawIndexCreator.java | 15 +- .../forward/ForwardIndexReaderFactory.java | 12 ++ .../forward/CLPForwardIndexReaderV1.java | 131 ++++++++++++++++++ .../segment/index/creator/CLPWriterTest.java | 79 +++++++++++ 6 files changed, 304 insertions(+), 18 deletions(-) create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java create mode 100644 pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index d6f0c3dd3cfb..9b4f32ff83e9 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -27,16 +27,16 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import org.apache.commons.io.FileUtils; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator; import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator; import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.creator.ColumnStatistics; -import org.apache.pinot.spi.data.DimensionFieldSpec; import org.apache.pinot.spi.data.FieldSpec; - /** * HEADER * version @@ -47,7 +47,9 @@ public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { // version (int, 4) + logType dict offset (int, 4) + logType fwd index offset (int, 4) + // dictVar dict offset (int, 4) + dictVar fwd index offset (int, 4) + + public static final byte[] MAGIC_BYTES = "CLP1".getBytes(StandardCharsets.UTF_8); private final String _column; + private final int _numDocs; private final File _baseIndexDir; private final FileChannel _dataFile; private final ByteBuffer _fileBuffer; @@ -59,38 +61,47 @@ public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { private final FixedBitSVForwardIndexWriter _logTypeFwdIndexWriter; private final FixedBitMVForwardIndexWriter _dictVarsFwdIndexWriter; private final MultiValueFixedByteRawIndexCreator _encodedVarsFwdIndexWriter; + private final File _logTypeDictFile; + private final File _dictVarsDictFile; + private final File _logTypeFwdIndexFile; + private final File _dictVarsFwdIndexFile; + private final File _encodedVarsFwdIndexFile; public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, int numDocs, ColumnStatistics columnStatistics) throws IOException { _column = column; + _numDocs = numDocs; _baseIndexDir = baseIndexDir; _dataFile = new RandomAccessFile(indexFile, "rw").getChannel(); _fileBuffer = _dataFile.map(FileChannel.MapMode.READ_WRITE, 0, Integer.MAX_VALUE); StringColumnPreIndexStatsCollector statsCollector = (StringColumnPreIndexStatsCollector) columnStatistics; _clpStats = statsCollector.getClpStats(); - _logTypeDictCreator = new SegmentDictionaryCreator( - new DimensionFieldSpec(_column + "_clp_logtype.dict", FieldSpec.DataType.STRING, true), _baseIndexDir, true); + _logTypeDictFile = new File(_baseIndexDir, _column + "_clp_logtype.dict"); + _logTypeDictCreator = + new SegmentDictionaryCreator(_column + "_clp_logtype.dict", FieldSpec.DataType.STRING, _logTypeDictFile, true); _logTypeDictCreator.build(_clpStats.getSortedLogTypeValues()); - _dictVarsDictCreator = new SegmentDictionaryCreator( - new DimensionFieldSpec(_column + "_clp_dictvars.dict", FieldSpec.DataType.STRING, false), _baseIndexDir, true); + _dictVarsDictFile = new File(_baseIndexDir, _column + "_clp_dictvars.dict"); + _dictVarsDictCreator = + new SegmentDictionaryCreator(_column + "_clp_dictvars.dict", FieldSpec.DataType.STRING, _dictVarsDictFile, + true); _dictVarsDictCreator.build(_clpStats.getSortedDictVarValues()); - File logTypeFwdIndexFile = new File(_baseIndexDir, column + "_clp_logtype.fwd"); - _logTypeFwdIndexWriter = new FixedBitSVForwardIndexWriter(logTypeFwdIndexFile, numDocs, + _logTypeFwdIndexFile = new File(_baseIndexDir, column + "_clp_logtype.fwd"); + _logTypeFwdIndexWriter = new FixedBitSVForwardIndexWriter(_logTypeFwdIndexFile, numDocs, PinotDataBitSet.getNumBitsPerValue(_clpStats.getSortedLogTypeValues().length - 1)); - File dictVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_dictvars.fwd"); + _dictVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_dictvars.fwd"); _dictVarsFwdIndexWriter = - new FixedBitMVForwardIndexWriter(dictVarsFwdIndexFile, numDocs, _clpStats.getTotalNumberOfDictVars(), + new FixedBitMVForwardIndexWriter(_dictVarsFwdIndexFile, numDocs, _clpStats.getTotalNumberOfDictVars(), PinotDataBitSet.getNumBitsPerValue(_clpStats.getSortedDictVarValues().length - 1)); + _encodedVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_encodedvars.fwd"); _encodedVarsFwdIndexWriter = - new MultiValueFixedByteRawIndexCreator(_baseIndexDir, ChunkCompressionType.PASS_THROUGH, - column + "_clp_encodedvars.fwd", numDocs, FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), - false, -1); + new MultiValueFixedByteRawIndexCreator(_encodedVarsFwdIndexFile, ChunkCompressionType.PASS_THROUGH, numDocs, + FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, 2); _clpStats.clear(); _clpEncodedMessage = new EncodedMessage(); @@ -130,7 +141,7 @@ private void addCLPFields(String logtype, String[] dictVars, Long[] encodedVars) long[] encodedVarsUnboxed = new long[encodedVars.length]; for (int i = 0; i < encodedVars.length; i++) { - encodedVarsUnboxed[i] = encodedVars[i].longValue(); + encodedVarsUnboxed[i] = encodedVars[i]; } _encodedVarsFwdIndexWriter.putLongMV(encodedVarsUnboxed); } @@ -153,5 +164,44 @@ public void putBytesMV(byte[][] values) { @Override public void close() throws IOException { + // Append all of these into fileBuffer + _logTypeDictCreator.seal(); + _logTypeDictCreator.close(); + + _dictVarsDictCreator.seal(); + _dictVarsDictCreator.close(); + + _logTypeFwdIndexWriter.close(); + _dictVarsFwdIndexWriter.close(); + _encodedVarsFwdIndexWriter.close(); + + int totalSize = MAGIC_BYTES.length + 10 * 4 + (int) _logTypeDictFile.length() + (int) _dictVarsDictFile.length() + + (int) _logTypeFwdIndexFile.length() + (int) _dictVarsFwdIndexFile.length() + + (int) _encodedVarsFwdIndexFile.length(); + + _fileBuffer.put(MAGIC_BYTES); + _fileBuffer.putInt(1); // version + _fileBuffer.putInt((int) _numDocs); // logType fwd index length + _fileBuffer.putInt(_clpStats.getTotalNumberOfEncodedVars()); + _fileBuffer.putInt((int) _logTypeDictFile.length()); // logType dict length + _fileBuffer.putInt((int) _dictVarsDictFile.length()); // dictVars dict length + _fileBuffer.putInt((int) _logTypeFwdIndexFile.length()); // logType fwd index length + _fileBuffer.putInt((int) _dictVarsFwdIndexFile.length()); // dictVars fwd index length + _fileBuffer.putInt((int) _encodedVarsFwdIndexFile.length()); // encodedVars fwd index length + + _fileBuffer.put(FileUtils.readFileToByteArray(_logTypeDictFile)); + _fileBuffer.put(FileUtils.readFileToByteArray(_dictVarsDictFile)); + _fileBuffer.put(FileUtils.readFileToByteArray(_logTypeFwdIndexFile)); + _fileBuffer.put(FileUtils.readFileToByteArray(_dictVarsFwdIndexFile)); + _fileBuffer.put(FileUtils.readFileToByteArray(_encodedVarsFwdIndexFile)); + + _dataFile.truncate(totalSize); + + // Delete all temp files + FileUtils.deleteQuietly(_logTypeDictFile); + FileUtils.deleteQuietly(_dictVarsDictFile); + FileUtils.deleteQuietly(_logTypeFwdIndexFile); + FileUtils.deleteQuietly(_dictVarsFwdIndexFile); + FileUtils.deleteQuietly(_encodedVarsFwdIndexFile); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java index 96fc96cee423..c78cd25d0725 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java @@ -70,6 +70,13 @@ public class SegmentDictionaryCreator implements IndexCreator { private Object2IntOpenHashMap _objectValueToIndexMap; private int _numBytesPerEntry = 0; + public SegmentDictionaryCreator(String columnName, DataType storedType, File indexFile, boolean useVarLengthDictionary) { + _columnName = columnName; + _storedType = storedType; + _dictionaryFile = indexFile; + _useVarLengthDictionary = useVarLengthDictionary; + } + public SegmentDictionaryCreator(FieldSpec fieldSpec, File indexDir, boolean useVarLengthDictionary) { _columnName = fieldSpec.getName(); _storedType = fieldSpec.getDataType().getStoredType(); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java index a1ba59f3b2ae..d459db1f6f27 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java @@ -73,15 +73,22 @@ public MultiValueFixedByteRawIndexCreator(File baseIndexDir, ChunkCompressionTyp int totalDocs, DataType valueType, int maxNumberOfMultiValueElements, boolean deriveNumDocsPerChunk, int writerVersion) throws IOException { - File file = new File(baseIndexDir, column + Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION); + this(new File(baseIndexDir, column + Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION), compressionType, totalDocs, + valueType, maxNumberOfMultiValueElements, deriveNumDocsPerChunk, writerVersion); + } + + public MultiValueFixedByteRawIndexCreator(File indexFile, ChunkCompressionType compressionType, int totalDocs, + DataType valueType, int maxNumberOfMultiValueElements, boolean deriveNumDocsPerChunk, int writerVersion) + throws IOException { // Store the length followed by the values int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size()); int numDocsPerChunk = deriveNumDocsPerChunk ? Math.max( TARGET_MAX_CHUNK_SIZE / (totalMaxLength + VarByteChunkForwardIndexWriter.CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE), 1) : DEFAULT_NUM_DOCS_PER_CHUNK; - _indexWriter = writerVersion < VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexWriter(file, - compressionType, totalDocs, numDocsPerChunk, totalMaxLength, writerVersion) - : new VarByteChunkForwardIndexWriterV4(file, compressionType, TARGET_MAX_CHUNK_SIZE); + _indexWriter = + writerVersion < VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexWriter(indexFile, + compressionType, totalDocs, numDocsPerChunk, totalMaxLength, writerVersion) + : new VarByteChunkForwardIndexWriterV4(indexFile, compressionType, TARGET_MAX_CHUNK_SIZE); _valueType = valueType; } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index b5ca2b83c17e..686479be450a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -19,7 +19,12 @@ package org.apache.pinot.segment.local.segment.index.forward; +import com.mchange.lang.ByteUtils; +import org.apache.commons.lang.ArrayUtils; +import org.apache.lucene.util.ArrayUtil; +import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; +import org.apache.pinot.segment.local.segment.index.readers.forward.CLPForwardIndexReaderV1; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitMVEntryDictForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitMVForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitSVForwardIndexReaderV2; @@ -83,6 +88,13 @@ public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, C } } } else { + if (dataBuffer.size() >= CLPForwardIndexWriterV1.MAGIC_BYTES.length) { + byte[] magicBytes = new byte[CLPForwardIndexWriterV1.MAGIC_BYTES.length]; + dataBuffer.copyTo(0, magicBytes); + if (ArrayUtils.isEquals(magicBytes, CLPForwardIndexWriterV1.MAGIC_BYTES)) { + return new CLPForwardIndexReaderV1(dataBuffer); + } + } return createRawIndexReader(dataBuffer, metadata.getDataType().getStoredType(), metadata.isSingleValue()); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java new file mode 100644 index 000000000000..fb9b97099a5b --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -0,0 +1,131 @@ +/** + * 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.segment.local.segment.index.readers.forward; + +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.MessageDecoder; +import java.io.IOException; +import org.apache.pinot.segment.local.io.util.PinotDataBitSet; +import org.apache.pinot.segment.local.io.util.VarLengthValueReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.FieldSpec; + +import static org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1.MAGIC_BYTES; + + +public class CLPForwardIndexReaderV1 implements ForwardIndexReader { + private final int _version; + private final int _numDocs; + private final int _totalEncodedVarValues; + private final VarLengthValueReader _logTypeDictReader; + private final VarLengthValueReader _dictVarsDictReader; + private final FixedBitSVForwardIndexReader _logTypeFwdIndexReader; + private final FixedBitMVForwardIndexReader _dictVarsFwdIndexReader; + private final VarByteChunkForwardIndexReaderV4 _encodedVarFwdIndexReader; + private final VarByteChunkForwardIndexReaderV4.ReaderContext _encodedVarContext; + private final MessageDecoder _clpMessageDecoder; + + public CLPForwardIndexReaderV1(PinotDataBuffer pinotDataBuffer) { + int offset = MAGIC_BYTES.length; + _version = pinotDataBuffer.getInt(offset); + offset += 4; + _numDocs = pinotDataBuffer.getInt(offset); + offset += 4; + _totalEncodedVarValues = pinotDataBuffer.getInt(offset); + offset += 4; + + int logTypeDictLength = pinotDataBuffer.getInt(offset); + offset += 4; + int dictVarDictLength = pinotDataBuffer.getInt(offset); + offset += 4; + int logTypeFwdIndexLength = pinotDataBuffer.getInt(offset); + offset += 4; + int dictVarsFwdIndexLength = pinotDataBuffer.getInt(offset); + offset += 4; + int encodedVarFwdIndexLength = pinotDataBuffer.getInt(offset); + offset += 4; + + _logTypeDictReader = new VarLengthValueReader(pinotDataBuffer.view(offset, logTypeDictLength)); + offset += logTypeDictLength; + + _dictVarsDictReader = new VarLengthValueReader(pinotDataBuffer.view(offset, dictVarDictLength)); + offset += dictVarDictLength; + + _logTypeFwdIndexReader = + new FixedBitSVForwardIndexReader(pinotDataBuffer.view(offset, logTypeFwdIndexLength), _numDocs, + PinotDataBitSet.getNumBitsPerValue(_dictVarsDictReader.getNumValues() - 1)); + offset += logTypeFwdIndexLength; + + _dictVarsFwdIndexReader = + new FixedBitMVForwardIndexReader(pinotDataBuffer.view(offset, dictVarsFwdIndexLength), _numDocs, + _totalEncodedVarValues, PinotDataBitSet.getNumBitsPerValue(_dictVarsDictReader.getNumValues() - 1)); + offset += dictVarsFwdIndexLength; + + _encodedVarFwdIndexReader = + new VarByteChunkForwardIndexReaderV4(pinotDataBuffer.view(offset, encodedVarFwdIndexLength), + FieldSpec.DataType.LONG, false); + offset += encodedVarFwdIndexLength; + _encodedVarContext = _encodedVarFwdIndexReader.createContext(); + + _clpMessageDecoder = new MessageDecoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return true; + } + + @Override + public FieldSpec.DataType getStoredType() { + return FieldSpec.DataType.STRING; + } + + @Override + public String getString(int docId, ForwardIndexReaderContext context) { + int logTypeDictId = _logTypeFwdIndexReader.getDictId(docId, _logTypeFwdIndexReader.createContext()); + String logType = _logTypeDictReader.getUnpaddedString(logTypeDictId, 10000, new byte[10000]); + int[] dictVarsDictIds = _dictVarsFwdIndexReader.getDictIdMV(docId, _dictVarsFwdIndexReader.createContext()); + + String[] dictVars = new String[dictVarsDictIds.length]; + for (int i = 0; i < dictVarsDictIds.length; i++) { + dictVars[i] = _dictVarsDictReader.getUnpaddedString(dictVarsDictIds[i], 10000, new byte[10000]); + } + long[] encodedVar = _encodedVarFwdIndexReader.getLongMV(docId, _encodedVarContext); + + try { + return _clpMessageDecoder.decodeMessage(logType, dictVars, encodedVar); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() + throws IOException { + + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java new file mode 100644 index 000000000000..cae5afaa3c86 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java @@ -0,0 +1,79 @@ +package org.apache.pinot.segment.local.segment.index.creator; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; +import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; +import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; +import org.apache.pinot.spi.config.table.FieldConfig; +import org.apache.pinot.spi.config.table.IndexingConfig; +import org.apache.pinot.spi.config.table.SegmentsValidationAndRetentionConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableCustomConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.TenantConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.testng.annotations.Test; + + +/** + * 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. + */ +public class CLPWriterTest { + + @Test + public void testCLPWriter() throws Exception { + List logLines = new ArrayList<>(); + logLines.add("2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); + logLines.add("2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + logLines.add("2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12.15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); + logLines.add("2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 10.12.19.5 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type application/json status code 200 OK"); + + Schema schema = new Schema(); + schema.addField(new DimensionFieldSpec("column1", FieldSpec.DataType.STRING, true)); + TableConfig tableConfig = new TableConfig("mytable", TableType.REALTIME.name(), new SegmentsValidationAndRetentionConfig(), + new TenantConfig(null, null, null), new IndexingConfig(), new TableCustomConfig(null), null, null, null, + null, null, null, null, null, null, null, null, false, null, null, null); + List fieldConfigList = new ArrayList<>(); + fieldConfigList.add(new FieldConfig("column1", FieldConfig.EncodingType.RAW, Collections.EMPTY_LIST, + FieldConfig.CompressionCodec.CLP, Collections.EMPTY_MAP)); + tableConfig.setFieldConfigList(fieldConfigList); + StatsCollectorConfig statsCollectorConfig = new StatsCollectorConfig(tableConfig, schema, null); + StringColumnPreIndexStatsCollector + statsCollector = new StringColumnPreIndexStatsCollector("column1", statsCollectorConfig); + for (String logLine : logLines) { + statsCollector.collect(logLine); + } + + statsCollector.seal(); + System.out.println(statsCollector.getClpStats()); + + File tmpDir = new File("/tmp/"); + File indexFile = new File(tmpDir, "column1.fwd"); + CLPForwardIndexWriterV1 clpForwardIndexWriterV1 = new CLPForwardIndexWriterV1(tmpDir, indexFile, "column1", 4, statsCollector); + + for (String logLine : logLines) { + clpForwardIndexWriterV1.putString(logLine); + } + clpForwardIndexWriterV1.close(); + } +} From c80f6e0c82c1fdd42ed58c4c49a1b5ce7d3cd489 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Fri, 12 Jan 2024 11:21:49 +0530 Subject: [PATCH 10/30] Complete the reader --- .../local/io/util/VarLengthValueWriter.java | 25 +++----------- .../writer/impl/CLPForwardIndexWriterV1.java | 9 ++--- .../forward/ForwardIndexReaderFactory.java | 2 +- .../forward/CLPForwardIndexReaderV1.java | 33 +++++++++++-------- .../segment/index/creator/CLPWriterTest.java | 8 +++++ 5 files changed, 39 insertions(+), 38 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java index 778a2c541d55..df61cacc0e3e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueWriter.java @@ -93,18 +93,6 @@ public class VarLengthValueWriter implements Closeable { private final ByteBuffer _offsetBuffer; private final ByteBuffer _valueBuffer; - public VarLengthValueWriter(ByteBuffer offsetBuffer, int numValues) { - _offsetBuffer = offsetBuffer; - _offsetBuffer.put(MAGIC_BYTES); - _offsetBuffer.putInt(VERSION); - _offsetBuffer.putInt(numValues); - _offsetBuffer.putInt(HEADER_LENGTH); - - _valueBuffer = _offsetBuffer.duplicate(); - _valueBuffer.position(HEADER_LENGTH + (numValues + 1) * Integer.BYTES); - _fileChannel = null; - } - public VarLengthValueWriter(File outputFile, int numValues) throws IOException { _fileChannel = new RandomAccessFile(outputFile, "rw").getChannel(); @@ -136,14 +124,11 @@ public void close() throws IOException { int fileLength = _valueBuffer.position(); _offsetBuffer.putInt(fileLength); - - if (_fileChannel != null) { - _fileChannel.truncate(fileLength); - _fileChannel.close(); - if (CleanerUtil.UNMAP_SUPPORTED) { - CleanerUtil.BufferCleaner cleaner = CleanerUtil.getCleaner(); - cleaner.freeBuffer(_offsetBuffer); - } + _fileChannel.truncate(fileLength); + _fileChannel.close(); + if (CleanerUtil.UNMAP_SUPPORTED) { + CleanerUtil.BufferCleaner cleaner = CleanerUtil.getCleaner(); + cleaner.freeBuffer(_offsetBuffer); } } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index 9b4f32ff83e9..f0d74a4c149b 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -101,7 +101,7 @@ public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, _encodedVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_encodedvars.fwd"); _encodedVarsFwdIndexWriter = new MultiValueFixedByteRawIndexCreator(_encodedVarsFwdIndexFile, ChunkCompressionType.PASS_THROUGH, numDocs, - FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, 2); + FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, VarByteChunkForwardIndexWriterV4.VERSION); _clpStats.clear(); _clpEncodedMessage = new EncodedMessage(); @@ -175,14 +175,15 @@ public void close() _dictVarsFwdIndexWriter.close(); _encodedVarsFwdIndexWriter.close(); - int totalSize = MAGIC_BYTES.length + 10 * 4 + (int) _logTypeDictFile.length() + (int) _dictVarsDictFile.length() + + int totalSize = MAGIC_BYTES.length + 9 * 4 + (int) _logTypeDictFile.length() + (int) _dictVarsDictFile.length() + (int) _logTypeFwdIndexFile.length() + (int) _dictVarsFwdIndexFile.length() + (int) _encodedVarsFwdIndexFile.length(); _fileBuffer.put(MAGIC_BYTES); _fileBuffer.putInt(1); // version - _fileBuffer.putInt((int) _numDocs); // logType fwd index length - _fileBuffer.putInt(_clpStats.getTotalNumberOfEncodedVars()); + _fileBuffer.putInt(_clpStats.getTotalNumberOfDictVars()); + _fileBuffer.putInt(_logTypeDictCreator.getNumBytesPerEntry()); + _fileBuffer.putInt(_dictVarsDictCreator.getNumBytesPerEntry()); _fileBuffer.putInt((int) _logTypeDictFile.length()); // logType dict length _fileBuffer.putInt((int) _dictVarsDictFile.length()); // dictVars dict length _fileBuffer.putInt((int) _logTypeFwdIndexFile.length()); // logType fwd index length diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index 686479be450a..a6ff8037de7d 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -92,7 +92,7 @@ public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, C byte[] magicBytes = new byte[CLPForwardIndexWriterV1.MAGIC_BYTES.length]; dataBuffer.copyTo(0, magicBytes); if (ArrayUtils.isEquals(magicBytes, CLPForwardIndexWriterV1.MAGIC_BYTES)) { - return new CLPForwardIndexReaderV1(dataBuffer); + return new CLPForwardIndexReaderV1(dataBuffer, metadata.getTotalDocs()); } } return createRawIndexReader(dataBuffer, metadata.getDataType().getStoredType(), metadata.isSingleValue()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index fb9b97099a5b..349cc24d6cac 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -34,7 +34,9 @@ public class CLPForwardIndexReaderV1 implements ForwardIndexReader { private final int _version; private final int _numDocs; - private final int _totalEncodedVarValues; + private final int _totalDictVarValues; + private final int _logTypeDictNumBytesPerValue; + private final int _dictVarsDictNumBytesPerValue; private final VarLengthValueReader _logTypeDictReader; private final VarLengthValueReader _dictVarsDictReader; private final FixedBitSVForwardIndexReader _logTypeFwdIndexReader; @@ -43,13 +45,16 @@ public class CLPForwardIndexReaderV1 implements ForwardIndexReader Date: Mon, 15 Jan 2024 10:14:36 +0530 Subject: [PATCH 11/30] Lint fix --- .../writer/impl/CLPForwardIndexWriterV1.java | 10 ++-- .../impl/SegmentDictionaryCreator.java | 13 +++-- .../forward/ForwardIndexReaderFactory.java | 35 ++++++------- .../forward/CLPForwardIndexReaderV1.java | 10 ++-- .../segment/creator/DictionariesTest.java | 27 ++++++---- .../segment/index/creator/CLPWriterTest.java | 52 +++++++++++++++---- .../spi/creator/StatsCollectorConfig.java | 1 - 7 files changed, 90 insertions(+), 58 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index f0d74a4c149b..4267c992afa9 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -37,6 +37,7 @@ import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.spi.data.FieldSpec; + /** * HEADER * version @@ -101,7 +102,8 @@ public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, _encodedVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_encodedvars.fwd"); _encodedVarsFwdIndexWriter = new MultiValueFixedByteRawIndexCreator(_encodedVarsFwdIndexFile, ChunkCompressionType.PASS_THROUGH, numDocs, - FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, VarByteChunkForwardIndexWriterV4.VERSION); + FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, + VarByteChunkForwardIndexWriterV4.VERSION); _clpStats.clear(); _clpEncodedMessage = new EncodedMessage(); @@ -175,9 +177,9 @@ public void close() _dictVarsFwdIndexWriter.close(); _encodedVarsFwdIndexWriter.close(); - int totalSize = MAGIC_BYTES.length + 9 * 4 + (int) _logTypeDictFile.length() + (int) _dictVarsDictFile.length() + - (int) _logTypeFwdIndexFile.length() + (int) _dictVarsFwdIndexFile.length() + - (int) _encodedVarsFwdIndexFile.length(); + int totalSize = MAGIC_BYTES.length + 9 * 4 + (int) _logTypeDictFile.length() + (int) _dictVarsDictFile.length() + + (int) _logTypeFwdIndexFile.length() + (int) _dictVarsFwdIndexFile.length() + + (int) _encodedVarsFwdIndexFile.length(); _fileBuffer.put(MAGIC_BYTES); _fileBuffer.putInt(1); // version diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java index c78cd25d0725..202153088d42 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentDictionaryCreator.java @@ -45,6 +45,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; + /** * The IndexCreator for dictionaries. * @@ -70,7 +71,8 @@ public class SegmentDictionaryCreator implements IndexCreator { private Object2IntOpenHashMap _objectValueToIndexMap; private int _numBytesPerEntry = 0; - public SegmentDictionaryCreator(String columnName, DataType storedType, File indexFile, boolean useVarLengthDictionary) { + public SegmentDictionaryCreator(String columnName, DataType storedType, File indexFile, + boolean useVarLengthDictionary) { _columnName = columnName; _storedType = storedType; _dictionaryFile = indexFile; @@ -83,6 +85,11 @@ public SegmentDictionaryCreator(FieldSpec fieldSpec, File indexDir, boolean useV _dictionaryFile = new File(indexDir, _columnName + DictionaryIndexType.getFileExtension()); _useVarLengthDictionary = useVarLengthDictionary; } + + public SegmentDictionaryCreator(FieldSpec fieldSpec, File indexDir) { + this(fieldSpec, indexDir, false); + } + @Override public void add(@Nonnull Object value, int dictId) throws IOException { @@ -95,10 +102,6 @@ public void add(@Nonnull Object[] values, @Nullable int[] dictIds) throw new UnsupportedOperationException("Dictionaries should not be built as a normal index"); } - public SegmentDictionaryCreator(FieldSpec fieldSpec, File indexDir) { - this(fieldSpec, indexDir, false); - } - public void build(Object sortedValues) throws IOException { FileUtils.touch(_dictionaryFile); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index a6ff8037de7d..0b640ebdba47 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -19,9 +19,7 @@ package org.apache.pinot.segment.local.segment.index.forward; -import com.mchange.lang.ByteUtils; import org.apache.commons.lang.ArrayUtils; -import org.apache.lucene.util.ArrayUtil; import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; import org.apache.pinot.segment.local.segment.index.readers.forward.CLPForwardIndexReaderV1; @@ -49,24 +47,12 @@ public class ForwardIndexReaderFactory extends IndexReaderFactory.Default { private static volatile ForwardIndexReaderFactory _instance = new ForwardIndexReaderFactory(); - public static void setInstance(ForwardIndexReaderFactory factory) { - _instance = factory; - } - public static ForwardIndexReaderFactory getInstance() { return _instance; } - @Override - protected IndexType getIndexType() { - return StandardIndexes.forward(); - } - - @Override - protected ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata, - ForwardIndexConfig indexConfig) - throws IndexReaderConstraintException { - return createIndexReader(dataBuffer, metadata); + public static void setInstance(ForwardIndexReaderFactory factory) { + _instance = factory; } public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata) { @@ -103,9 +89,8 @@ public static ForwardIndexReader createRawIndexReader(PinotDataBuffer dataBuffer boolean isSingleValue) { int version = dataBuffer.getInt(0); if (isSingleValue && storedType.isFixedWidth()) { - return version == FixedBytePower2ChunkSVForwardIndexReader.VERSION - ? new FixedBytePower2ChunkSVForwardIndexReader(dataBuffer, storedType) - : new FixedByteChunkSVForwardIndexReader(dataBuffer, storedType); + return version == FixedBytePower2ChunkSVForwardIndexReader.VERSION ? new FixedBytePower2ChunkSVForwardIndexReader( + dataBuffer, storedType) : new FixedByteChunkSVForwardIndexReader(dataBuffer, storedType); } if (version == VarByteChunkForwardIndexWriterV4.VERSION) { @@ -129,4 +114,16 @@ private static ForwardIndexReader createNonV4RawIndexReader(PinotDataBuffer data } } } + + @Override + protected IndexType getIndexType() { + return StandardIndexes.forward(); + } + + @Override + protected ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata, + ForwardIndexConfig indexConfig) + throws IndexReaderConstraintException { + return createIndexReader(dataBuffer, metadata); + } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index 349cc24d6cac..f4e5a9e3c87d 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -23,14 +23,12 @@ import java.io.IOException; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.io.util.VarLengthValueReader; +import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; import org.apache.pinot.spi.data.FieldSpec; -import static org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1.MAGIC_BYTES; - - public class CLPForwardIndexReaderV1 implements ForwardIndexReader { private final int _version; private final int _numDocs; @@ -47,7 +45,7 @@ public class CLPForwardIndexReaderV1 implements ForwardIndexReader logLines = new ArrayList<>(); - logLines.add("2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); - logLines.add("2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); - logLines.add("2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12.15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); - logLines.add("2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 10.12.19.5 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type application/json status code 200 OK"); + logLines.add( + "2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] " + + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); + logLines.add( + "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " + + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + logLines.add( + "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12" + + ".15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); + logLines.add( + "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 10.12" + + ".19.5 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + + "application/json status code 200 OK"); Schema schema = new Schema(); schema.addField(new DimensionFieldSpec("column1", FieldSpec.DataType.STRING, true)); - TableConfig tableConfig = new TableConfig("mytable", TableType.REALTIME.name(), new SegmentsValidationAndRetentionConfig(), - new TenantConfig(null, null, null), new IndexingConfig(), new TableCustomConfig(null), null, null, null, - null, null, null, null, null, null, null, null, false, null, null, null); + TableConfig tableConfig = + new TableConfig("mytable", TableType.REALTIME.name(), new SegmentsValidationAndRetentionConfig(), + new TenantConfig(null, null, null), new IndexingConfig(), new TableCustomConfig(null), null, null, null, + null, null, null, null, null, null, null, null, false, null, null, null); List fieldConfigList = new ArrayList<>(); fieldConfigList.add(new FieldConfig("column1", FieldConfig.EncodingType.RAW, Collections.EMPTY_LIST, FieldConfig.CompressionCodec.CLP, Collections.EMPTY_MAP)); tableConfig.setFieldConfigList(fieldConfigList); StatsCollectorConfig statsCollectorConfig = new StatsCollectorConfig(tableConfig, schema, null); - StringColumnPreIndexStatsCollector - statsCollector = new StringColumnPreIndexStatsCollector("column1", statsCollectorConfig); + StringColumnPreIndexStatsCollector statsCollector = + new StringColumnPreIndexStatsCollector("column1", statsCollectorConfig); for (String logLine : logLines) { statsCollector.collect(logLine); } @@ -71,7 +100,8 @@ public void testCLPWriter() throws Exception { File tmpDir = new File("/tmp/"); File indexFile = new File(tmpDir, "column1.fwd"); - CLPForwardIndexWriterV1 clpForwardIndexWriterV1 = new CLPForwardIndexWriterV1(tmpDir, indexFile, "column1", 4, statsCollector); + CLPForwardIndexWriterV1 clpForwardIndexWriterV1 = + new CLPForwardIndexWriterV1(tmpDir, indexFile, "column1", 4, statsCollector); for (String logLine : logLines) { clpForwardIndexWriterV1.putString(logLine); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java index f8c48650dbfe..fdd8abfcd7f9 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java @@ -58,7 +58,6 @@ public StatsCollectorConfig(TableConfig tableConfig, Schema schema, _columnFieldConfigMap.put(fieldConfig.getName(), fieldConfig); } } - } @Nullable From abb3f64b4bf384df4db6d5bcad380c3a9e4471e2 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 15 Jan 2024 10:52:18 +0530 Subject: [PATCH 12/30] Avoid full file copy --- .../io/writer/impl/CLPForwardIndexWriterV1.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index 4267c992afa9..c9f848fcea7c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -28,6 +28,7 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.charset.StandardCharsets; +import java.nio.file.StandardOpenOption; import org.apache.commons.io.FileUtils; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator; @@ -192,11 +193,11 @@ public void close() _fileBuffer.putInt((int) _dictVarsFwdIndexFile.length()); // dictVars fwd index length _fileBuffer.putInt((int) _encodedVarsFwdIndexFile.length()); // encodedVars fwd index length - _fileBuffer.put(FileUtils.readFileToByteArray(_logTypeDictFile)); - _fileBuffer.put(FileUtils.readFileToByteArray(_dictVarsDictFile)); - _fileBuffer.put(FileUtils.readFileToByteArray(_logTypeFwdIndexFile)); - _fileBuffer.put(FileUtils.readFileToByteArray(_dictVarsFwdIndexFile)); - _fileBuffer.put(FileUtils.readFileToByteArray(_encodedVarsFwdIndexFile)); + copyFileIntoBuffer(_logTypeDictFile); + copyFileIntoBuffer(_dictVarsDictFile); + copyFileIntoBuffer(_logTypeFwdIndexFile); + copyFileIntoBuffer(_dictVarsFwdIndexFile); + copyFileIntoBuffer(_encodedVarsFwdIndexFile); _dataFile.truncate(totalSize); @@ -207,4 +208,10 @@ public void close() FileUtils.deleteQuietly(_dictVarsFwdIndexFile); FileUtils.deleteQuietly(_encodedVarsFwdIndexFile); } + + private void copyFileIntoBuffer(File file) throws IOException { + try (FileChannel from = (FileChannel.open(file.toPath(), StandardOpenOption.READ))) { + _fileBuffer.put(from.map(FileChannel.MapMode.READ_ONLY, 0, file.length())); + } + } } From acffbb77d0d44a1792abc90719178e18b9c0d584 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 15 Jan 2024 14:14:31 +0530 Subject: [PATCH 13/30] Add mutable CLP index --- .../writer/impl/CLPForwardIndexWriterV1.java | 5 +- .../MutableNoDictionaryColStatistics.java | 13 +- .../impl/forward/CLPMutableForwardIndex.java | 187 ++++++++++++++++++ .../creator/impl/stats/CLPStatsProvider.java | 114 +++++++++++ .../StringColumnPreIndexStatsCollector.java | 91 +-------- .../index/forward/ForwardIndexType.java | 6 +- .../segment/creator/DictionariesTest.java | 2 +- .../segment/index/creator/CLPWriterTest.java | 2 +- 8 files changed, 332 insertions(+), 88 deletions(-) create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index c9f848fcea7c..f75be2a8c324 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -33,6 +33,7 @@ import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator; import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator; +import org.apache.pinot.segment.local.segment.creator.impl.stats.CLPStatsProvider; import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.creator.ColumnStatistics; @@ -78,8 +79,8 @@ public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, _dataFile = new RandomAccessFile(indexFile, "rw").getChannel(); _fileBuffer = _dataFile.map(FileChannel.MapMode.READ_WRITE, 0, Integer.MAX_VALUE); - StringColumnPreIndexStatsCollector statsCollector = (StringColumnPreIndexStatsCollector) columnStatistics; - _clpStats = statsCollector.getClpStats(); + CLPStatsProvider statsCollector = (CLPStatsProvider) columnStatistics; + _clpStats = statsCollector.getCLPStats(); _logTypeDictFile = new File(_baseIndexDir, _column + "_clp_logtype.dict"); _logTypeDictCreator = new SegmentDictionaryCreator(_column + "_clp_logtype.dict", FieldSpec.DataType.STRING, _logTypeDictFile, true); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableNoDictionaryColStatistics.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableNoDictionaryColStatistics.java index 0cf4c37b9669..5f2d98893b72 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableNoDictionaryColStatistics.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableNoDictionaryColStatistics.java @@ -21,6 +21,8 @@ import com.google.common.base.Preconditions; import java.util.Map; import java.util.Set; +import org.apache.pinot.segment.local.realtime.impl.forward.CLPMutableForwardIndex; +import org.apache.pinot.segment.local.segment.creator.impl.stats.CLPStatsProvider; import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; @@ -30,7 +32,7 @@ import static org.apache.pinot.segment.spi.Constants.UNKNOWN_CARDINALITY; -public class MutableNoDictionaryColStatistics implements ColumnStatistics { +public class MutableNoDictionaryColStatistics implements ColumnStatistics, CLPStatsProvider { private final DataSourceMetadata _dataSourceMetadata; private final MutableForwardIndex _forwardIndex; @@ -111,4 +113,13 @@ public Map getPartitionFunctionConfig() { public Set getPartitions() { return _dataSourceMetadata.getPartitions(); } + + @Override + public CLPStats getCLPStats() { + if (_forwardIndex instanceof CLPMutableForwardIndex) { + return ((CLPMutableForwardIndex) _forwardIndex).getCLPStats(); + } + throw new IllegalStateException( + "CLP stats not available for column: " + _dataSourceMetadata.getFieldSpec().getName()); + } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java new file mode 100644 index 000000000000..5d36e48cea5d --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java @@ -0,0 +1,187 @@ +/** + * 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.segment.local.realtime.impl.forward; + +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageDecoder; +import com.yscope.clp.compressorfrontend.MessageEncoder; +import java.io.IOException; +import org.apache.pinot.segment.local.realtime.impl.dictionary.StringOffHeapMutableDictionary; +import org.apache.pinot.segment.local.segment.creator.impl.stats.CLPStatsProvider; +import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; +import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexType; +import org.apache.pinot.segment.spi.index.mutable.MutableDictionary; +import org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex; +import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; +import org.apache.pinot.spi.data.FieldSpec; + +/** + * 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. + */ +public class CLPMutableForwardIndex implements MutableForwardIndex { + private final EncodedMessage _clpEncodedMessage; + private final MessageEncoder _clpMessageEncoder; + private final MessageDecoder _clpMessageDecoder; + private final MutableDictionary _logTypeDictCreator; + private final MutableDictionary _dictVarsDictCreator; + private final FixedByteSVMutableForwardIndex _logTypeFwdIndex; + private final FixedByteMVMutableForwardIndex _dictVarsFwdIndex; + private final FixedByteMVMutableForwardIndex _encodedVarsFwdIndex; + private int _lengthOfShortestElement; + private int _lengthOfLongestElement; + + // clp stats + int _totalNumberOfDictVars = 0; + int _maxNumberOfEncodedVars = 0; + int _totalNumberOfEncodedVars = 0; + + + public CLPMutableForwardIndex(String columnName, PinotDataBufferMemoryManager memoryManager, int capacity) { + _clpEncodedMessage = new EncodedMessage(); + _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + _logTypeDictCreator = + new StringOffHeapMutableDictionary(10000, 100, memoryManager, columnName + "_logType.dict", 1000); + _dictVarsDictCreator = + new StringOffHeapMutableDictionary(10000, 100, memoryManager, columnName + "_dictVars.dict", 1000); + + _logTypeFwdIndex = new FixedByteSVMutableForwardIndex(true, FieldSpec.DataType.INT, capacity, memoryManager, + columnName + "_logType.fwd"); + _dictVarsFwdIndex = + new FixedByteMVMutableForwardIndex(ForwardIndexType.MAX_MULTI_VALUES_PER_ROW, 20, capacity, Integer.BYTES, + memoryManager, columnName + "_dictVars.fwd", true, FieldSpec.DataType.INT); + _encodedVarsFwdIndex = + new FixedByteMVMutableForwardIndex(ForwardIndexType.MAX_MULTI_VALUES_PER_ROW, 20, capacity, Long.BYTES, + memoryManager, columnName + "_encodedVars.fwd", true, FieldSpec.DataType.LONG); + _clpMessageDecoder = new MessageDecoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + @Override + public int getLengthOfShortestElement() { + return _lengthOfShortestElement; + } + + @Override + public int getLengthOfLongestElement() { + return _lengthOfLongestElement; + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return true; + } + + @Override + public FieldSpec.DataType getStoredType() { + return null; + } + + @Override + public void setString(int docId, String value) { + String logtype; + String[] dictVars; + Long[] encodedVars; + + _lengthOfLongestElement = Math.max(_lengthOfLongestElement, value.length()); + _lengthOfShortestElement = Math.min(_lengthOfShortestElement, value.length()); + + try { + _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); + logtype = _clpEncodedMessage.getLogTypeAsString(); + dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + encodedVars = _clpEncodedMessage.getEncodedVarsAsBoxedLongs(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to encode message: " + value, e); + } + + _totalNumberOfDictVars += dictVars.length; + _totalNumberOfEncodedVars += encodedVars.length; + _maxNumberOfEncodedVars = Math.max(_maxNumberOfEncodedVars, encodedVars.length); + + int logTypeDictId = _logTypeDictCreator.index(logtype); + _logTypeFwdIndex.setDictId(docId, logTypeDictId); + + int[] dictVarsDictIds = new int[dictVars.length]; + for (int i = 0; i < dictVars.length; i++) { + dictVarsDictIds[i] = _dictVarsDictCreator.index(dictVars[i]); + } + _dictVarsFwdIndex.setDictIdMV(docId, dictVarsDictIds); + + long[] encodedVarsLongs = new long[encodedVars.length]; + for (int i = 0; i < encodedVars.length; i++) { + encodedVarsLongs[i] = encodedVars[i]; + } + _encodedVarsFwdIndex.setLongMV(docId, encodedVarsLongs); + } + + @Override + public String getString(int docId) { + String logType = _logTypeDictCreator.getStringValue(_logTypeFwdIndex.getDictId(docId)); + int[] dictVarsDictIds = _dictVarsFwdIndex.getDictIdMV(docId); + String[] dictVars = new String[dictVarsDictIds.length]; + for (int i = 0; i < dictVarsDictIds.length; i++) { + dictVars[i] = _dictVarsDictCreator.getStringValue(dictVarsDictIds[i]); + } + long[] encodedVarsLongs = _encodedVarsFwdIndex.getLongMV(docId); + try { + return _clpMessageDecoder.decodeMessage(logType, dictVars, encodedVarsLongs); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to encode message: " + logType, e); + } + } + + public StringColumnPreIndexStatsCollector.CLPStats getCLPStats() { + return new CLPStatsProvider.CLPStats((String[]) _logTypeDictCreator.getSortedValues(), + (String[]) _dictVarsDictCreator.getSortedValues(), + _totalNumberOfDictVars, _totalNumberOfEncodedVars, _maxNumberOfEncodedVars); + } + + + @Override + public void close() + throws IOException { + _logTypeDictCreator.close(); + _dictVarsDictCreator.close(); + _logTypeFwdIndex.close(); + _dictVarsFwdIndex.close(); + _encodedVarsFwdIndex.close(); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java new file mode 100644 index 000000000000..f2a9e26779b3 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java @@ -0,0 +1,114 @@ +/** + * 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.segment.local.segment.creator.impl.stats; + +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageEncoder; +import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; +import java.io.IOException; +import java.util.Arrays; +import java.util.Set; + + +public interface CLPStatsProvider { + + StringColumnPreIndexStatsCollector.CLPStats getCLPStats(); + + class CLPStats { + private final EncodedMessage _clpEncodedMessage; + private final MessageEncoder _clpMessageEncoder; + int _totalNumberOfDictVars = 0; + int _totalNumberOfEncodedVars = 0; + int _maxNumberOfEncodedVars = 0; + private String[] _sortedLogTypeValues; + private String[] _sortedDictVarValues; + private Set _logTypes = new ObjectOpenHashSet<>(AbstractColumnStatisticsCollector.INITIAL_HASH_SET_SIZE); + private Set _dictVars = new ObjectOpenHashSet<>(AbstractColumnStatisticsCollector.INITIAL_HASH_SET_SIZE); + + public CLPStats(String[] sortedLogTypeValues, String[] sortedDictVarValues, int totalNumberOfDictVars, + int totalNumberOfEncodedVars, int maxNumberOfEncodedVars) { + _sortedLogTypeValues = sortedLogTypeValues; + _sortedDictVarValues = sortedDictVarValues; + _totalNumberOfDictVars = totalNumberOfDictVars; + _totalNumberOfEncodedVars = totalNumberOfEncodedVars; + _maxNumberOfEncodedVars = maxNumberOfEncodedVars; + _clpEncodedMessage = null; + _clpMessageEncoder = null; + } + + public CLPStats() { + _clpEncodedMessage = new EncodedMessage(); + _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + public int getMaxNumberOfEncodedVars() { + return _maxNumberOfEncodedVars; + } + + public int getTotalNumberOfDictVars() { + return _totalNumberOfDictVars; + } + + public int getTotalNumberOfEncodedVars() { + return _totalNumberOfEncodedVars; + } + + public void collect(String value) { + String logType; + String[] dictVars; + try { + _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); + logType = _clpEncodedMessage.getLogTypeAsString(); + dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to encode message: " + value, e); + } + _logTypes.add(logType); + _dictVars.addAll(Arrays.asList(dictVars)); + _totalNumberOfDictVars += dictVars.length; + _totalNumberOfEncodedVars += _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length; + _maxNumberOfEncodedVars = + Math.max(_maxNumberOfEncodedVars, _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length); + } + + public void seal() { + _sortedLogTypeValues = _logTypes.toArray(new String[0]); + _logTypes = null; + Arrays.sort(_sortedLogTypeValues); + _sortedDictVarValues = _dictVars.toArray(new String[0]); + _dictVars = null; + Arrays.sort(_sortedDictVarValues); + } + + public void clear() { + _sortedLogTypeValues = null; + _sortedDictVarValues = null; + } + + public String[] getSortedLogTypeValues() { + return _sortedLogTypeValues; + } + + public String[] getSortedDictVarValues() { + return _sortedDictVarValues; + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java index 3c44d7eb6512..08288ba34ea1 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java @@ -18,11 +18,7 @@ */ package org.apache.pinot.segment.local.segment.creator.impl.stats; -import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; -import com.yscope.clp.compressorfrontend.EncodedMessage; -import com.yscope.clp.compressorfrontend.MessageEncoder; import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; -import java.io.IOException; import java.util.Arrays; import java.util.Set; import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; @@ -31,13 +27,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; -public class StringColumnPreIndexStatsCollector extends AbstractColumnStatisticsCollector { - public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { - super(column, statsCollectorConfig); - if (_fieldConfig != null && _fieldConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { - _clpStats = new CLPStats(); - } - } +public class StringColumnPreIndexStatsCollector extends AbstractColumnStatisticsCollector implements CLPStatsProvider { private Set _values = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); private int _minLength = Integer.MAX_VALUE; private int _maxLength = 0; @@ -45,6 +35,12 @@ public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig st private String[] _sortedValues; private boolean _sealed = false; private CLPStats _clpStats; + public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { + super(column, statsCollectorConfig); + if (_fieldConfig != null && _fieldConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { + _clpStats = new CLPStats(); + } + } @Override public void collect(Object entry) { @@ -88,7 +84,8 @@ public void collect(Object entry) { } } - public CLPStats getClpStats() { + @Override + public CLPStats getCLPStats() { return _clpStats; } @@ -149,74 +146,4 @@ public void seal() { _sealed = true; } } - - public static class CLPStats { - int _totalNumberOfDictVars = 0; - private String[] _sortedLogTypeValues; - int _totalNumberOfEncodedVars = 0; - private String[] _sortedDictVarValues; - - public int getMaxNumberOfEncodedVars() { - return _maxNumberOfEncodedVars; - } - - int _maxNumberOfEncodedVars = 0; - private Set _logTypes = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); - private Set _dictVars = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); - private final EncodedMessage _clpEncodedMessage; - private final MessageEncoder _clpMessageEncoder; - public CLPStats() { - _clpEncodedMessage = new EncodedMessage(); - _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, - BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); - } - - public int getTotalNumberOfDictVars() { - return _totalNumberOfDictVars; - } - - public int getTotalNumberOfEncodedVars() { - return _totalNumberOfEncodedVars; - } - - public void collect(String value) { - String logType; - String[] dictVars; - try { - _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); - logType = _clpEncodedMessage.getLogTypeAsString(); - dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to encode message: " + value, e); - } - _logTypes.add(logType); - _dictVars.addAll(Arrays.asList(dictVars)); - _totalNumberOfDictVars += dictVars.length; - _totalNumberOfEncodedVars += _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length; - _maxNumberOfEncodedVars = - Math.max(_maxNumberOfEncodedVars, _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length); - } - - public void seal() { - _sortedLogTypeValues = _logTypes.toArray(new String[0]); - _logTypes = null; - Arrays.sort(_sortedLogTypeValues); - _sortedDictVarValues = _dictVars.toArray(new String[0]); - _dictVars = null; - Arrays.sort(_sortedDictVarValues); - } - - public void clear() { - _sortedLogTypeValues = null; - _sortedDictVarValues = null; - } - - public String[] getSortedLogTypeValues() { - return _sortedLogTypeValues; - } - - public String[] getSortedDictVarValues() { - return _sortedDictVarValues; - } - } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java index a454bf9cfb89..16c35ae155e2 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.pinot.segment.local.realtime.impl.forward.CLPMutableForwardIndex; import org.apache.pinot.segment.local.realtime.impl.forward.FixedByteMVMutableForwardIndex; import org.apache.pinot.segment.local.realtime.impl.forward.FixedByteSVMutableForwardIndex; import org.apache.pinot.segment.local.realtime.impl.forward.VarByteSVMutableForwardIndex; @@ -68,7 +69,7 @@ public class ForwardIndexType extends AbstractIndexType EXTENSIONS = Lists.newArrayList( @@ -283,6 +284,9 @@ public MutableIndex createMutableIndex(MutableIndexContext context, ForwardIndex // Use a smaller capacity as opposed to segment flush size int initialCapacity = Math.min(context.getCapacity(), NODICT_VARIABLE_WIDTH_ESTIMATED_NUMBER_OF_VALUES_DEFAULT); + if (config.getChunkCompressionType() == ChunkCompressionType.CLP) { + return new CLPMutableForwardIndex(column, context.getMemoryManager(), context.getCapacity()); + } return new VarByteSVMutableForwardIndex(storedType, context.getMemoryManager(), allocationContext, initialCapacity, NODICT_VARIABLE_WIDTH_ESTIMATED_AVERAGE_VALUE_LENGTH_DEFAULT); } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java index e46257cb3ee8..0a9ceb81830d 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java @@ -554,6 +554,6 @@ public void clpTest() { + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); statsCollector.seal(); - System.out.println(statsCollector.getClpStats()); + System.out.println(statsCollector.getCLPStats()); } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java index d1de5036c679..f4c9f3ed01c3 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java @@ -96,7 +96,7 @@ public void testCLPWriter() } statsCollector.seal(); - System.out.println(statsCollector.getClpStats()); + System.out.println(statsCollector.getCLPStats()); File tmpDir = new File("/tmp/"); File indexFile = new File(tmpDir, "column1.fwd"); From ac69d83f70d6dd6a661c666db878540dce97e68b Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 15 Jan 2024 14:34:45 +0530 Subject: [PATCH 14/30] Fix NPe --- .../impl/forward/CLPMutableForwardIndex.java | 36 ++++++------------- .../index/forward/ForwardIndexType.java | 2 +- .../forward/CLPForwardIndexReaderV1.java | 6 ++++ 3 files changed, 17 insertions(+), 27 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java index 5d36e48cea5d..28ca5aecd1ce 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java @@ -32,25 +32,9 @@ import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; import org.apache.pinot.spi.data.FieldSpec; -/** - * 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. - */ + public class CLPMutableForwardIndex implements MutableForwardIndex { + private FieldSpec.DataType _storedType; private final EncodedMessage _clpEncodedMessage; private final MessageEncoder _clpMessageEncoder; private final MessageDecoder _clpMessageDecoder; @@ -59,16 +43,16 @@ public class CLPMutableForwardIndex implements MutableForwardIndex { private final FixedByteSVMutableForwardIndex _logTypeFwdIndex; private final FixedByteMVMutableForwardIndex _dictVarsFwdIndex; private final FixedByteMVMutableForwardIndex _encodedVarsFwdIndex; - private int _lengthOfShortestElement; - private int _lengthOfLongestElement; // clp stats int _totalNumberOfDictVars = 0; int _maxNumberOfEncodedVars = 0; int _totalNumberOfEncodedVars = 0; + private int _lengthOfShortestElement; + private int _lengthOfLongestElement; - - public CLPMutableForwardIndex(String columnName, PinotDataBufferMemoryManager memoryManager, int capacity) { + public CLPMutableForwardIndex(String columnName, FieldSpec.DataType storedType, + PinotDataBufferMemoryManager memoryManager, int capacity) { _clpEncodedMessage = new EncodedMessage(); _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); @@ -87,6 +71,7 @@ public CLPMutableForwardIndex(String columnName, PinotDataBufferMemoryManager me memoryManager, columnName + "_encodedVars.fwd", true, FieldSpec.DataType.LONG); _clpMessageDecoder = new MessageDecoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + _storedType = storedType; } @Override @@ -111,7 +96,7 @@ public boolean isSingleValue() { @Override public FieldSpec.DataType getStoredType() { - return null; + return _storedType; } @Override @@ -170,11 +155,10 @@ public String getString(int docId) { public StringColumnPreIndexStatsCollector.CLPStats getCLPStats() { return new CLPStatsProvider.CLPStats((String[]) _logTypeDictCreator.getSortedValues(), - (String[]) _dictVarsDictCreator.getSortedValues(), - _totalNumberOfDictVars, _totalNumberOfEncodedVars, _maxNumberOfEncodedVars); + (String[]) _dictVarsDictCreator.getSortedValues(), _totalNumberOfDictVars, _totalNumberOfEncodedVars, + _maxNumberOfEncodedVars); } - @Override public void close() throws IOException { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java index 16c35ae155e2..1dc1ddc62577 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java @@ -285,7 +285,7 @@ public MutableIndex createMutableIndex(MutableIndexContext context, ForwardIndex int initialCapacity = Math.min(context.getCapacity(), NODICT_VARIABLE_WIDTH_ESTIMATED_NUMBER_OF_VALUES_DEFAULT); if (config.getChunkCompressionType() == ChunkCompressionType.CLP) { - return new CLPMutableForwardIndex(column, context.getMemoryManager(), context.getCapacity()); + return new CLPMutableForwardIndex(column, storedType, context.getMemoryManager(), context.getCapacity()); } return new VarByteSVMutableForwardIndex(storedType, context.getMemoryManager(), allocationContext, initialCapacity, NODICT_VARIABLE_WIDTH_ESTIMATED_AVERAGE_VALUE_LENGTH_DEFAULT); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index f4e5a9e3c87d..8b4f30d57ed0 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -24,6 +24,7 @@ import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.io.util.VarLengthValueReader; import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; +import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; @@ -128,6 +129,11 @@ public String getString(int docId, ForwardIndexReaderContext context) { } } + @Override + public ChunkCompressionType getCompressionType() { + return ChunkCompressionType.CLP; + } + @Override public void close() throws IOException { } From 9f3a4d66c8c2a82d04b046955bbab7b178c78f91 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Tue, 16 Jan 2024 17:08:31 +0530 Subject: [PATCH 15/30] Fix for null values --- .../writer/impl/CLPForwardIndexWriterV1.java | 12 ++++++++++ .../creator/impl/stats/CLPStatsProvider.java | 22 ++++++++++++++++--- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index f75be2a8c324..ea7479ebcbe5 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -133,6 +133,18 @@ public void putString(String value) { throw new IllegalArgumentException("Failed to encode message: " + value, e); } + if (logtype == null) { + logtype = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; + } + + if (dictVars == null) { + dictVars = new String[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING}; + } + + if (encodedVars == null) { + encodedVars = new Long[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_LONG}; + } + addCLPFields(logtype, dictVars, encodedVars); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java index f2a9e26779b3..ecbab62710c9 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Set; +import org.apache.pinot.spi.data.FieldSpec; public interface CLPStatsProvider { @@ -74,19 +75,34 @@ public int getTotalNumberOfEncodedVars() { public void collect(String value) { String logType; String[] dictVars; + Long[] encodedVars; + try { _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); logType = _clpEncodedMessage.getLogTypeAsString(); dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + encodedVars = _clpEncodedMessage.getEncodedVarsAsBoxedLongs(); } catch (IOException e) { throw new IllegalArgumentException("Failed to encode message: " + value, e); } + + if (logType == null) { + logType = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; + } + + if (dictVars == null) { + dictVars = new String[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING}; + } + + if (encodedVars == null) { + encodedVars = new Long[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_LONG}; + } + _logTypes.add(logType); _dictVars.addAll(Arrays.asList(dictVars)); _totalNumberOfDictVars += dictVars.length; - _totalNumberOfEncodedVars += _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length; - _maxNumberOfEncodedVars = - Math.max(_maxNumberOfEncodedVars, _clpEncodedMessage.getEncodedVarsAsBoxedLongs().length); + _totalNumberOfEncodedVars += encodedVars.length; + _maxNumberOfEncodedVars = Math.max(_maxNumberOfEncodedVars, encodedVars.length); } public void seal() { From 71457e9c5d64a5c06532ccb2feeece9a110a2941 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Tue, 16 Jan 2024 17:12:08 +0530 Subject: [PATCH 16/30] Fix class name --- .../local/segment/creator/impl/stats/CLPStatsProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java index ecbab62710c9..84dfabc2fab0 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java @@ -30,7 +30,7 @@ public interface CLPStatsProvider { - StringColumnPreIndexStatsCollector.CLPStats getCLPStats(); + CLPStats getCLPStats(); class CLPStats { private final EncodedMessage _clpEncodedMessage; From 86823d358a5d4d67496418db6ae5e7f92b2d002e Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 22 Jan 2024 10:41:45 +0530 Subject: [PATCH 17/30] Add context handlint --- .../forward/CLPForwardIndexReaderV1.java | 42 +++++++++++++++---- 1 file changed, 35 insertions(+), 7 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index 8b4f30d57ed0..0fdf4d67e45a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -30,7 +30,7 @@ import org.apache.pinot.segment.spi.memory.PinotDataBuffer; import org.apache.pinot.spi.data.FieldSpec; -public class CLPForwardIndexReaderV1 implements ForwardIndexReader { +public class CLPForwardIndexReaderV1 implements ForwardIndexReader { private final int _version; private final int _numDocs; private final int _totalDictVarValues; @@ -41,7 +41,6 @@ public class CLPForwardIndexReaderV1 implements ForwardIndexReader Date: Tue, 9 Jan 2024 11:17:29 +0530 Subject: [PATCH 18/30] Enrichment configs --- .../manager/realtime/RealtimeSegmentDataManager.java | 9 ++++++++- .../local/recordenricher/RecordEnricherPipeline.java | 4 ++-- .../apache/pinot/segment/local/utils/IngestionUtils.java | 7 ++----- pom.xml | 1 - 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 8e1e98a69a21..daa7d54f17b0 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -1479,7 +1479,14 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf new SegmentErrorInfo(now(), "Failed to initialize the StreamMessageDecoder", e)); throw e; } - _recordEnricherPipeline = RecordEnricherPipeline.fromTableConfig(tableConfig); + + try { + _recordEnricherPipeline = RecordEnricherPipeline.fromTableConfig(tableConfig); + } catch (Exception e) { + _realtimeTableDataManager.addSegmentError(_segmentNameStr, + new SegmentErrorInfo(now(), "Failed to initialize the RecordEnricherPipeline", e)); + throw e; + } _transformPipeline = new TransformPipeline(tableConfig, schema); // Acquire semaphore to create stream consumers try { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java index 2b4e6e12a976..db70e19c5907 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordenricher/RecordEnricherPipeline.java @@ -47,8 +47,8 @@ public static RecordEnricherPipeline fromIngestionConfig(IngestionConfig ingesti RecordEnricher enricher = (RecordEnricher) Class.forName(enrichmentConfig.getEnricherClassName()).newInstance(); enricher.init(enrichmentConfig.getProperties()); pipeline.add(enricher); - } catch (Exception e) { - throw new RuntimeException("Failed to instantiate record enricher: " + enrichmentConfig.getEnricherClassName(), + } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { + throw new RuntimeException("Failed to instantiate record enricher" + enrichmentConfig.getEnricherClassName(), e); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java index 3f31d772f600..092071790e37 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java @@ -382,17 +382,14 @@ private static void extractFieldsFromIngestionConfig(@Nullable IngestionConfig i } fields.addAll(RecordEnricherPipeline.fromIngestionConfig(ingestionConfig).getColumnsToExtract()); - List transformConfigs = ingestionConfig.getTransformConfigs(); if (transformConfigs != null) { for (TransformConfig transformConfig : transformConfigs) { FunctionEvaluator expressionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(transformConfig.getTransformFunction()); fields.addAll(expressionEvaluator.getArguments()); - fields.add( - transformConfig.getColumnName()); // add the column itself too, so that if it is already transformed, - // we won't - // transform again + // add the column itself too, so that if it is already transformed, we won't transform again + fields.add(transformConfig.getColumnName()); } } ComplexTypeConfig complexTypeConfig = ingestionConfig.getComplexTypeConfig(); diff --git a/pom.xml b/pom.xml index 19f9f189dcc2..bcee2e06afc0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,6 @@ org.apache apache 21 - org.apache.pinot From e5471b6674722e85e1f7fa01b8ece90305f05ba6 Mon Sep 17 00:00:00 2001 From: Seunghyun Lee Date: Mon, 22 Jan 2024 00:55:40 -0800 Subject: [PATCH 19/30] Refactoring the upsert compaction related code (#12275) * Refactoring the upsert compaction related code 1. Fix the issue with fetching validDocId metadata for table with a large number of segments. (Added POST API with list of segments to be part of the request body) 2. Added POST support for MultiHttpRequest to cover 1. 3. Added GET /tables//validDocIdMetadata API on the controller for improving debuggability. * Addressing comments --- .../BaseBrokerRequestHandler.java | 4 +- .../pinot/common/http/MultiHttpRequest.java | 51 ++++-- .../resources/ValidDocIdMetadataInfo.java | 56 ++++++ .../common/http/MultiHttpRequestTest.java | 106 ++++++++++-- .../resources/PinotRunningQueryResource.java | 2 +- .../resources/PinotTableRestletResource.java | 37 ++++ .../util/CompletionServiceHelper.java | 52 +++++- .../util/ServerSegmentMetadataReader.java | 161 +++++++++++++++--- .../controller/util/TableMetadataReader.java | 30 +++- .../plugin/minion/tasks/MinionTaskUtils.java | 41 +++++ .../UpsertCompactionTaskExecutor.java | 141 +-------------- .../UpsertCompactionTaskGenerator.java | 106 +++--------- .../UpsertCompactionTaskExecutorTest.java | 7 +- .../UpsertCompactionTaskGeneratorTest.java | 86 +++------- .../CompactedPinotSegmentRecordReader.java | 106 ++++++++++++ .../server/api/resources/TablesResource.java | 57 +++++-- .../pinot/server/api/TablesResourceTest.java | 22 +++ 17 files changed, 715 insertions(+), 350 deletions(-) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ValidDocIdMetadataInfo.java create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/CompactedPinotSegmentRecordReader.java diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java index b2da16651025..cc640110fcf3 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java @@ -209,9 +209,9 @@ public boolean cancelQuery(long requestId, int timeoutMs, Executor executor, Htt // TODO: Use different global query id for OFFLINE and REALTIME table after releasing 0.12.0. See QueryIdUtils for // details String globalQueryId = getGlobalQueryId(requestId); - List serverUrls = new ArrayList<>(); + List> serverUrls = new ArrayList<>(); for (ServerInstance serverInstance : queryServers._servers) { - serverUrls.add(String.format("%s/query/%s", serverInstance.getAdminEndpoint(), globalQueryId)); + serverUrls.add(Pair.of(String.format("%s/query/%s", serverInstance.getAdminEndpoint(), globalQueryId), null)); } LOGGER.debug("Cancelling the query: {} via server urls: {}", queryServers._query, serverUrls); CompletionService completionService = diff --git a/pinot-common/src/main/java/org/apache/pinot/common/http/MultiHttpRequest.java b/pinot-common/src/main/java/org/apache/pinot/common/http/MultiHttpRequest.java index a28674e2ad01..73efb2c59856 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/http/MultiHttpRequest.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/http/MultiHttpRequest.java @@ -19,6 +19,7 @@ package org.apache.pinot.common.http; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletionService; @@ -26,11 +27,14 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.function.Function; import javax.annotation.Nullable; +import org.apache.commons.lang3.tuple.Pair; import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpRequestBase; import org.apache.http.conn.HttpClientConnectionManager; +import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.client.HttpClients; @@ -66,14 +70,29 @@ public MultiHttpRequest(Executor executor, HttpClientConnectionManager connectio * @return instance of CompletionService. Completion service will provide * results as they arrive. The order is NOT same as the order of URLs */ - public CompletionService execute(List urls, + public CompletionService executeGet(List urls, @Nullable Map requestHeaders, int timeoutMs) { - return execute(urls, requestHeaders, timeoutMs, "GET", HttpGet::new); + List> urlsAndRequestBodies = new ArrayList<>(); + urls.forEach(url -> urlsAndRequestBodies.add(Pair.of(url, ""))); + return execute(urlsAndRequestBodies, requestHeaders, timeoutMs, "GET", HttpGet::new); + } + + /** + * POST urls in parallel using the executor service. + * @param urlsAndRequestBodies absolute URLs to POST + * @param requestHeaders headers to set when making the request + * @param timeoutMs timeout in milliseconds for each POST request + * @return instance of CompletionService. Completion service will provide + * results as they arrive. The order is NOT same as the order of URLs + */ + public CompletionService executePost(List> urlsAndRequestBodies, + @Nullable Map requestHeaders, int timeoutMs) { + return execute(urlsAndRequestBodies, requestHeaders, timeoutMs, "POST", HttpPost::new); } /** * Execute certain http method on the urls in parallel using the executor service. - * @param urls absolute URLs to execute the http method + * @param urlsAndRequestBodies absolute URLs to execute the http method * @param requestHeaders headers to set when making the request * @param timeoutMs timeout in milliseconds for each http request * @param httpMethodName the name of the http method like GET, DELETE etc. @@ -81,22 +100,28 @@ public CompletionService execute(List urls, * @return instance of CompletionService. Completion service will provide * results as they arrive. The order is NOT same as the order of URLs */ - public CompletionService execute(List urls, - @Nullable Map requestHeaders, int timeoutMs, String httpMethodName, - Function httpRequestBaseSupplier) { + public CompletionService execute( + List> urlsAndRequestBodies, @Nullable Map requestHeaders, int timeoutMs, + String httpMethodName, Function httpRequestBaseSupplier) { // Create global request configuration - RequestConfig defaultRequestConfig = RequestConfig.custom() - .setConnectionRequestTimeout(timeoutMs) - .setSocketTimeout(timeoutMs).build(); // setting the socket + RequestConfig defaultRequestConfig = + RequestConfig.custom().setConnectionRequestTimeout(timeoutMs).setSocketTimeout(timeoutMs) + .build(); // setting the socket - HttpClientBuilder httpClientBuilder = HttpClients.custom() - .setConnectionManager(_connectionManager).setDefaultRequestConfig(defaultRequestConfig); + HttpClientBuilder httpClientBuilder = + HttpClients.custom().setConnectionManager(_connectionManager).setDefaultRequestConfig(defaultRequestConfig); CompletionService completionService = new ExecutorCompletionService<>(_executor); CloseableHttpClient client = httpClientBuilder.build(); - for (String url : urls) { + for (Pair pair : urlsAndRequestBodies) { completionService.submit(() -> { - T httpMethod = httpRequestBaseSupplier.apply(url); + String url = pair.getLeft(); + String body = pair.getRight(); + HttpRequestBase httpMethod = httpRequestBaseSupplier.apply(url); + // If the http method is POST, set the request body + if (httpMethod instanceof HttpPost) { + ((HttpPost) httpMethod).setEntity(new StringEntity(body)); + } if (requestHeaders != null) { requestHeaders.forEach(((HttpRequestBase) httpMethod)::setHeader); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ValidDocIdMetadataInfo.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ValidDocIdMetadataInfo.java new file mode 100644 index 000000000000..475ba9143214 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ValidDocIdMetadataInfo.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.common.restlet.resources; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + + +@JsonIgnoreProperties(ignoreUnknown = true) +public class ValidDocIdMetadataInfo { + private final String _segmentName; + private final long _totalValidDocs; + private final long _totalInvalidDocs; + private final long _totalDocs; + + public ValidDocIdMetadataInfo(@JsonProperty("segmentName") String segmentName, + @JsonProperty("totalValidDocs") long totalValidDocs, @JsonProperty("totalInvalidDocs") long totalInvalidDocs, + @JsonProperty("totalDocs") long totalDocs) { + _segmentName = segmentName; + _totalValidDocs = totalValidDocs; + _totalInvalidDocs = totalInvalidDocs; + _totalDocs = totalDocs; + } + + public String getSegmentName() { + return _segmentName; + } + + public long getTotalValidDocs() { + return _totalValidDocs; + } + + public long getTotalInvalidDocs() { + return _totalInvalidDocs; + } + + public long getTotalDocs() { + return _totalDocs; + } +} diff --git a/pinot-common/src/test/java/org/apache/pinot/common/http/MultiHttpRequestTest.java b/pinot-common/src/test/java/org/apache/pinot/common/http/MultiHttpRequestTest.java index 61f58f78f651..292e2ec82eb6 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/http/MultiHttpRequestTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/http/MultiHttpRequestTest.java @@ -32,6 +32,7 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import org.apache.commons.lang3.tuple.Pair; import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; import org.apache.http.util.EntityUtils; import org.slf4j.Logger; @@ -43,6 +44,30 @@ public class MultiHttpRequestTest { + class TestResult { + private final int _success; + private final int _errors; + private final int _timeouts; + + public TestResult(int success, int errors, int timeouts) { + _success = success; + _errors = errors; + _timeouts = timeouts; + } + + public int getSuccess() { + return _success; + } + + public int getErrors() { + return _errors; + } + + public int getTimeouts() { + return _timeouts; + } + } + private static final Logger LOGGER = LoggerFactory.getLogger(MultiHttpRequest.class); private static final String SUCCESS_MSG = "success"; private static final String ERROR_MSG = "error"; @@ -61,6 +86,7 @@ public void setUpTest() startServer(_portStart, createHandler(SUCCESS_CODE, SUCCESS_MSG, 0)); startServer(_portStart + 1, createHandler(ERROR_CODE, ERROR_MSG, 0)); startServer(_portStart + 2, createHandler(SUCCESS_CODE, TIMEOUT_MSG, TIMEOUT_MS)); + startServer(_portStart + 3, createPostHandler(SUCCESS_CODE, SUCCESS_MSG, 0)); } @AfterTest @@ -90,6 +116,33 @@ public void handle(HttpExchange httpExchange) }; } + private HttpHandler createPostHandler(final int status, final String msg, final int sleepTimeMs) { + return new HttpHandler() { + @Override + public void handle(HttpExchange httpExchange) + throws IOException { + if (sleepTimeMs > 0) { + try { + Thread.sleep(sleepTimeMs); + } catch (InterruptedException e) { + LOGGER.info("Handler interrupted during sleep"); + } + } + if (httpExchange.getRequestMethod().equals("POST")) { + httpExchange.sendResponseHeaders(status, msg.length()); + OutputStream responseBody = httpExchange.getResponseBody(); + responseBody.write(msg.getBytes()); + responseBody.close(); + } else { + httpExchange.sendResponseHeaders(ERROR_CODE, ERROR_MSG.length()); + OutputStream responseBody = httpExchange.getResponseBody(); + responseBody.write(ERROR_MSG.getBytes()); + responseBody.close(); + } + } + }; + } + private void startServer(int port, HttpHandler handler) throws IOException { final HttpServer server = HttpServer.create(new InetSocketAddress(port), 0); @@ -104,22 +157,58 @@ public void run() { } @Test - public void testMultiGet() throws Exception { - MultiHttpRequest mget = - new MultiHttpRequest(Executors.newCachedThreadPool(), new PoolingHttpClientConnectionManager()); + public void testMultiGet() { List urls = Arrays.asList("http://localhost:" + String.valueOf(_portStart) + URI_PATH, "http://localhost:" + String.valueOf(_portStart + 1) + URI_PATH, "http://localhost:" + String.valueOf(_portStart + 2) + URI_PATH, // 2nd request to the same server - "http://localhost:" + String.valueOf(_portStart) + URI_PATH); + "http://localhost:" + String.valueOf(_portStart) + URI_PATH, + "http://localhost:" + String.valueOf(_portStart + 3) + URI_PATH); + + MultiHttpRequest mget = + new MultiHttpRequest(Executors.newCachedThreadPool(), new PoolingHttpClientConnectionManager()); + // timeout value needs to be less than 5000ms set above for // third server final int requestTimeoutMs = 1000; - CompletionService completionService = mget.execute(urls, null, requestTimeoutMs); + CompletionService completionService = mget.executeGet(urls, null, requestTimeoutMs); + + TestResult result = collectResult(completionService, urls.size()); + Assert.assertEquals(result.getSuccess(), 2); + Assert.assertEquals(result.getErrors(), 2); + Assert.assertEquals(result.getTimeouts(), 1); + } + + @Test + public void testMultiPost() { + List> urlsAndRequestBodies = + List.of(Pair.of("http://localhost:" + String.valueOf(_portStart) + URI_PATH, "b0"), + Pair.of("http://localhost:" + String.valueOf(_portStart + 1) + URI_PATH, "b1"), + Pair.of("http://localhost:" + String.valueOf(_portStart + 2) + URI_PATH, "b2"), + // 2nd request to the same server + Pair.of("http://localhost:" + String.valueOf(_portStart) + URI_PATH, "b3"), + Pair.of("http://localhost:" + String.valueOf(_portStart + 3) + URI_PATH, "b4")); + + MultiHttpRequest mpost = + new MultiHttpRequest(Executors.newCachedThreadPool(), new PoolingHttpClientConnectionManager()); + + // timeout value needs to be less than 5000ms set above for + // third server + final int requestTimeoutMs = 1000; + CompletionService completionService = + mpost.executePost(urlsAndRequestBodies, null, requestTimeoutMs); + + TestResult result = collectResult(completionService, urlsAndRequestBodies.size()); + Assert.assertEquals(result.getSuccess(), 3); + Assert.assertEquals(result.getErrors(), 1); + Assert.assertEquals(result.getTimeouts(), 1); + } + + private TestResult collectResult(CompletionService completionService, int size) { int success = 0; int errors = 0; int timeouts = 0; - for (int i = 0; i < urls.size(); i++) { + for (int i = 0; i < size; i++) { try (MultiHttpRequestResponse httpRequestResponse = completionService.take().get()) { if (httpRequestResponse.getResponse().getStatusLine().getStatusCode() >= 300) { errors++; @@ -143,9 +232,6 @@ public void testMultiGet() throws Exception { errors++; } } - - Assert.assertEquals(success, 2); - Assert.assertEquals(errors, 1); - Assert.assertEquals(timeouts, 1); + return new TestResult(success, errors, timeouts); } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotRunningQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotRunningQueryResource.java index 8aa004b35b65..29bdccf4ebd4 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotRunningQueryResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotRunningQueryResource.java @@ -187,7 +187,7 @@ private Map> getRunningQueries(Map completionService = - new MultiHttpRequest(_executor, _httpConnMgr).execute(brokerUrls, requestHeaders, timeoutMs); + new MultiHttpRequest(_executor, _httpConnMgr).executeGet(brokerUrls, requestHeaders, timeoutMs); Map> queriesByBroker = new HashMap<>(); List errMsgs = new ArrayList<>(brokerUrls.size()); for (int i = 0; i < brokerUrls.size(); i++) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java index 5d35e1f0c3d2..d7894c08f7df 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java @@ -950,6 +950,43 @@ public String getTableAggregateMetadata( return segmentsMetadata; } + @GET + @Path("tables/{tableName}/validDocIdMetadata") + @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.GET_METADATA) + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Get the aggregate valid doc id metadata of all segments for a table", notes = "Get the " + + "aggregate valid doc id metadata of all segments for a table") + public String getTableAggregateValidDocIdMetadata( + @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName, + @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr, + @ApiParam(value = "A list of segments", allowMultiple = true) @QueryParam("segmentNames") + List segmentNames) { + LOGGER.info("Received a request to fetch aggregate valid doc id metadata for a table {}", tableName); + TableType tableType = Constants.validateTableType(tableTypeStr); + if (tableType == TableType.OFFLINE) { + throw new ControllerApplicationException(LOGGER, "Table type : " + tableTypeStr + " not yet supported.", + Response.Status.NOT_IMPLEMENTED); + } + String tableNameWithType = + ResourceUtils.getExistingTableNamesWithType(_pinotHelixResourceManager, tableName, tableType, LOGGER).get(0); + + String validDocIdMetadata; + try { + TableMetadataReader tableMetadataReader = + new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager); + JsonNode segmentsMetadataJson = + tableMetadataReader.getAggregateValidDocIdMetadata(tableNameWithType, segmentNames, + _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000); + validDocIdMetadata = JsonUtils.objectToPrettyString(segmentsMetadataJson); + } catch (InvalidConfigException e) { + throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST); + } catch (IOException ioe) { + throw new ControllerApplicationException(LOGGER, "Error parsing Pinot server response: " + ioe.getMessage(), + Response.Status.INTERNAL_SERVER_ERROR, ioe); + } + return validDocIdMetadata; + } + @GET @Path("tables/{tableName}/indexes") @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.GET_METADATA) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/CompletionServiceHelper.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/CompletionServiceHelper.java index d26f5ba60353..f0b36b2cfa1f 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/CompletionServiceHelper.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/CompletionServiceHelper.java @@ -27,6 +27,7 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.Executor; import javax.annotation.Nullable; +import org.apache.commons.lang3.tuple.Pair; import org.apache.http.conn.HttpClientConnectionManager; import org.apache.http.util.EntityUtils; import org.apache.pinot.common.http.MultiHttpRequest; @@ -79,12 +80,46 @@ public CompletionServiceResponse doMultiGetRequest(List serverURLs, Stri public CompletionServiceResponse doMultiGetRequest(List serverURLs, String tableNameWithType, boolean multiRequestPerServer, @Nullable Map requestHeaders, int timeoutMs, @Nullable String useCase) { - CompletionServiceResponse completionServiceResponse = new CompletionServiceResponse(); - // TODO: use some service other than completion service so that we know which server encounters the error CompletionService completionService = - new MultiHttpRequest(_executor, _httpConnectionManager).execute(serverURLs, requestHeaders, timeoutMs); - for (int i = 0; i < serverURLs.size(); i++) { + new MultiHttpRequest(_executor, _httpConnectionManager).executeGet(serverURLs, requestHeaders, timeoutMs); + + return collectResponse(tableNameWithType, serverURLs.size(), completionService, multiRequestPerServer, useCase); + } + + /** + * This method makes a MultiPost call to all given URLs and its corresponding bodies. + * @param serverURLsAndRequestBodies server urls to send GET request. + * @param tableNameWithType table name with type suffix + * @param multiRequestPerServer it's possible that need to send multiple requests to a same server. + * If multiRequestPerServer is set as false, return as long as one of the requests get + * response; If multiRequestPerServer is set as true, wait until all requests + * get response. + * @param requestHeaders Headers to be set when making the http calls. + * @param timeoutMs timeout in milliseconds to wait per request. + * @param useCase the use case initiating the multi-get request. If not null and an exception is thrown, only the + * error message and the use case are logged instead of the full stack trace. + * @return CompletionServiceResponse Map of the endpoint(server instance, or full request path if + * multiRequestPerServer is true) to the response from that endpoint. + */ + public CompletionServiceResponse doMultiPostRequest(List> serverURLsAndRequestBodies, + String tableNameWithType, boolean multiRequestPerServer, @Nullable Map requestHeaders, + int timeoutMs, @Nullable String useCase) { + + CompletionService completionService = + new MultiHttpRequest(_executor, _httpConnectionManager).executePost(serverURLsAndRequestBodies, requestHeaders, + timeoutMs); + + return collectResponse(tableNameWithType, serverURLsAndRequestBodies.size(), completionService, + multiRequestPerServer, useCase); + } + + private CompletionServiceResponse collectResponse(String tableNameWithType, int size, + CompletionService completionService, boolean multiRequestPerServer, + @Nullable String useCase) { + CompletionServiceResponse completionServiceResponse = new CompletionServiceResponse(); + + for (int i = 0; i < size; i++) { MultiHttpRequestResponse multiHttpRequestResponse = null; try { multiHttpRequestResponse = completionService.take().get(); @@ -93,7 +128,8 @@ public CompletionServiceResponse doMultiGetRequest(List serverURLs, Stri _endpointsToServers.get(String.format("%s://%s:%d", uri.getScheme(), uri.getHost(), uri.getPort())); int statusCode = multiHttpRequestResponse.getResponse().getStatusLine().getStatusCode(); if (statusCode >= 300) { - LOGGER.error("Server: {} returned error: {}", instance, statusCode); + String reason = multiHttpRequestResponse.getResponse().getStatusLine().getReasonPhrase(); + LOGGER.error("Server: {} returned error: {}, reason: {}", instance, statusCode, reason); completionServiceResponse._failedResponseCount++; continue; } @@ -102,7 +138,7 @@ public CompletionServiceResponse doMultiGetRequest(List serverURLs, Stri .put(multiRequestPerServer ? uri.toString() : instance, responseString); } catch (Exception e) { String reason = useCase == null ? "" : String.format(" in '%s'", useCase); - LOGGER.error("Connection error{}. Details: {}", reason, e.getMessage()); + LOGGER.error("Connection error {}. Details: {}", reason, e.getMessage()); completionServiceResponse._failedResponseCount++; } finally { if (multiHttpRequestResponse != null) { @@ -116,9 +152,9 @@ public CompletionServiceResponse doMultiGetRequest(List serverURLs, Stri } int numServersResponded = completionServiceResponse._httpResponses.size(); - if (numServersResponded != serverURLs.size()) { + if (numServersResponded != size) { LOGGER.warn("Finished reading information for table: {} with {}/{} server responses", tableNameWithType, - numServersResponded, serverURLs.size()); + numServersResponded, size); } else { LOGGER.info("Finished reading information for table: {}", tableNameWithType); } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java index a51881d9e3e0..de1cae193ce4 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java @@ -18,19 +18,35 @@ */ package org.apache.pinot.controller.util; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import javax.annotation.Nullable; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.core.Response; +import org.apache.commons.lang3.tuple.Pair; import org.apache.http.conn.HttpClientConnectionManager; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; +import org.apache.pinot.common.restlet.resources.TableSegments; +import org.apache.pinot.common.restlet.resources.ValidDocIdMetadataInfo; +import org.apache.pinot.common.utils.RoaringBitmapUtils; import org.apache.pinot.spi.utils.JsonUtils; +import org.glassfish.jersey.client.ClientConfig; +import org.glassfish.jersey.client.ClientProperties; +import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +63,11 @@ public class ServerSegmentMetadataReader { private final Executor _executor; private final HttpClientConnectionManager _connectionManager; + public ServerSegmentMetadataReader() { + _executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + _connectionManager = new PoolingHttpClientConnectionManager(); + } + public ServerSegmentMetadataReader(Executor executor, HttpClientConnectionManager connectionManager) { _executor = executor; _connectionManager = connectionManager; @@ -100,11 +121,11 @@ public TableMetadataInfo getAggregatedTableMetadataFromServer(String tableNameWi tableMetadataInfo.getColumnCardinalityMap().forEach((k, v) -> columnCardinalityMap.merge(k, v, Double::sum)); tableMetadataInfo.getMaxNumMultiValuesMap().forEach((k, v) -> maxNumMultiValuesMap.merge(k, v, Double::sum)); tableMetadataInfo.getColumnIndexSizeMap().forEach((k, v) -> columnIndexSizeMap.merge(k, v, (l, r) -> { - for (Map.Entry e : r.entrySet()) { - l.put(e.getKey(), l.getOrDefault(e.getKey(), 0d) + e.getValue()); - } - return l; - })); + for (Map.Entry e : r.entrySet()) { + l.put(e.getKey(), l.getOrDefault(e.getKey(), 0d) + e.getValue()); + } + return l; + })); } catch (IOException e) { failedParses++; LOGGER.error("Unable to parse server {} response due to an error: ", streamResponse.getKey(), e); @@ -180,27 +201,129 @@ public List getSegmentMetadataFromServer(String tableNameWithType, return segmentsMetadata; } + /** + * This method is called when the API request is to fetch validDocId metadata for a list segments of the given table. + * This method will pick a server that hosts the target segment and fetch the segment metadata result. + * + * @return segment metadata as a JSON string + */ + public List getValidDocIdMetadataFromServer(String tableNameWithType, + Map> serverToSegmentsMap, BiMap serverToEndpoints, + @Nullable List segmentNames, int timeoutMs) { + List> serverURLsAndBodies = new ArrayList<>(); + for (Map.Entry> serverToSegments : serverToSegmentsMap.entrySet()) { + List segmentsForServer = serverToSegments.getValue(); + List segmentsToQuery = new ArrayList<>(); + if (segmentNames == null || segmentNames.isEmpty()) { + segmentsToQuery.addAll(segmentsForServer); + } else { + Set segmentNamesLookUpTable = new HashSet<>(segmentNames); + for (String segment : segmentsForServer) { + if (segmentNamesLookUpTable.contains(segment)) { + segmentsToQuery.add(segment); + } + } + } + serverURLsAndBodies.add(generateValidDocIdMetadataURL(tableNameWithType, segmentsToQuery, + serverToEndpoints.get(serverToSegments.getKey()))); + } + + // request the urls from the servers + CompletionServiceHelper completionServiceHelper = + new CompletionServiceHelper(_executor, _connectionManager, serverToEndpoints); + + Map requestHeaders = Map.of("Content-Type", "application/json"); + CompletionServiceHelper.CompletionServiceResponse serviceResponse = + completionServiceHelper.doMultiPostRequest(serverURLsAndBodies, tableNameWithType, false, requestHeaders, + timeoutMs, null); + + List validDocIdMetadataInfos = new ArrayList<>(); + int failedParses = 0; + int returnedSegmentsCount = 0; + for (Map.Entry streamResponse : serviceResponse._httpResponses.entrySet()) { + try { + String validDocIdMetadataList = streamResponse.getValue(); + List validDocIdMetadataInfo = + JsonUtils.stringToObject(validDocIdMetadataList, new TypeReference>() { + }); + validDocIdMetadataInfos.addAll(validDocIdMetadataInfo); + returnedSegmentsCount++; + } catch (Exception e) { + failedParses++; + LOGGER.error("Unable to parse server {} response due to an error: ", streamResponse.getKey(), e); + } + } + if (failedParses != 0) { + LOGGER.error("Unable to parse server {} / {} response due to an error: ", failedParses, + serverURLsAndBodies.size()); + } + + if (segmentNames != null && returnedSegmentsCount != segmentNames.size()) { + LOGGER.error("Unable to get validDocIdMetadata from all servers. Expected: {}, Actual: {}", segmentNames.size(), + returnedSegmentsCount); + } + LOGGER.info("Retrieved valid doc id metadata for {} segments from {} servers.", returnedSegmentsCount, + serverURLsAndBodies.size()); + return validDocIdMetadataInfos; + } + + /** + * This method is called when the API request is to fetch validDocIds for a segment of the given table. This method + * will pick a server that hosts the target segment and fetch the validDocIds result. + * + * @return a bitmap of validDocIds + */ + public RoaringBitmap getValidDocIdsFromServer(String tableNameWithType, String segmentName, String endpoint, + int timeoutMs) { + // Build the endpoint url + String url = generateValidDocIdsURL(tableNameWithType, segmentName, endpoint); + + // Set timeout + ClientConfig clientConfig = new ClientConfig(); + clientConfig.property(ClientProperties.CONNECT_TIMEOUT, timeoutMs); + clientConfig.property(ClientProperties.READ_TIMEOUT, timeoutMs); + + Response response = ClientBuilder.newClient(clientConfig).target(url).request().get(Response.class); + Preconditions.checkState(response.getStatus() == Response.Status.OK.getStatusCode(), + "Unable to retrieve validDocIds from %s", url); + byte[] validDocIds = response.readEntity(byte[].class); + return RoaringBitmapUtils.deserialize(validDocIds); + } + private String generateAggregateSegmentMetadataServerURL(String tableNameWithType, List columns, String endpoint) { - try { - tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8.name()); - String paramsStr = generateColumnsParam(columns); - return String.format("%s/tables/%s/metadata?%s", endpoint, tableNameWithType, paramsStr); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e.getCause()); - } + tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8); + String paramsStr = generateColumnsParam(columns); + return String.format("%s/tables/%s/metadata?%s", endpoint, tableNameWithType, paramsStr); } private String generateSegmentMetadataServerURL(String tableNameWithType, String segmentName, List columns, String endpoint) { + tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8); + segmentName = URLEncoder.encode(segmentName, StandardCharsets.UTF_8); + String paramsStr = generateColumnsParam(columns); + return String.format("%s/tables/%s/segments/%s/metadata?%s", endpoint, tableNameWithType, segmentName, paramsStr); + } + + private String generateValidDocIdsURL(String tableNameWithType, String segmentName, String endpoint) { + tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8); + segmentName = URLEncoder.encode(segmentName, StandardCharsets.UTF_8); + return String.format("%s/segments/%s/%s/validDocIds", endpoint, tableNameWithType, segmentName); + } + + private Pair generateValidDocIdMetadataURL(String tableNameWithType, List segmentNames, + String endpoint) { + tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8); + TableSegments tableSegments = new TableSegments(segmentNames); + String jsonTableSegments; try { - tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8.name()); - segmentName = URLEncoder.encode(segmentName, StandardCharsets.UTF_8.name()); - String paramsStr = generateColumnsParam(columns); - return String.format("%s/tables/%s/segments/%s/metadata?%s", endpoint, tableNameWithType, segmentName, paramsStr); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e.getCause()); + jsonTableSegments = JsonUtils.objectToString(tableSegments); + } catch (JsonProcessingException e) { + LOGGER.error("Failed to convert segment names to json request body: segmentNames={}", segmentNames); + throw new RuntimeException(e); } + return Pair.of( + String.format("%s/tables/%s/validDocIdMetadata", endpoint, tableNameWithType), jsonTableSegments); } private String generateColumnsParam(List columns) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java index 7ef5302c1b11..f92747b49764 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java @@ -31,6 +31,7 @@ import org.apache.http.conn.HttpClientConnectionManager; import org.apache.pinot.common.exception.InvalidConfigException; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdMetadataInfo; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; import org.apache.pinot.spi.utils.JsonUtils; @@ -81,8 +82,9 @@ private JsonNode getSegmentsMetadataInternal(String tableNameWithType, List segmentsMetadata = serverSegmentMetadataReader - .getSegmentMetadataFromServer(tableNameWithType, serverToSegmentsMap, endpoints, columns, timeoutMs); + List segmentsMetadata = + serverSegmentMetadataReader.getSegmentMetadataFromServer(tableNameWithType, serverToSegmentsMap, endpoints, + columns, timeoutMs); Map response = new HashMap<>(); for (String segmentMetadata : segmentsMetadata) { JsonNode responseJson = JsonUtils.stringToJsonNode(segmentMetadata); @@ -146,8 +148,28 @@ public JsonNode getAggregateTableMetadata(String tableNameWithType, List ServerSegmentMetadataReader serverSegmentMetadataReader = new ServerSegmentMetadataReader(_executor, _connectionManager); - TableMetadataInfo aggregateTableMetadataInfo = serverSegmentMetadataReader - .getAggregatedTableMetadataFromServer(tableNameWithType, endpoints, columns, numReplica, timeoutMs); + TableMetadataInfo aggregateTableMetadataInfo = + serverSegmentMetadataReader.getAggregatedTableMetadataFromServer(tableNameWithType, endpoints, columns, + numReplica, timeoutMs); + return JsonUtils.objectToJsonNode(aggregateTableMetadataInfo); + } + + /** + * This method retrieves the aggregated valid doc id metadata for a given table. + * @return a list of ValidDocIdMetadataInfo + */ + public JsonNode getAggregateValidDocIdMetadata(String tableNameWithType, List segmentNames, int timeoutMs) + throws InvalidConfigException { + final Map> serverToSegments = + _pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType); + BiMap endpoints = + _pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet()); + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_executor, _connectionManager); + + List aggregateTableMetadataInfo = + serverSegmentMetadataReader.getValidDocIdMetadataFromServer(tableNameWithType, serverToSegments, endpoints, + segmentNames, timeoutMs); return JsonUtils.objectToJsonNode(aggregateTableMetadataInfo); } } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java index a46f903ed91d..31f5d6039c0e 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java @@ -21,15 +21,23 @@ import java.net.URI; import java.util.HashMap; import java.util.Map; +import org.apache.helix.HelixAdmin; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.InstanceConfig; +import org.apache.pinot.common.utils.config.InstanceUtils; import org.apache.pinot.controller.helix.core.minion.ClusterInfoAccessor; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.minion.MinionContext; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.filesystem.LocalPinotFS; import org.apache.pinot.spi.filesystem.PinotFS; import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.plugin.PluginManager; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,4 +138,37 @@ public static String normalizeDirectoryURI(String dirInStr) { } return dirInStr; } + + public static RoaringBitmap getValidDocIds(String tableNameWithType, String segmentName, Map configs, + MinionContext minionContext) { + HelixAdmin helixAdmin = minionContext.getHelixManager().getClusterManagmentTool(); + String clusterName = minionContext.getHelixManager().getClusterName(); + + String server = getServer(segmentName, tableNameWithType, helixAdmin, clusterName); + InstanceConfig instanceConfig = helixAdmin.getInstanceConfig(clusterName, server); + String endpoint = InstanceUtils.getServerAdminEndpoint(instanceConfig); + + // We only need aggregated table size and the total number of docs/rows. Skipping column related stats, by + // passing an empty list. + ServerSegmentMetadataReader serverSegmentMetadataReader = new ServerSegmentMetadataReader(); + return serverSegmentMetadataReader.getValidDocIdsFromServer(tableNameWithType, segmentName, endpoint, 60_000); + } + + public static String getServer(String segmentName, String tableNameWithType, HelixAdmin helixAdmin, + String clusterName) { + ExternalView externalView = helixAdmin.getResourceExternalView(clusterName, tableNameWithType); + if (externalView == null) { + throw new IllegalStateException("External view does not exist for table: " + tableNameWithType); + } + Map instanceStateMap = externalView.getStateMap(segmentName); + if (instanceStateMap == null) { + throw new IllegalStateException("Failed to find segment: " + segmentName); + } + for (Map.Entry entry : instanceStateMap.entrySet()) { + if (entry.getValue().equals(CommonConstants.Helix.StateModel.SegmentStateModel.ONLINE)) { + return entry.getKey(); + } + } + throw new IllegalStateException("Failed to find ONLINE server for segment: " + segmentName); + } } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutor.java index aa37ac871a15..4c200b9606b4 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutor.java @@ -18,118 +18,28 @@ */ package org.apache.pinot.plugin.minion.tasks.upsertcompaction; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; -import java.util.Set; -import javax.annotation.Nullable; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.core.Response; import org.apache.commons.io.FileUtils; -import org.apache.helix.HelixAdmin; -import org.apache.helix.HelixManager; -import org.apache.helix.model.ExternalView; -import org.apache.helix.model.InstanceConfig; -import org.apache.http.client.utils.URIBuilder; import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; -import org.apache.pinot.common.utils.config.InstanceUtils; import org.apache.pinot.core.common.MinionConstants; import org.apache.pinot.core.minion.PinotTaskConfig; import org.apache.pinot.plugin.minion.tasks.BaseSingleSegmentConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; -import org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.data.readers.RecordReader; -import org.apache.pinot.spi.data.readers.RecordReaderConfig; -import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; -import org.roaringbitmap.PeekableIntIterator; -import org.roaringbitmap.buffer.ImmutableRoaringBitmap; +import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class UpsertCompactionTaskExecutor extends BaseSingleSegmentConversionExecutor { private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskExecutor.class); - private static HelixManager _helixManager = MINION_CONTEXT.getHelixManager(); - private static HelixAdmin _clusterManagementTool = _helixManager.getClusterManagmentTool(); - private static String _clusterName = _helixManager.getClusterName(); - - private class CompactedRecordReader implements RecordReader { - private final PinotSegmentRecordReader _pinotSegmentRecordReader; - private final PeekableIntIterator _validDocIdsIterator; - // Reusable generic row to store the next row to return - GenericRow _nextRow = new GenericRow(); - // Flag to mark whether we need to fetch another row - boolean _nextRowReturned = true; - - CompactedRecordReader(File indexDir, ImmutableRoaringBitmap validDocIds) { - _pinotSegmentRecordReader = new PinotSegmentRecordReader(); - _pinotSegmentRecordReader.init(indexDir, null, null); - _validDocIdsIterator = validDocIds.getIntIterator(); - } - - @Override - public void init(File dataFile, Set fieldsToRead, @Nullable RecordReaderConfig recordReaderConfig) { - } - - @Override - public boolean hasNext() { - if (!_validDocIdsIterator.hasNext() && _nextRowReturned) { - return false; - } - - // If next row has not been returned, return true - if (!_nextRowReturned) { - return true; - } - - // Try to get the next row to return - if (_validDocIdsIterator.hasNext()) { - int docId = _validDocIdsIterator.next(); - _nextRow.clear(); - _pinotSegmentRecordReader.getRecord(docId, _nextRow); - _nextRowReturned = false; - return true; - } - - // Cannot find next row to return, return false - return false; - } - - @Override - public GenericRow next() { - return next(new GenericRow()); - } - - @Override - public GenericRow next(GenericRow reuse) { - Preconditions.checkState(!_nextRowReturned); - reuse.init(_nextRow); - _nextRowReturned = true; - return reuse; - } - - @Override - public void rewind() { - _pinotSegmentRecordReader.rewind(); - _nextRowReturned = true; - } - - @Override - public void close() - throws IOException { - _pinotSegmentRecordReader.close(); - } - } @Override protected SegmentConversionResult convert(PinotTaskConfig pinotTaskConfig, File indexDir, File workingDir) @@ -143,7 +53,7 @@ protected SegmentConversionResult convert(PinotTaskConfig pinotTaskConfig, File String tableNameWithType = configs.get(MinionConstants.TABLE_NAME_KEY); TableConfig tableConfig = getTableConfig(tableNameWithType); - ImmutableRoaringBitmap validDocIds = getValidDocIds(tableNameWithType, configs); + RoaringBitmap validDocIds = MinionTaskUtils.getValidDocIds(tableNameWithType, segmentName, configs, MINION_CONTEXT); if (validDocIds.isEmpty()) { // prevents empty segment generation @@ -159,7 +69,8 @@ protected SegmentConversionResult convert(PinotTaskConfig pinotTaskConfig, File } SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); - try (CompactedRecordReader compactedRecordReader = new CompactedRecordReader(indexDir, validDocIds)) { + try (CompactedPinotSegmentRecordReader compactedRecordReader = new CompactedPinotSegmentRecordReader(indexDir, + validDocIds)) { SegmentGeneratorConfig config = getSegmentGeneratorConfig(workingDir, tableConfig, segmentMetadata, segmentName); SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); driver.init(config, compactedRecordReader); @@ -198,46 +109,6 @@ private static SegmentGeneratorConfig getSegmentGeneratorConfig(File workingDir, return config; } - // TODO: Consider moving this method to a more appropriate class (eg ServerSegmentMetadataReader) - private static ImmutableRoaringBitmap getValidDocIds(String tableNameWithType, Map configs) - throws URISyntaxException { - String segmentName = configs.get(MinionConstants.SEGMENT_NAME_KEY); - String server = getServer(segmentName, tableNameWithType); - - // get the url for the validDocIds for the server - InstanceConfig instanceConfig = _clusterManagementTool.getInstanceConfig(_clusterName, server); - String endpoint = InstanceUtils.getServerAdminEndpoint(instanceConfig); - String url = - new URIBuilder(endpoint).setPath(String.format("/segments/%s/%s/validDocIds", tableNameWithType, segmentName)) - .toString(); - - // get the validDocIds from that server - Response response = ClientBuilder.newClient().target(url).request().get(Response.class); - Preconditions.checkState(response.getStatus() == Response.Status.OK.getStatusCode(), - "Unable to retrieve validDocIds from %s", url); - byte[] snapshot = response.readEntity(byte[].class); - ImmutableRoaringBitmap validDocIds = new ImmutableRoaringBitmap(ByteBuffer.wrap(snapshot)); - return validDocIds; - } - - @VisibleForTesting - public static String getServer(String segmentName, String tableNameWithType) { - ExternalView externalView = _clusterManagementTool.getResourceExternalView(_clusterName, tableNameWithType); - if (externalView == null) { - throw new IllegalStateException("External view does not exist for table: " + tableNameWithType); - } - Map instanceStateMap = externalView.getStateMap(segmentName); - if (instanceStateMap == null) { - throw new IllegalStateException("Failed to find segment: " + segmentName); - } - for (Map.Entry entry : instanceStateMap.entrySet()) { - if (entry.getValue().equals(SegmentStateModel.ONLINE)) { - return entry.getKey(); - } - } - throw new IllegalStateException("Failed to find ONLINE server for segment: " + segmentName); - } - @Override protected SegmentZKMetadataCustomMapModifier getSegmentZKMetadataCustomMapModifier(PinotTaskConfig pinotTaskConfig, SegmentConversionResult segmentConversionResult) { diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java index 590102319e6d..e6eaf3679ea6 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java @@ -18,26 +18,20 @@ */ package org.apache.pinot.plugin.minion.tasks.upsertcompaction; -import com.fasterxml.jackson.databind.JsonNode; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.BiMap; -import java.io.IOException; -import java.net.URISyntaxException; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.http.client.utils.URIBuilder; import org.apache.pinot.common.exception.InvalidConfigException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdMetadataInfo; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; -import org.apache.pinot.controller.util.CompletionServiceHelper; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; import org.apache.pinot.core.common.MinionConstants; import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask; import org.apache.pinot.core.minion.PinotTaskConfig; @@ -45,7 +39,6 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.TimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -89,6 +82,7 @@ public List generateTasks(List tableConfigs) { List pinotTaskConfigs = new ArrayList<>(); for (TableConfig tableConfig : tableConfigs) { if (!validate(tableConfig)) { + LOGGER.warn("Validation failed for table {}. Skipping..", tableConfig.getTableName()); continue; } @@ -103,6 +97,8 @@ public List generateTasks(List tableConfigs) { continue; } + // TODO: add a check to see if the task is already running for the table + // get server to segment mappings PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager(); Map> serverToSegments = pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType); @@ -113,27 +109,21 @@ public List generateTasks(List tableConfigs) { throw new RuntimeException(e); } - Map completedSegmentsMap = - completedSegments.stream().collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity())); + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_clusterInfoAccessor.getExecutor(), + _clusterInfoAccessor.getConnectionManager()); - List validDocIdUrls; - try { - validDocIdUrls = getValidDocIdMetadataUrls(serverToSegments, serverToEndpoints, tableNameWithType, - completedSegmentsMap.keySet()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } + // TODO: currently, we put segmentNames=null to get metadata for all segments. We can change this to get + // valid doc id metadata in batches with the loop. + List validDocIdMetadataList = + serverSegmentMetadataReader.getValidDocIdMetadataFromServer(tableNameWithType, serverToSegments, + serverToEndpoints, null, 60_000); - // request the urls from the servers - CompletionServiceHelper completionServiceHelper = - new CompletionServiceHelper(_clusterInfoAccessor.getExecutor(), _clusterInfoAccessor.getConnectionManager(), - serverToEndpoints.inverse()); - - CompletionServiceHelper.CompletionServiceResponse serviceResponse = - completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000); + Map completedSegmentsMap = + completedSegments.stream().collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity())); SegmentSelectionResult segmentSelectionResult = - processValidDocIdMetadata(taskConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet()); + processValidDocIdMetadata(taskConfigs, completedSegmentsMap, validDocIdMetadataList); if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) { pinotHelixResourceManager.deleteSegments(tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), @@ -163,7 +153,7 @@ public List generateTasks(List tableConfigs) { @VisibleForTesting public static SegmentSelectionResult processValidDocIdMetadata(Map taskConfigs, - Map completedSegmentsMap, Set> responseSet) { + Map completedSegmentsMap, List validDocIdMetadataInfoList) { double invalidRecordsThresholdPercent = Double.parseDouble( taskConfigs.getOrDefault(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT, String.valueOf(DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT))); @@ -172,62 +162,22 @@ public static SegmentSelectionResult processValidDocIdMetadata(Map segmentsForCompaction = new ArrayList<>(); List segmentsForDeletion = new ArrayList<>(); - for (Map.Entry streamResponse : responseSet) { - JsonNode allValidDocIdMetadata; - try { - allValidDocIdMetadata = JsonUtils.stringToJsonNode(streamResponse.getValue()); - } catch (IOException e) { - LOGGER.error("Unable to parse validDocIdMetadata response for: {}", streamResponse.getKey()); - continue; - } - Iterator iterator = allValidDocIdMetadata.elements(); - while (iterator.hasNext()) { - JsonNode validDocIdMetadata = iterator.next(); - long totalInvalidDocs = validDocIdMetadata.get("totalInvalidDocs").asLong(); - String segmentName = validDocIdMetadata.get("segmentName").asText(); - SegmentZKMetadata segment = completedSegmentsMap.get(segmentName); - long totalDocs = validDocIdMetadata.get("totalDocs").asLong(); - double invalidRecordPercent = ((double) totalInvalidDocs / totalDocs) * 100; - if (totalInvalidDocs == totalDocs) { - segmentsForDeletion.add(segment.getSegmentName()); - } else if (invalidRecordPercent > invalidRecordsThresholdPercent - && totalInvalidDocs > invalidRecordsThresholdCount) { - segmentsForCompaction.add(segment); - } + for (ValidDocIdMetadataInfo validDocIdMetadata : validDocIdMetadataInfoList) { + long totalInvalidDocs = validDocIdMetadata.getTotalInvalidDocs(); + String segmentName = validDocIdMetadata.getSegmentName(); + SegmentZKMetadata segment = completedSegmentsMap.get(segmentName); + long totalDocs = validDocIdMetadata.getTotalDocs(); + double invalidRecordPercent = ((double) totalInvalidDocs / totalDocs) * 100; + if (totalInvalidDocs == totalDocs) { + segmentsForDeletion.add(segment.getSegmentName()); + } else if (invalidRecordPercent > invalidRecordsThresholdPercent + && totalInvalidDocs > invalidRecordsThresholdCount) { + segmentsForCompaction.add(segment); } } return new SegmentSelectionResult(segmentsForCompaction, segmentsForDeletion); } - @VisibleForTesting - public static List getValidDocIdMetadataUrls(Map> serverToSegments, - BiMap serverToEndpoints, String tableNameWithType, Set completedSegments) - throws URISyntaxException { - Set remainingSegments = new HashSet<>(completedSegments); - List urls = new ArrayList<>(); - for (Map.Entry> entry : serverToSegments.entrySet()) { - if (remainingSegments.isEmpty()) { - break; - } - String server = entry.getKey(); - List segmentNames = entry.getValue(); - URIBuilder uriBuilder = new URIBuilder(serverToEndpoints.get(server)).setPath( - String.format("/tables/%s/validDocIdMetadata", tableNameWithType)); - int completedSegmentCountPerServer = 0; - for (String segmentName : segmentNames) { - if (remainingSegments.remove(segmentName)) { - completedSegmentCountPerServer++; - uriBuilder.addParameter("segmentNames", segmentName); - } - } - if (completedSegmentCountPerServer > 0) { - // only add to the list if the server has completed segments - urls.add(uriBuilder.toString()); - } - } - return urls; - } - private List getCompletedSegments(String tableNameWithType, Map taskConfigs) { List completedSegments = new ArrayList<>(); String bufferPeriod = taskConfigs.getOrDefault(UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY, DEFAULT_BUFFER_PERIOD); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutorTest.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutorTest.java index 604c58f6d03d..0869880ccf51 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutorTest.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskExecutorTest.java @@ -24,6 +24,7 @@ import org.apache.helix.HelixManager; import org.apache.helix.model.ExternalView; import org.apache.pinot.minion.MinionContext; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.mockito.Mockito; import org.testng.Assert; @@ -51,13 +52,15 @@ public void testGetServer() { Mockito.when(helixManager.getClusterManagmentTool()).thenReturn(clusterManagementTool); minionContext.setHelixManager(helixManager); - String server = UpsertCompactionTaskExecutor.getServer(SEGMENT_NAME, REALTIME_TABLE_NAME); + String server = MinionTaskUtils.getServer(SEGMENT_NAME, REALTIME_TABLE_NAME, helixManager.getClusterManagmentTool(), + helixManager.getClusterName()); Assert.assertEquals(server, "server1"); // verify exception thrown with OFFLINE server map.put("server1", SegmentStateModel.OFFLINE); Assert.assertThrows(IllegalStateException.class, - () -> UpsertCompactionTaskExecutor.getServer(SEGMENT_NAME, REALTIME_TABLE_NAME)); + () -> MinionTaskUtils.getServer(SEGMENT_NAME, REALTIME_TABLE_NAME, + helixManager.getClusterManagmentTool(), helixManager.getClusterName())); } } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGeneratorTest.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGeneratorTest.java index 03908a958eab..7da9a7f1e86b 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGeneratorTest.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/test/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGeneratorTest.java @@ -18,20 +18,17 @@ */ package org.apache.pinot.plugin.minion.tasks.upsertcompaction; -import com.google.common.collect.BiMap; -import com.google.common.collect.HashBiMap; -import java.net.URISyntaxException; -import java.util.AbstractMap; +import com.fasterxml.jackson.core.type.TypeReference; +import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdMetadataInfo; import org.apache.pinot.controller.helix.core.minion.ClusterInfoAccessor; import org.apache.pinot.core.common.MinionConstants; import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask; @@ -41,6 +38,7 @@ import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.config.table.UpsertConfig; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.TimeUtils; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.testng.annotations.BeforeClass; @@ -171,57 +169,6 @@ public void testGenerateTasksWithNewlyCompletedSegment() { assertEquals(pinotTaskConfigs.size(), 0); } - @Test - public void testGetValidDocIdMetadataUrls() - throws URISyntaxException { - Map> serverToSegments = new HashMap<>(); - serverToSegments.put("server1", - Lists.newArrayList(_completedSegment.getSegmentName(), _completedSegment2.getSegmentName())); - serverToSegments.put("server2", Lists.newArrayList("consumingSegment")); - BiMap serverToEndpoints = HashBiMap.create(1); - serverToEndpoints.put("server1", "http://endpoint1"); - serverToEndpoints.put("server2", "http://endpoint2"); - Set completedSegments = new HashSet<>(); - completedSegments.add(_completedSegment.getSegmentName()); - completedSegments.add(_completedSegment2.getSegmentName()); - - List validDocIdUrls = - UpsertCompactionTaskGenerator.getValidDocIdMetadataUrls(serverToSegments, serverToEndpoints, - REALTIME_TABLE_NAME, completedSegments); - - String expectedUrl = - String.format("%s/tables/%s/validDocIdMetadata?segmentNames=%s&segmentNames=%s", "http://endpoint1", - REALTIME_TABLE_NAME, _completedSegment.getSegmentName(), _completedSegment2.getSegmentName()); - assertEquals(validDocIdUrls.get(0), expectedUrl); - assertEquals(validDocIdUrls.size(), 1); - } - - @Test - public void testGetValidDocIdMetadataUrlsWithReplicatedSegments() - throws URISyntaxException { - Map> serverToSegments = new LinkedHashMap<>(); - serverToSegments.put("server1", - Lists.newArrayList(_completedSegment.getSegmentName(), _completedSegment2.getSegmentName())); - serverToSegments.put("server2", - Lists.newArrayList(_completedSegment.getSegmentName(), _completedSegment2.getSegmentName())); - BiMap serverToEndpoints = HashBiMap.create(1); - serverToEndpoints.put("server1", "http://endpoint1"); - serverToEndpoints.put("server2", "http://endpoint2"); - Set completedSegments = new HashSet<>(); - completedSegments.add(_completedSegment.getSegmentName()); - completedSegments.add(_completedSegment2.getSegmentName()); - - List validDocIdUrls = - UpsertCompactionTaskGenerator.getValidDocIdMetadataUrls(serverToSegments, serverToEndpoints, - REALTIME_TABLE_NAME, completedSegments); - - String expectedUrl = - String.format("%s/tables/%s/validDocIdMetadata?segmentNames=%s&segmentNames=%s", "http://endpoint1", - REALTIME_TABLE_NAME, _completedSegment.getSegmentName(), _completedSegment2.getSegmentName()); - assertEquals(validDocIdUrls.get(0), expectedUrl); - assertEquals(validDocIdUrls.size(), 1); - } - @Test public void testGetMaxTasks() { Map taskConfigs = new HashMap<>(); @@ -234,16 +181,20 @@ public void testGetMaxTasks() { } @Test - public void testProcessValidDocIdMetadata() { + public void testProcessValidDocIdMetadata() + throws IOException { Map compactionConfigs = getCompactionConfigs("1", "10"); - Set> responseSet = new HashSet<>(); + List validDocIdMetadataInfoList = new ArrayList<>(); String json = "[{" + "\"totalValidDocs\" : 50," + "\"totalInvalidDocs\" : 50," + "\"segmentName\" : \"" + _completedSegment.getSegmentName() + "\"," + "\"totalDocs\" : 100" + "}," + "{" + "\"totalValidDocs\" : 0," + "\"totalInvalidDocs\" : 10," + "\"segmentName\" : \"" + _completedSegment2.getSegmentName() + "\"," + "\"totalDocs\" : 10" + "}]"; - responseSet.add(new AbstractMap.SimpleEntry<>("", json)); + List validDocIdMetadataInfo = + JsonUtils.stringToObject(json, new TypeReference>() { + }); UpsertCompactionTaskGenerator.SegmentSelectionResult segmentSelectionResult = - UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, responseSet); + UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, + validDocIdMetadataInfo); assertEquals(segmentSelectionResult.getSegmentsForCompaction().get(0).getSegmentName(), _completedSegment.getSegmentName()); assertEquals(segmentSelectionResult.getSegmentsForDeletion().get(0), _completedSegment2.getSegmentName()); @@ -251,20 +202,23 @@ public void testProcessValidDocIdMetadata() { // test with a higher invalidRecordsThresholdPercent compactionConfigs = getCompactionConfigs("60", "10"); segmentSelectionResult = - UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, responseSet); + UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, + validDocIdMetadataInfo); assertTrue(segmentSelectionResult.getSegmentsForCompaction().isEmpty()); // test without an invalidRecordsThresholdPercent compactionConfigs = getCompactionConfigs("0", "10"); segmentSelectionResult = - UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, responseSet); + UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, + validDocIdMetadataInfo); assertEquals(segmentSelectionResult.getSegmentsForCompaction().get(0).getSegmentName(), _completedSegment.getSegmentName()); // test without a invalidRecordsThresholdCount compactionConfigs = getCompactionConfigs("30", "0"); segmentSelectionResult = - UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, responseSet); + UpsertCompactionTaskGenerator.processValidDocIdMetadata(compactionConfigs, _completedSegmentsMap, + validDocIdMetadataInfo); assertEquals(segmentSelectionResult.getSegmentsForCompaction().get(0).getSegmentName(), _completedSegment.getSegmentName()); } @@ -284,7 +238,7 @@ private Map getCompactionConfigs(String invalidRecordsThresholdP private IdealState getIdealState(String tableName, List segmentNames) { IdealState idealState = new IdealState(tableName); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - for (String segmentName: segmentNames) { + for (String segmentName : segmentNames) { idealState.setPartitionState(segmentName, "Server_0", "ONLINE"); } return idealState; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/CompactedPinotSegmentRecordReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/CompactedPinotSegmentRecordReader.java new file mode 100644 index 000000000000..2795982ab660 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/CompactedPinotSegmentRecordReader.java @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.segment.local.segment.readers; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.IOException; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.data.readers.RecordReaderConfig; +import org.roaringbitmap.PeekableIntIterator; +import org.roaringbitmap.RoaringBitmap; + + +/** + * Compacted Pinot Segment Record Reader used for upsert compaction + */ +public class CompactedPinotSegmentRecordReader implements RecordReader { + private final PinotSegmentRecordReader _pinotSegmentRecordReader; + private final PeekableIntIterator _validDocIdsIterator; + // Reusable generic row to store the next row to return + GenericRow _nextRow = new GenericRow(); + // Flag to mark whether we need to fetch another row + boolean _nextRowReturned = true; + + public CompactedPinotSegmentRecordReader(File indexDir, RoaringBitmap validDocIds) { + _pinotSegmentRecordReader = new PinotSegmentRecordReader(); + _pinotSegmentRecordReader.init(indexDir, null, null); + _validDocIdsIterator = validDocIds.getIntIterator(); + } + + @Override + public void init(File dataFile, @Nullable Set fieldsToRead, @Nullable RecordReaderConfig recordReaderConfig) + throws IOException { + } + + @Override + public boolean hasNext() { + if (!_validDocIdsIterator.hasNext() && _nextRowReturned) { + return false; + } + + // If next row has not been returned, return true + if (!_nextRowReturned) { + return true; + } + + // Try to get the next row to return + if (_validDocIdsIterator.hasNext()) { + int docId = _validDocIdsIterator.next(); + _nextRow.clear(); + _pinotSegmentRecordReader.getRecord(docId, _nextRow); + _nextRowReturned = false; + return true; + } + + // Cannot find next row to return, return false + return false; + } + + @Override + public GenericRow next() + throws IOException { + return next(new GenericRow()); + } + + @Override + public GenericRow next(GenericRow reuse) + throws IOException { + Preconditions.checkState(!_nextRowReturned); + reuse.init(_nextRow); + _nextRowReturned = true; + return reuse; + } + + @Override + public void rewind() + throws IOException { + _pinotSegmentRecordReader.rewind(); + _nextRowReturned = true; + } + + @Override + public void close() + throws IOException { + _pinotSegmentRecordReader.close(); + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index b08833b966d1..29f90715da54 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -529,31 +529,64 @@ public Response downloadValidDocIds( public String getValidDocIdMetadata( @ApiParam(value = "Table name including type", required = true, example = "myTable_REALTIME") @PathParam("tableNameWithType") String tableNameWithType, - @ApiParam(value = "Segment name", allowMultiple = true, required = true) @QueryParam("segmentNames") + @ApiParam(value = "Segment name", allowMultiple = true) @QueryParam("segmentNames") List segmentNames) { + return ResourceUtils.convertToJsonString(processValidDocIdMetadata(tableNameWithType, segmentNames)); + } + + @POST + @Path("/tables/{tableNameWithType}/validDocIdMetadata") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Provides segment validDocId metadata", notes = "Provides segment validDocId metadata") + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Success"), + @ApiResponse(code = 500, message = "Internal server error", response = ErrorInfo.class), + @ApiResponse(code = 404, message = "Table or segment not found", response = ErrorInfo.class) + }) + public String getValidDocIdMetadata( + @ApiParam(value = "Table name including type", required = true, example = "myTable_REALTIME") + @PathParam("tableNameWithType") String tableNameWithType, TableSegments tableSegments) { + List segmentNames = tableSegments.getSegments(); + return ResourceUtils.convertToJsonString(processValidDocIdMetadata(tableNameWithType, segmentNames)); + } + + private List> processValidDocIdMetadata(String tableNameWithType, List segments) { TableDataManager tableDataManager = ServerResourceUtils.checkGetTableDataManager(_serverInstance, tableNameWithType); List missingSegments = new ArrayList<>(); - List segmentDataManagers = tableDataManager.acquireSegments(segmentNames, missingSegments); - if (!missingSegments.isEmpty()) { - throw new WebApplicationException(String.format("Table %s has missing segments", tableNameWithType), - Response.Status.NOT_FOUND); + List segmentDataManagers; + if (segments == null) { + segmentDataManagers = tableDataManager.acquireAllSegments(); + } else { + segmentDataManagers = tableDataManager.acquireSegments(segments, missingSegments); + if (!missingSegments.isEmpty()) { + throw new WebApplicationException( + String.format("Table %s has missing segments: %s)", tableNameWithType, segments), + Response.Status.NOT_FOUND); + } } List> allValidDocIdMetadata = new ArrayList<>(); for (SegmentDataManager segmentDataManager : segmentDataManagers) { try { IndexSegment indexSegment = segmentDataManager.getSegment(); + if (indexSegment == null) { + LOGGER.warn("Table {} segment {} does not exist", tableNameWithType, segmentDataManager.getSegmentName()); + continue; + } + // Skip the consuming segments if (!(indexSegment instanceof ImmutableSegmentImpl)) { - throw new WebApplicationException( - String.format("Table %s segment %s is not a immutable segment", tableNameWithType, - segmentDataManager.getSegmentName()), Response.Status.BAD_REQUEST); + String msg = String.format("Table %s segment %s is not a immutable segment", tableNameWithType, + segmentDataManager.getSegmentName()); + LOGGER.warn(msg); + continue; } MutableRoaringBitmap validDocIds = indexSegment.getValidDocIds() != null ? indexSegment.getValidDocIds().getMutableRoaringBitmap() : null; if (validDocIds == null) { - throw new WebApplicationException( - String.format("Missing validDocIds for table %s segment %s does not exist", tableNameWithType, - segmentDataManager.getSegmentName()), Response.Status.NOT_FOUND); + String msg = String.format("Missing validDocIds for table %s segment %s does not exist", tableNameWithType, + segmentDataManager.getSegmentName()); + LOGGER.warn(msg); + throw new WebApplicationException(msg, Response.Status.NOT_FOUND); } Map validDocIdMetadata = new HashMap<>(); int totalDocs = indexSegment.getSegmentMetadata().getTotalDocs(); @@ -568,7 +601,7 @@ public String getValidDocIdMetadata( tableDataManager.releaseSegment(segmentDataManager); } } - return ResourceUtils.convertToJsonString(allValidDocIdMetadata); + return allValidDocIdMetadata; } /** diff --git a/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java b/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java index 0469d805d241..79b17396de7c 100644 --- a/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java +++ b/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.ws.rs.client.Entity; import javax.ws.rs.core.Response; import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.commons.io.FileUtils; @@ -311,6 +312,27 @@ public void testValidDocIdMetadata() Assert.assertEquals(validDocIdMetadata.get("totalInvalidDocs").asInt(), 99992); } + @Test + public void testValidDocIdMetadataPost() + throws IOException { + IndexSegment segment = _realtimeIndexSegments.get(0); + // Verify the content of the downloaded snapshot from a realtime table. + downLoadAndVerifyValidDocIdsSnapshot(TableNameBuilder.REALTIME.tableNameWithType(TABLE_NAME), + (ImmutableSegmentImpl) segment); + List segments = List.of(segment.getSegmentName()); + TableSegments tableSegments = new TableSegments(segments); + String validDocIdMetadataPath = + "/tables/" + TableNameBuilder.REALTIME.tableNameWithType(TABLE_NAME) + "/validDocIdMetadata"; + String response = + _webTarget.path(validDocIdMetadataPath).queryParam("segmentNames", segment.getSegmentName()).request() + .post(Entity.json(tableSegments), String.class); + JsonNode validDocIdMetadata = JsonUtils.stringToJsonNode(response).get(0); + + Assert.assertEquals(validDocIdMetadata.get("totalDocs").asInt(), 100000); + Assert.assertEquals(validDocIdMetadata.get("totalValidDocs").asInt(), 8); + Assert.assertEquals(validDocIdMetadata.get("totalInvalidDocs").asInt(), 99992); + } + // Verify metadata file from segments. private void downLoadAndVerifySegmentContent(String tableNameWithType, IndexSegment segment) throws IOException, ConfigurationException { From 7c16c925b3635c346bf2434900166d3388ad2dc5 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Mon, 11 Mar 2024 15:02:02 +0530 Subject: [PATCH 20/30] Add mutable index tests --- .../writer/impl/CLPForwardIndexWriterV1.java | 13 ++- .../segment/index/creator/CLPWriterTest.java | 47 +++++----- .../mutable/CLPMutableForwardIndexTest.java | 87 +++++++++++++++++++ 3 files changed, 118 insertions(+), 29 deletions(-) create mode 100644 pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index cc3ccaff4400..9d7d06f46da2 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -41,10 +41,15 @@ /** - * HEADER - * version - * _LOGTYPE_DICTIONARY_OFFSET - * column_LOGTYPE_DICTIONARY buffer + * Writer for CLP forward index. + *

CLP forward index contains 3 parts: + *

    + *
  • LogType dictionary: dictionary for logType column
  • + *
  • DictVars dictionary: dictionary for dictVars column
  • + *
  • EncodedVars forward index: forward index for encodedVars column
  • + *
  • LogType forward index: forward index for logType column
  • + *
  • DictVars forward index: forward index for dictVars column
  • + *
  • Metadata: magic bytes, version, number of dictVars, number of bytes per entry for logType and dictVars */ public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java index f4c9f3ed01c3..b869d2c31eef 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPWriterTest.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.commons.io.FileUtils; import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.index.readers.forward.CLPForwardIndexReaderV1; @@ -37,28 +38,20 @@ import org.apache.pinot.spi.data.DimensionFieldSpec; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.util.TestUtils; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; - -/** - * 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. - */ public class CLPWriterTest { + private static final File TEMP_DIR = new File(FileUtils.getTempDirectory(), "CLPForwardIndexCreatorTest"); + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(TEMP_DIR); + } @Test public void testCLPWriter() @@ -94,14 +87,11 @@ public void testCLPWriter() for (String logLine : logLines) { statsCollector.collect(logLine); } - statsCollector.seal(); - System.out.println(statsCollector.getCLPStats()); - File tmpDir = new File("/tmp/"); - File indexFile = new File(tmpDir, "column1.fwd"); + File indexFile = new File(TEMP_DIR, "column1.fwd"); CLPForwardIndexWriterV1 clpForwardIndexWriterV1 = - new CLPForwardIndexWriterV1(tmpDir, indexFile, "column1", 4, statsCollector); + new CLPForwardIndexWriterV1(TEMP_DIR, indexFile, "column1", 4, statsCollector); for (String logLine : logLines) { clpForwardIndexWriterV1.putString(logLine); @@ -111,7 +101,14 @@ public void testCLPWriter() PinotDataBuffer pinotDataBuffer = PinotDataBuffer.mapReadOnlyBigEndianFile(indexFile); CLPForwardIndexReaderV1 clpForwardIndexReaderV1 = new CLPForwardIndexReaderV1(pinotDataBuffer, logLines.size()); for (int i = 0; i < logLines.size(); i++) { - System.out.println(clpForwardIndexReaderV1.getString(i, clpForwardIndexReaderV1.createContext())); + Assert.assertEquals(clpForwardIndexReaderV1.getString(i, clpForwardIndexReaderV1.createContext()), + logLines.get(i)); } } + + @AfterClass + public void tearDown() + throws Exception { + FileUtils.deleteDirectory(TEMP_DIR); + } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java new file mode 100644 index 000000000000..7f3babf17d9a --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java @@ -0,0 +1,87 @@ +/** + * 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.segment.local.segment.index.forward.mutable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import org.apache.commons.lang.RandomStringUtils; +import org.apache.pinot.segment.local.io.writer.impl.DirectMemoryManager; +import org.apache.pinot.segment.local.realtime.impl.forward.CLPMutableForwardIndex; +import org.apache.pinot.segment.local.realtime.impl.forward.VarByteSVMutableForwardIndex; +import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; +import org.apache.pinot.spi.data.FieldSpec; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static java.nio.charset.StandardCharsets.UTF_8; + + +public class CLPMutableForwardIndexTest { + private PinotDataBufferMemoryManager _memoryManager; + + @BeforeClass + public void setUp() { + _memoryManager = new DirectMemoryManager(VarByteSVMutableForwardIndexTest.class.getName()); + } + + @AfterClass + public void tearDown() + throws Exception { + _memoryManager.close(); + } + + @Test + public void testString() + throws IOException { + // use arbitrary cardinality and avg string length + // we will test with complete randomness + int initialCapacity = 5; + int estimatedAvgStringLength = 30; + try (CLPMutableForwardIndex readerWriter = new CLPMutableForwardIndex("col1", FieldSpec.DataType.STRING, + _memoryManager, initialCapacity)) { + int rows = 3; + List logLines = new ArrayList<>(); + logLines.add( + "2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] " + + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); + logLines.add( + "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " + + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + logLines.add( + "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12" + + ".15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); + logLines.add( + "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 10.12" + + ".19.5 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + + "application/json status code 200 OK"); + + for (int i = 0; i < rows; i++) { + readerWriter.setString(i, logLines.get(i)); + } + + for (int i = 0; i < rows; i++) { + Assert.assertEquals(readerWriter.getString(i), logLines.get(i)); + } + } + } +} From 8ca1650e733338fa3d1bd55897cc358678f1bf6e Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Tue, 12 Mar 2024 14:56:27 +0530 Subject: [PATCH 21/30] Lint --- .../writer/impl/CLPForwardIndexWriterV1.java | 52 ++++++++++++++----- .../impl/forward/CLPMutableForwardIndex.java | 11 +++- .../creator/impl/stats/CLPStatsProvider.java | 24 ++++----- .../segment/creator/DictionariesTest.java | 17 ++++-- .../mutable/CLPMutableForwardIndexTest.java | 27 +++++++--- 5 files changed, 93 insertions(+), 38 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java index 9d7d06f46da2..4ff95b4a6aca 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java @@ -44,12 +44,13 @@ * Writer for CLP forward index. *

    CLP forward index contains 3 parts: *

      + *
    • Header bytes: MAGIC_BYTES, version,
    • *
    • LogType dictionary: dictionary for logType column
    • *
    • DictVars dictionary: dictionary for dictVars column
    • - *
    • EncodedVars forward index: forward index for encodedVars column
    • - *
    • LogType forward index: forward index for logType column
    • - *
    • DictVars forward index: forward index for dictVars column
    • - *
    • Metadata: magic bytes, version, number of dictVars, number of bytes per entry for logType and dictVars + *
    • LogType fwd index: fwd index for logType column
    • + *
    • DictVars fwd index: fwd index for dictVars column
    • + *
    • EncodedVars fwd index: raw fwd index for encodedVars column
    • + *
    */ public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { @@ -106,7 +107,7 @@ public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, _encodedVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_encodedvars.fwd"); _encodedVarsFwdIndexWriter = - new MultiValueFixedByteRawIndexCreator(_encodedVarsFwdIndexFile, ChunkCompressionType.PASS_THROUGH, numDocs, + new MultiValueFixedByteRawIndexCreator(_encodedVarsFwdIndexFile, ChunkCompressionType.LZ4, numDocs, FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, VarByteChunkForwardIndexWriterV4.VERSION); _clpStats.clear(); @@ -118,7 +119,7 @@ public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, @Override public void putBigDecimal(BigDecimal value) { - throw new UnsupportedOperationException("String only"); + throw new UnsupportedOperationException("Non string types are not supported"); } @Override @@ -167,17 +168,17 @@ private void addCLPFields(String logtype, String[] dictVars, Long[] encodedVars) @Override public void putBytes(byte[] value) { - throw new UnsupportedOperationException("String only"); + throw new UnsupportedOperationException("Non string types are not supported"); } @Override public void putStringMV(String[] values) { - throw new UnsupportedOperationException("String only"); + throw new UnsupportedOperationException("Non string types are not supported"); } @Override public void putBytesMV(byte[][] values) { - throw new UnsupportedOperationException("String only"); + throw new UnsupportedOperationException("Non string types are not supported"); } @Override @@ -194,26 +195,51 @@ public void close() _dictVarsFwdIndexWriter.close(); _encodedVarsFwdIndexWriter.close(); - int totalSize = MAGIC_BYTES.length + 9 * 4 + (int) _logTypeDictFile.length() + (int) _dictVarsDictFile.length() - + (int) _logTypeFwdIndexFile.length() + (int) _dictVarsFwdIndexFile.length() - + (int) _encodedVarsFwdIndexFile.length(); - + long totalSize = 0; _fileBuffer.put(MAGIC_BYTES); + totalSize += MAGIC_BYTES.length; + _fileBuffer.putInt(1); // version + totalSize += Integer.BYTES; + _fileBuffer.putInt(_clpStats.getTotalNumberOfDictVars()); + totalSize += Integer.BYTES; + _fileBuffer.putInt(_logTypeDictCreator.getNumBytesPerEntry()); + totalSize += Integer.BYTES; + _fileBuffer.putInt(_dictVarsDictCreator.getNumBytesPerEntry()); + totalSize += Integer.BYTES; + _fileBuffer.putInt((int) _logTypeDictFile.length()); // logType dict length + totalSize += Integer.BYTES; + _fileBuffer.putInt((int) _dictVarsDictFile.length()); // dictVars dict length + totalSize += Integer.BYTES; + _fileBuffer.putInt((int) _logTypeFwdIndexFile.length()); // logType fwd index length + totalSize += Integer.BYTES; + _fileBuffer.putInt((int) _dictVarsFwdIndexFile.length()); // dictVars fwd index length + totalSize += Integer.BYTES; + _fileBuffer.putInt((int) _encodedVarsFwdIndexFile.length()); // encodedVars fwd index length + totalSize += Integer.BYTES; copyFileIntoBuffer(_logTypeDictFile); + totalSize += _logTypeDictFile.length(); + copyFileIntoBuffer(_dictVarsDictFile); + totalSize += _dictVarsDictFile.length(); + copyFileIntoBuffer(_logTypeFwdIndexFile); + totalSize += _logTypeFwdIndexFile.length(); + copyFileIntoBuffer(_dictVarsFwdIndexFile); + totalSize += _dictVarsFwdIndexFile.length(); + copyFileIntoBuffer(_encodedVarsFwdIndexFile); + totalSize += _encodedVarsFwdIndexFile.length(); _dataFile.truncate(totalSize); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java index 28ca5aecd1ce..bed7a4e9bed3 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/CLPMutableForwardIndex.java @@ -34,6 +34,11 @@ public class CLPMutableForwardIndex implements MutableForwardIndex { + // TODO: We can get better dynamic estimates using segment stats + private static final int ESTIMATED_LOG_TYPE_CARDINALITY = 10000; + private static final int ESTIMATED_DICT_VARS_CARDINALITY = 10000; + private static final int ESTIMATED_LOG_TYPE_LENGTH = 200; + private static final int ESTIMATED_DICT_VARS_LENGTH = 50; private FieldSpec.DataType _storedType; private final EncodedMessage _clpEncodedMessage; private final MessageEncoder _clpMessageEncoder; @@ -57,9 +62,11 @@ public CLPMutableForwardIndex(String columnName, FieldSpec.DataType storedType, _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); _logTypeDictCreator = - new StringOffHeapMutableDictionary(10000, 100, memoryManager, columnName + "_logType.dict", 1000); + new StringOffHeapMutableDictionary(ESTIMATED_LOG_TYPE_CARDINALITY, ESTIMATED_LOG_TYPE_CARDINALITY / 10, + memoryManager, columnName + "_logType.dict", ESTIMATED_LOG_TYPE_LENGTH); _dictVarsDictCreator = - new StringOffHeapMutableDictionary(10000, 100, memoryManager, columnName + "_dictVars.dict", 1000); + new StringOffHeapMutableDictionary(ESTIMATED_DICT_VARS_CARDINALITY, ESTIMATED_DICT_VARS_CARDINALITY / 10, + memoryManager, columnName + "_dictVars.dict", ESTIMATED_DICT_VARS_LENGTH); _logTypeFwdIndex = new FixedByteSVMutableForwardIndex(true, FieldSpec.DataType.INT, capacity, memoryManager, columnName + "_logType.fwd"); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java index 84dfabc2fab0..58db3628d8f5 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java @@ -60,18 +60,6 @@ public CLPStats() { BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); } - public int getMaxNumberOfEncodedVars() { - return _maxNumberOfEncodedVars; - } - - public int getTotalNumberOfDictVars() { - return _totalNumberOfDictVars; - } - - public int getTotalNumberOfEncodedVars() { - return _totalNumberOfEncodedVars; - } - public void collect(String value) { String logType; String[] dictVars; @@ -119,6 +107,18 @@ public void clear() { _sortedDictVarValues = null; } + public int getMaxNumberOfEncodedVars() { + return _maxNumberOfEncodedVars; + } + + public int getTotalNumberOfDictVars() { + return _totalNumberOfDictVars; + } + + public int getTotalNumberOfEncodedVars() { + return _totalNumberOfEncodedVars; + } + public String[] getSortedLogTypeValues() { return _sortedLogTypeValues; } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java index 0a9ceb81830d..a678be426984 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java @@ -536,7 +536,7 @@ private AbstractColumnStatisticsCollector buildStatsCollector(String column, Dat } @Test - public void clpTest() { + public void clpStatsCollectorTest() { Schema schema = new Schema(); schema.addField(new DimensionFieldSpec("column1", DataType.STRING, true)); List fieldConfigList = new ArrayList<>(); @@ -546,14 +546,23 @@ public void clpTest() { StatsCollectorConfig statsCollectorConfig = new StatsCollectorConfig(_tableConfig, schema, null); StringColumnPreIndexStatsCollector statsCollector = new StringColumnPreIndexStatsCollector("column1", statsCollectorConfig); - statsCollector.collect( + + List logLines = new ArrayList<>(); + logLines.add( "2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] " + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); - statsCollector.collect( + logLines.add( "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + for (String logLine : logLines) { + statsCollector.collect(logLine); + } statsCollector.seal(); - System.out.println(statsCollector.getCLPStats()); + + Assert.assertNotNull(statsCollector.getCLPStats()); + + // Same log line format + Assert.assertEquals(statsCollector.getCLPStats().getSortedLogTypeValues().length, 1); } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java index 7f3babf17d9a..417577a5245c 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java @@ -21,11 +21,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Random; -import org.apache.commons.lang.RandomStringUtils; import org.apache.pinot.segment.local.io.writer.impl.DirectMemoryManager; import org.apache.pinot.segment.local.realtime.impl.forward.CLPMutableForwardIndex; -import org.apache.pinot.segment.local.realtime.impl.forward.VarByteSVMutableForwardIndex; +import org.apache.pinot.segment.local.segment.creator.impl.stats.CLPStatsProvider; import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; import org.apache.pinot.spi.data.FieldSpec; import org.testng.Assert; @@ -33,8 +31,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static java.nio.charset.StandardCharsets.UTF_8; - public class CLPMutableForwardIndexTest { private PinotDataBufferMemoryManager _memoryManager; @@ -63,10 +59,12 @@ public void testString() List logLines = new ArrayList<>(); logLines.add( "2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] " - + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); + + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: " + + "true"); logLines.add( "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " - + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: " + + "true"); logLines.add( "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12" + ".15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); @@ -82,6 +80,21 @@ public void testString() for (int i = 0; i < rows; i++) { Assert.assertEquals(readerWriter.getString(i), logLines.get(i)); } + + // Verify clp stats + CLPStatsProvider.CLPStats stats = new CLPStatsProvider.CLPStats(); + for (int i = 0; i < rows; i++) { + stats.collect(logLines.get(i)); + } + stats.seal(); + + CLPStatsProvider.CLPStats mutableIndexStats = readerWriter.getCLPStats(); + Assert.assertEquals(stats.getTotalNumberOfDictVars(), mutableIndexStats.getTotalNumberOfDictVars()); + Assert.assertEquals(stats.getMaxNumberOfEncodedVars(), mutableIndexStats.getMaxNumberOfEncodedVars()); + Assert.assertEquals(stats.getSortedDictVarValues(), mutableIndexStats.getSortedDictVarValues()); + Assert.assertEquals(stats.getTotalNumberOfEncodedVars(), mutableIndexStats.getTotalNumberOfEncodedVars()); + Assert.assertEquals(stats.getSortedLogTypeValues(), mutableIndexStats.getSortedLogTypeValues()); + Assert.assertEquals(stats.getSortedDictVarValues(), mutableIndexStats.getSortedDictVarValues()); } } } From 965f7c2ca3314c52900df15ec5d9a65daea919bb Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Tue, 12 Mar 2024 15:56:35 +0530 Subject: [PATCH 22/30] Fix tests --- .../org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java index 14c148bbb271..e85556d78d9f 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java @@ -472,7 +472,7 @@ CompressionCodec getCompressionCodec() { CompressionCodec[] compressionCodecs = CompressionCodec.values(); while (true) { CompressionCodec compressionCodec = compressionCodecs[RANDOM.nextInt(compressionCodecs.length)]; - if (compressionCodec.isApplicableToRawIndex()) { + if (compressionCodec.isApplicableToRawIndex() && compressionCodec != CompressionCodec.CLP) { return compressionCodec; } } From e474bb2fa61d2050cba300c5835d61a200bae73f Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Wed, 13 Mar 2024 09:36:43 +0530 Subject: [PATCH 23/30] Remove chunkCompressionType --- .../impl/fwd/CLPForwardIndexCreatorV1.java} | 47 ++++++++++--------- .../SingleValueVarByteRawIndexCreator.java | 22 +++------ .../forward/ForwardIndexCreatorFactory.java | 13 +++-- .../forward/ForwardIndexReaderFactory.java | 8 ++-- .../index/forward/ForwardIndexType.java | 2 +- .../forward/CLPForwardIndexReaderV1.java | 10 +--- ...t.java => CLPForwardIndexCreatorTest.java} | 15 +++--- .../spi/compression/ChunkCompressionType.java | 2 +- .../segment/spi/index/ForwardIndexConfig.java | 2 +- .../DictionaryToRawIndexConverter.java | 2 +- 10 files changed, 60 insertions(+), 63 deletions(-) rename pinot-segment-local/src/main/java/org/apache/pinot/segment/local/{io/writer/impl/CLPForwardIndexWriterV1.java => segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java} (90%) rename pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/{CLPWriterTest.java => CLPForwardIndexCreatorTest.java} (90%) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java similarity index 90% rename from pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java rename to pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java index 4ff95b4a6aca..a2a50d43679c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/CLPForwardIndexWriterV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.segment.local.io.writer.impl; +package org.apache.pinot.segment.local.segment.creator.impl.fwd; import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; import com.yscope.clp.compressorfrontend.EncodedMessage; @@ -31,12 +31,16 @@ import java.nio.file.StandardOpenOption; import org.apache.commons.io.FileUtils; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; +import org.apache.pinot.segment.local.io.writer.impl.FixedBitMVForwardIndexWriter; +import org.apache.pinot.segment.local.io.writer.impl.FixedBitSVForwardIndexWriter; +import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator; -import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator; import org.apache.pinot.segment.local.segment.creator.impl.stats.CLPStatsProvider; import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; +import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.creator.ColumnStatistics; +import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator; import org.apache.pinot.spi.data.FieldSpec; @@ -53,7 +57,7 @@ *
*/ -public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { +public class CLPForwardIndexCreatorV1 implements ForwardIndexCreator { public static final byte[] MAGIC_BYTES = "CLP.v1".getBytes(StandardCharsets.UTF_8); private final String _column; private final int _numDocs; @@ -74,13 +78,14 @@ public class CLPForwardIndexWriterV1 implements VarByteChunkWriter { private final File _dictVarsFwdIndexFile; private final File _encodedVarsFwdIndexFile; - public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, int numDocs, - ColumnStatistics columnStatistics) + public CLPForwardIndexCreatorV1(File baseIndexDir, String column, int numDocs, ColumnStatistics columnStatistics) throws IOException { _column = column; _numDocs = numDocs; _baseIndexDir = baseIndexDir; - _dataFile = new RandomAccessFile(indexFile, "rw").getChannel(); + _dataFile = + new RandomAccessFile(new File(baseIndexDir, column + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION), + "rw").getChannel(); _fileBuffer = _dataFile.map(FileChannel.MapMode.READ_WRITE, 0, Integer.MAX_VALUE); CLPStatsProvider statsCollector = (CLPStatsProvider) columnStatistics; @@ -117,6 +122,21 @@ public CLPForwardIndexWriterV1(File baseIndexDir, File indexFile, String column, BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); } + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return true; + } + + @Override + public FieldSpec.DataType getValueType() { + return FieldSpec.DataType.STRING; + } + @Override public void putBigDecimal(BigDecimal value) { throw new UnsupportedOperationException("Non string types are not supported"); @@ -166,21 +186,6 @@ private void addCLPFields(String logtype, String[] dictVars, Long[] encodedVars) _encodedVarsFwdIndexWriter.putLongMV(encodedVarsUnboxed); } - @Override - public void putBytes(byte[] value) { - throw new UnsupportedOperationException("Non string types are not supported"); - } - - @Override - public void putStringMV(String[] values) { - throw new UnsupportedOperationException("Non string types are not supported"); - } - - @Override - public void putBytesMV(byte[][] values) { - throw new UnsupportedOperationException("Non string types are not supported"); - } - @Override public void close() throws IOException { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java index 0bc9655d84d5..9e3658802f57 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueVarByteRawIndexCreator.java @@ -22,13 +22,11 @@ import java.io.File; import java.io.IOException; import java.math.BigDecimal; -import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkWriter; import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; -import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.index.ForwardIndexConfig; import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator; import org.apache.pinot.spi.data.FieldSpec.DataType; @@ -59,7 +57,7 @@ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType int totalDocs, DataType valueType, int maxLength) throws IOException { this(baseIndexDir, compressionType, column, totalDocs, valueType, maxLength, false, - ForwardIndexConfig.DEFAULT_RAW_WRITER_VERSION, null); + ForwardIndexConfig.DEFAULT_RAW_WRITER_VERSION); } /** @@ -75,20 +73,14 @@ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType * @throws IOException */ public SingleValueVarByteRawIndexCreator(File baseIndexDir, ChunkCompressionType compressionType, String column, - int totalDocs, DataType valueType, int maxLength, boolean deriveNumDocsPerChunk, int writerVersion, - ColumnStatistics columnStatistics) + int totalDocs, DataType valueType, int maxLength, boolean deriveNumDocsPerChunk, int writerVersion) throws IOException { File file = new File(baseIndexDir, column + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION); - if (compressionType == ChunkCompressionType.CLP) { - _indexWriter = new CLPForwardIndexWriterV1(baseIndexDir, file, column, totalDocs, columnStatistics); - _valueType = valueType; - } else { - int numDocsPerChunk = deriveNumDocsPerChunk ? getNumDocsPerChunk(maxLength) : DEFAULT_NUM_DOCS_PER_CHUNK; - _indexWriter = writerVersion < VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexWriter(file, - compressionType, totalDocs, numDocsPerChunk, maxLength, writerVersion) - : new VarByteChunkForwardIndexWriterV4(file, compressionType, TARGET_MAX_CHUNK_SIZE); - _valueType = valueType; - } + int numDocsPerChunk = deriveNumDocsPerChunk ? getNumDocsPerChunk(maxLength) : DEFAULT_NUM_DOCS_PER_CHUNK; + _indexWriter = writerVersion < VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexWriter(file, + compressionType, totalDocs, numDocsPerChunk, maxLength, writerVersion) + : new VarByteChunkForwardIndexWriterV4(file, compressionType, TARGET_MAX_CHUNK_SIZE); + _valueType = valueType; } @VisibleForTesting diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java index 503df68dc8ae..61ee335c0a4a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.IOException; +import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1; import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueEntryDictForwardIndexCreator; import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator; import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueUnsortedForwardIndexCreator; @@ -31,10 +32,10 @@ import org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.compression.DictIdCompressionType; -import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.creator.IndexCreationContext; import org.apache.pinot.segment.spi.index.ForwardIndexConfig; import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator; +import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; @@ -70,6 +71,10 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex } else { // Dictionary disabled columns DataType storedType = fieldSpec.getDataType().getStoredType(); + if (indexConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { + // CLP compression codec + return new CLPForwardIndexCreatorV1(indexDir, columnName, numTotalDocs, context.getColumnStatistics()); + } ChunkCompressionType chunkCompressionType = indexConfig.getChunkCompressionType(); if (chunkCompressionType == null) { chunkCompressionType = ForwardIndexType.getDefaultCompressionType(fieldSpec.getFieldType()); @@ -78,7 +83,7 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex int writerVersion = indexConfig.getRawIndexWriterVersion(); if (fieldSpec.isSingleValueField()) { return getRawIndexCreatorForSVColumn(indexDir, chunkCompressionType, columnName, storedType, numTotalDocs, - context.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion, context.getColumnStatistics()); + context.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion); } else { return getRawIndexCreatorForMVColumn(indexDir, chunkCompressionType, columnName, storedType, numTotalDocs, context.getMaxNumberOfMultiValueElements(), deriveNumDocsPerChunk, writerVersion, @@ -93,7 +98,7 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex */ public static ForwardIndexCreator getRawIndexCreatorForSVColumn(File indexDir, ChunkCompressionType compressionType, String column, DataType storedType, int numTotalDocs, int lengthOfLongestEntry, boolean deriveNumDocsPerChunk, - int writerVersion, ColumnStatistics columnStatistics) + int writerVersion) throws IOException { switch (storedType) { case INT: @@ -106,7 +111,7 @@ public static ForwardIndexCreator getRawIndexCreatorForSVColumn(File indexDir, C case STRING: case BYTES: return new SingleValueVarByteRawIndexCreator(indexDir, compressionType, column, numTotalDocs, storedType, - lengthOfLongestEntry, deriveNumDocsPerChunk, writerVersion, columnStatistics); + lengthOfLongestEntry, deriveNumDocsPerChunk, writerVersion); default: throw new IllegalStateException("Unsupported stored type: " + storedType); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index 42e322fe233d..236d04ba1e1e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -20,8 +20,8 @@ package org.apache.pinot.segment.local.segment.index.forward; import org.apache.commons.lang.ArrayUtils; -import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; +import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1; import org.apache.pinot.segment.local.segment.index.readers.forward.CLPForwardIndexReaderV1; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitMVEntryDictForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitMVForwardIndexReader; @@ -79,10 +79,10 @@ public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, C } } } else { - if (dataBuffer.size() >= CLPForwardIndexWriterV1.MAGIC_BYTES.length) { - byte[] magicBytes = new byte[CLPForwardIndexWriterV1.MAGIC_BYTES.length]; + if (dataBuffer.size() >= CLPForwardIndexCreatorV1.MAGIC_BYTES.length) { + byte[] magicBytes = new byte[CLPForwardIndexCreatorV1.MAGIC_BYTES.length]; dataBuffer.copyTo(0, magicBytes); - if (ArrayUtils.isEquals(magicBytes, CLPForwardIndexWriterV1.MAGIC_BYTES)) { + if (ArrayUtils.isEquals(magicBytes, CLPForwardIndexCreatorV1.MAGIC_BYTES)) { return new CLPForwardIndexReaderV1(dataBuffer, metadata.getTotalDocs()); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java index 1dc1ddc62577..4172a6bdbb57 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java @@ -284,7 +284,7 @@ public MutableIndex createMutableIndex(MutableIndexContext context, ForwardIndex // Use a smaller capacity as opposed to segment flush size int initialCapacity = Math.min(context.getCapacity(), NODICT_VARIABLE_WIDTH_ESTIMATED_NUMBER_OF_VALUES_DEFAULT); - if (config.getChunkCompressionType() == ChunkCompressionType.CLP) { + if (config.getCompressionCodec() == CompressionCodec.CLP) { return new CLPMutableForwardIndex(column, storedType, context.getMemoryManager(), context.getCapacity()); } return new VarByteSVMutableForwardIndex(storedType, context.getMemoryManager(), allocationContext, diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index 0fdf4d67e45a..76f4e3c6f288 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -23,8 +23,7 @@ import java.io.IOException; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.io.util.VarLengthValueReader; -import org.apache.pinot.segment.local.io.writer.impl.CLPForwardIndexWriterV1; -import org.apache.pinot.segment.spi.compression.ChunkCompressionType; +import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; @@ -45,7 +44,7 @@ public class CLPForwardIndexReaderV1 implements ForwardIndexReader Date: Wed, 13 Mar 2024 11:14:42 +0530 Subject: [PATCH 24/30] Add validations --- .../core/startree/v2/BaseStarTreeV2Test.java | 2 +- .../impl/fwd/CLPForwardIndexCreatorV1.java | 38 +++++++++++-------- .../segment/local/utils/TableConfigUtils.java | 10 ++++- .../creator/CLPForwardIndexCreatorTest.java | 1 + .../pinot/spi/config/table/FieldConfig.java | 4 +- 5 files changed, 36 insertions(+), 19 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java index e85556d78d9f..14c148bbb271 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java @@ -472,7 +472,7 @@ CompressionCodec getCompressionCodec() { CompressionCodec[] compressionCodecs = CompressionCodec.values(); while (true) { CompressionCodec compressionCodec = compressionCodecs[RANDOM.nextInt(compressionCodecs.length)]; - if (compressionCodec.isApplicableToRawIndex() && compressionCodec != CompressionCodec.CLP) { + if (compressionCodec.isApplicableToRawIndex()) { return compressionCodec; } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java index a2a50d43679c..5a1a436d5780 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java @@ -61,7 +61,7 @@ public class CLPForwardIndexCreatorV1 implements ForwardIndexCreator { public static final byte[] MAGIC_BYTES = "CLP.v1".getBytes(StandardCharsets.UTF_8); private final String _column; private final int _numDocs; - private final File _baseIndexDir; + private final File _intermediateFilesDir; private final FileChannel _dataFile; private final ByteBuffer _fileBuffer; private final EncodedMessage _clpEncodedMessage; @@ -82,7 +82,14 @@ public CLPForwardIndexCreatorV1(File baseIndexDir, String column, int numDocs, C throws IOException { _column = column; _numDocs = numDocs; - _baseIndexDir = baseIndexDir; + _intermediateFilesDir = + new File(baseIndexDir, column + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION + ".clp.tmp"); + if (_intermediateFilesDir.exists()) { + FileUtils.cleanDirectory(_intermediateFilesDir); + } else { + FileUtils.forceMkdir(_intermediateFilesDir); + } + _dataFile = new RandomAccessFile(new File(baseIndexDir, column + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION), "rw").getChannel(); @@ -90,27 +97,27 @@ public CLPForwardIndexCreatorV1(File baseIndexDir, String column, int numDocs, C CLPStatsProvider statsCollector = (CLPStatsProvider) columnStatistics; _clpStats = statsCollector.getCLPStats(); - _logTypeDictFile = new File(_baseIndexDir, _column + "_clp_logtype.dict"); + _logTypeDictFile = new File(_intermediateFilesDir, _column + "_clp_logtype.dict"); _logTypeDictCreator = new SegmentDictionaryCreator(_column + "_clp_logtype.dict", FieldSpec.DataType.STRING, _logTypeDictFile, true); _logTypeDictCreator.build(_clpStats.getSortedLogTypeValues()); - _dictVarsDictFile = new File(_baseIndexDir, _column + "_clp_dictvars.dict"); + _dictVarsDictFile = new File(_intermediateFilesDir, _column + "_clp_dictvars.dict"); _dictVarsDictCreator = new SegmentDictionaryCreator(_column + "_clp_dictvars.dict", FieldSpec.DataType.STRING, _dictVarsDictFile, true); _dictVarsDictCreator.build(_clpStats.getSortedDictVarValues()); - _logTypeFwdIndexFile = new File(_baseIndexDir, column + "_clp_logtype.fwd"); + _logTypeFwdIndexFile = new File(_intermediateFilesDir, column + "_clp_logtype.fwd"); _logTypeFwdIndexWriter = new FixedBitSVForwardIndexWriter(_logTypeFwdIndexFile, numDocs, PinotDataBitSet.getNumBitsPerValue(_clpStats.getSortedLogTypeValues().length - 1)); - _dictVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_dictvars.fwd"); + _dictVarsFwdIndexFile = new File(_intermediateFilesDir, column + "_clp_dictvars.fwd"); _dictVarsFwdIndexWriter = new FixedBitMVForwardIndexWriter(_dictVarsFwdIndexFile, numDocs, _clpStats.getTotalNumberOfDictVars(), PinotDataBitSet.getNumBitsPerValue(_clpStats.getSortedDictVarValues().length - 1)); - _encodedVarsFwdIndexFile = new File(_baseIndexDir, column + "_clp_encodedvars.fwd"); + _encodedVarsFwdIndexFile = new File(_intermediateFilesDir, column + "_clp_encodedvars.fwd"); _encodedVarsFwdIndexWriter = new MultiValueFixedByteRawIndexCreator(_encodedVarsFwdIndexFile, ChunkCompressionType.LZ4, numDocs, FieldSpec.DataType.LONG, _clpStats.getMaxNumberOfEncodedVars(), false, @@ -187,7 +194,7 @@ private void addCLPFields(String logtype, String[] dictVars, Long[] encodedVars) } @Override - public void close() + public void seal() throws IOException { // Append all of these into fileBuffer _logTypeDictCreator.seal(); @@ -247,13 +254,6 @@ public void close() totalSize += _encodedVarsFwdIndexFile.length(); _dataFile.truncate(totalSize); - - // Delete all temp files - FileUtils.deleteQuietly(_logTypeDictFile); - FileUtils.deleteQuietly(_dictVarsDictFile); - FileUtils.deleteQuietly(_logTypeFwdIndexFile); - FileUtils.deleteQuietly(_dictVarsFwdIndexFile); - FileUtils.deleteQuietly(_encodedVarsFwdIndexFile); } private void copyFileIntoBuffer(File file) throws IOException { @@ -261,4 +261,12 @@ private void copyFileIntoBuffer(File file) throws IOException { _fileBuffer.put(from.map(FileChannel.MapMode.READ_ONLY, 0, file.length())); } } + + @Override + public void close() + throws IOException { + // Delete all temp file + _dataFile.close(); + FileUtils.deleteDirectory(_intermediateFilesDir); + } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java index 43fd69488b99..6c83ae9cb833 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java @@ -1192,8 +1192,14 @@ private static void validateFieldConfigList(@Nullable List fieldCon CompressionCodec compressionCodec = fieldConfig.getCompressionCodec(); switch (encodingType) { case RAW: - Preconditions.checkArgument(compressionCodec == null || compressionCodec.isApplicableToRawIndex(), - "Compression codec: %s is not applicable to raw index", compressionCodec); + Preconditions.checkArgument(compressionCodec == null || compressionCodec.isApplicableToRawIndex() + || compressionCodec == CompressionCodec.CLP, "Compression codec: %s is not applicable to raw index", + compressionCodec); + if (compressionCodec == CompressionCodec.CLP && schema != null) { + Preconditions.checkArgument( + schema.getFieldSpecFor(columnName).getDataType().getStoredType() == DataType.STRING, + "CLP compression codec can only be applied to string columns"); + } break; case DICTIONARY: Preconditions.checkArgument(compressionCodec == null || compressionCodec.isApplicableToDictEncodedIndex(), diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java index 3831ccaf302b..c9d36bd7e843 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java @@ -97,6 +97,7 @@ public void testCLPWriter() for (String logLine : logLines) { clpForwardIndexCreatorV1.putString(logLine); } + clpForwardIndexCreatorV1.seal(); clpForwardIndexCreatorV1.close(); PinotDataBuffer pinotDataBuffer = PinotDataBuffer.mapReadOnlyBigEndianFile(indexFile); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java index 2335c1cce715..e5708d1ab2eb 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java @@ -126,7 +126,9 @@ public enum CompressionCodec { SNAPPY(true, false), ZSTANDARD(true, false), LZ4(true, false), - CLP(true, false), + // CLP is a special type of compression codec that isn't generally applicable to all RAW columns and has a + // special handling (see {@link CLPForwardIndexCreatorV1}) + CLP(false, false), // For MV dictionary encoded forward index, add a second level dictionary encoding for the multi-value entries MV_ENTRY_DICT(false, true); From fcfac6b800c53cef505aa23194729ca3d12f7875 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Wed, 13 Mar 2024 12:36:26 +0530 Subject: [PATCH 25/30] Add ITs --- .../CLPEncodingRealtimeIntegrationTest.java | 149 ++++++++++++++++++ .../test/resources/clpEncodingITData.tar.gz | Bin 0 -> 1863 bytes ...codingRealtimeIntegrationTestSchema.schema | 18 +++ .../forward/CLPForwardIndexReaderV1.java | 11 +- .../spi/creator/SegmentGeneratorConfig.java | 4 +- .../segment/spi/index/ForwardIndexConfig.java | 5 +- 6 files changed, 178 insertions(+), 9 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java create mode 100644 pinot-integration-tests/src/test/resources/clpEncodingITData.tar.gz create mode 100644 pinot-integration-tests/src/test/resources/clpEncodingRealtimeIntegrationTestSchema.schema diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java new file mode 100644 index 000000000000..c94f1f723f9c --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java @@ -0,0 +1,149 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.integration.tests; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.pinot.spi.config.table.FieldConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.TransformConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.util.TestUtils; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + + +public class CLPEncodingRealtimeIntegrationTest extends BaseClusterIntegrationTestSet { + private List _avroFiles; + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + _avroFiles = unpackAvroData(_tempDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServers(1); + + startKafka(); + pushAvroIntoKafka(_avroFiles); + + Schema schema = createSchema(); + addSchema(schema); + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + addTableConfig(tableConfig); + + waitForAllDocsLoaded(600_000L); + } + + @Nullable + @Override + protected List getInvertedIndexColumns() { + return null; + } + + @Nullable + @Override + protected List getRangeIndexColumns() { + return null; + } + + @Nullable + @Override + protected List getBloomFilterColumns() { + return null; + } + + @Nullable + @Override + protected String getSortedColumn() { + return null; + } + + @Override + protected List getNoDictionaryColumns() { + return Collections.singletonList("logLine"); + } + + @Test + public void testValues() + throws Exception { + assertEquals(getCurrentCountStarResult(), getCountStarResult()); + } + + protected int getRealtimeSegmentFlushSize() { + return 30; + } + + @Override + protected long getCountStarResult() { + return 100; + } + + @Override + protected String getTableName() { + return "clpEncodingIT"; + } + + @Override + protected String getAvroTarFileName() { + return "clpEncodingITData.tar.gz"; + } + + @Override + protected String getSchemaFileName() { + return "clpEncodingRealtimeIntegrationTestSchema.schema"; + } + + @Override + protected String getTimeColumnName() { + return "timestampInEpoch"; + } + + @Override + protected List getFieldConfigs() { + List fieldConfigs = new ArrayList<>(); + fieldConfigs.add( + new FieldConfig("logLine", FieldConfig.EncodingType.RAW, null, null, FieldConfig.CompressionCodec.CLP, null, + null, null, null)); + + return fieldConfigs; + } + + @Override + protected IngestionConfig getIngestionConfig() { + List transforms = new ArrayList<>(); + transforms.add(new TransformConfig("timestampInEpoch", "now()")); + + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setTransformConfigs(transforms); + + return ingestionConfig; + } +} diff --git a/pinot-integration-tests/src/test/resources/clpEncodingITData.tar.gz b/pinot-integration-tests/src/test/resources/clpEncodingITData.tar.gz new file mode 100644 index 0000000000000000000000000000000000000000..c2f5baaa5f6b1541b3e2243956478a56aead8cfb GIT binary patch literal 1863 zcmV-N2e|kjiwFQFOYvm@1MOPPi`--w?}~>WJc{R+J#}E4zTY1Mde~}9v9-k(JctaL zne1-5Gs%)Q}N`R%*>l)-c6c!W|tytE;H$* z&+|UN=j-=8*|DGbp8p_;FPz;4A>@>T{8MmNd_p|G&%Y4}2|$zyiYX%m0;q2|y#U}D zp2=LZH1iXEl+>Rk{_wyX&xXPA3D4=r_V!xj%#8u&^M7*P9^T)2`L#R%ymWLBO#LTY z54W@9qhPzgodlye8E@}wNB%UwA5D%1VKx}+kQvM$*1zqA!DO6n_uqfGd~6c$-wdNb z|F)2mW=RGfyo${ROd`R3D$zimBzy!GQ2 z5?3I<0wqAI4^dy^dE=dHw?Xh>Fq&m?;=Q%_(g%07_cATr{045k2^feYa1f^%ms2AQ880*pKe!U*Un*be~IH<#{dMHuxwfCXY;c$@_( zNP=`W$v_%J8OUNVPQnL4^2e%H)NIa*R#nxCms$}j)IurR)c)F}_UJe}h$HV#@$v2~ ziUO_pUZ|n!>;Ikk6Yx49at93KIMZJ*BZhbMDZ=~)DYygnibLu<_{`FQ<~mYsjXtw? zE11T~@m+lv^ug{V9vy;YJRTr4Afk9ko9d?vGqk1C;(2f>NV9P2XZqJP{3rm3Yuk6V z<|GIJh0K%S?cLs&&(mHgyyZvwdyZ{9i^gDR4uJ;@=o^1BIsUdul5x@1*iX{ndgANI z9xwg1>yHkDXxshWpXdb(EnWETMRDydyt3^sf3kh0NjZ zX{mepp?AP2o*vDLI1jU5n&ftIEQO_Gr7Hc3Rn@OH9=I`Nb4B^VPKemTZAHj(E_-&c z;7-AU$pE8ugN0Dy;XGJQ%OtThNkG>7(-KP7N^K;KR956R(iXSm_G#(Grjki4ikc#L zo8Ip%y>!DoK@f>Ekcx=>*^YpgnB(H^wG!d+tCAJq>M=!9d6Nt$YL@6u0&s)83 zNzcWFh$=7?8DO};1uzu10b{$IJ5)wgx|~wua#?e^4ZYYD zdMW5vmQ?8my{;Au?Ck?KQ=D3vt3%IdNpTeG*}zJNTv8QaQ3-5~No)*(u7I0DP$zjz z83>4kJ-h3w?hpuE>ADp*#YhDM^zlm#Dq0%kFp2%~Nb}o)z?B~Pz@N<8n-k+^Cjv?IwuK`liqk%X%ntm=bTxm;8AL8R(`!Iu6fdLZr4 z9IbZHS#hl$2wv)?bYAt8Q#-Kqy4ECuIk6mtU07x%-l-@ZRU?^J!@+pXhV#j4;$Y&C zI0$Z#IUMF=na&*B`&}tn&HIWN?+eUx?{nTd=Gop~)$8nip^)u;vL1P=c2ZRJ{`Z!K zRnpguk3&OYDa5j8!?7usq>yl{EH0&XN(N@&z^p8b@O02Xbu>)TqMAkatggBg>)uuE z6E>6GHh&>Q>L}%^_qQc>J+6q5VJwuXIay?Gvy7$LIa0WkXCPG)Xrr`BJG=EXgiUi{ za4~7RqSUOp=5|)sYX)3~^rUIWrEny?syQyZvb}m-5)+pU3uF@^?7(;{OBM_pGTsKZ z6W`kAVh|6|N zomAhDYOQZ%(?%#HV|Gq8(}kAx22$n1RWWH-dm`3DLsT~ERWz&yPP#Vm)rGB@4nO{^*XN)ght-qOXvb!g#Ll7M+xPiQsW?UQ$$$X8;A%kt!ckrRO4`1KOhP}>Rb_l8p;+Ig1*w^3eQ5xAo)C13rUs~7zzXy@1Yb$*>+=hq9k{sYjGT<`!c002pl Bs~Z3S literal 0 HcmV?d00001 diff --git a/pinot-integration-tests/src/test/resources/clpEncodingRealtimeIntegrationTestSchema.schema b/pinot-integration-tests/src/test/resources/clpEncodingRealtimeIntegrationTestSchema.schema new file mode 100644 index 000000000000..9aa38a1a19d7 --- /dev/null +++ b/pinot-integration-tests/src/test/resources/clpEncodingRealtimeIntegrationTestSchema.schema @@ -0,0 +1,18 @@ +{ + "schemaName": "clpEncodingIT", + "dimensionFieldSpecs": [ + { + "name": "logLine", + "dataType": "STRING" + } + ], + "dateTimeFieldSpecs": [ + { + "name": "timestampInEpoch", + "dataType": "LONG", + "notNull": false, + "format": "1:MILLISECONDS:EPOCH", + "granularity": "1:MILLISECONDS" + } + ] +} \ No newline at end of file diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index 76f4e3c6f288..135035bb978a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -24,6 +24,7 @@ import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.io.util.VarLengthValueReader; import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1; +import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; @@ -127,7 +128,8 @@ public String getString(int docId, CLPReaderContext context) { } @Override - public void close() throws IOException { + public void close() + throws IOException { } @Override @@ -136,14 +138,17 @@ public CLPReaderContext createContext() { _encodedVarFwdIndexReader.createContext()); } + @Override + public ChunkCompressionType getCompressionType() { + return ChunkCompressionType.PASS_THROUGH; + } public static final class CLPReaderContext implements ForwardIndexReaderContext { private final FixedBitMVForwardIndexReader.Context _dictVarsReaderContext; private final ForwardIndexReaderContext _logTypeReaderContext; private final VarByteChunkForwardIndexReaderV4.ReaderContext _encodedVarReaderContext; - public CLPReaderContext( - FixedBitMVForwardIndexReader.Context dictVarsReaderContext, + public CLPReaderContext(FixedBitMVForwardIndexReader.Context dictVarsReaderContext, ForwardIndexReaderContext logTypeReaderContext, VarByteChunkForwardIndexReaderV4.ReaderContext encodedVarReaderContext) { _dictVarsReaderContext = dictVarsReaderContext; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java index 52abf41ea172..e7e1827efabf 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java @@ -273,8 +273,8 @@ private void extractCompressionCodecConfigsFromTableConfig(TableConfig tableConf List fieldConfigList = tableConfig.getFieldConfigList(); if (fieldConfigList != null) { for (FieldConfig fieldConfig : fieldConfigList) { - if (fieldConfig.getEncodingType() == FieldConfig.EncodingType.RAW - && fieldConfig.getCompressionCodec() != null) { + if (fieldConfig.getEncodingType() == FieldConfig.EncodingType.RAW && fieldConfig.getCompressionCodec() != null + && fieldConfig.getCompressionCodec().isApplicableToRawIndex()) { _rawIndexCreationColumns.add(fieldConfig.getName()); _rawIndexCompressionType.put(fieldConfig.getName(), ChunkCompressionType.valueOf(fieldConfig.getCompressionCodec().name())); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java index 35ee54aa3ae1..132705036b3e 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java @@ -58,6 +58,7 @@ public ForwardIndexConfig(@Nullable Boolean disabled, @Nullable CompressionCodec if (compressionCodec != null) { switch (compressionCodec) { case PASS_THROUGH: + case CLP: _chunkCompressionType = ChunkCompressionType.PASS_THROUGH; _dictIdCompressionType = null; break; @@ -77,10 +78,6 @@ public ForwardIndexConfig(@Nullable Boolean disabled, @Nullable CompressionCodec _dictIdCompressionType = DictIdCompressionType.MV_ENTRY_DICT; _chunkCompressionType = null; break; - case CLP: - _chunkCompressionType = null; - _dictIdCompressionType = null; - break; default: throw new IllegalStateException("Unsupported compression codec: " + compressionCodec); } From c81fa0d47d6b97e20b424c8412d7c591b87042b4 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Wed, 13 Mar 2024 13:06:42 +0530 Subject: [PATCH 26/30] Lint + misc --- .../creator/impl/stats/CLPStatsProvider.java | 54 ----------- .../StringColumnPreIndexStatsCollector.java | 94 +++++++++++++++++-- .../forward/ForwardIndexReaderFactory.java | 14 +-- .../creator/CLPForwardIndexCreatorTest.java | 8 +- .../mutable/CLPMutableForwardIndexTest.java | 23 ++--- .../spi/creator/IndexCreationContext.java | 19 ++-- .../spi/creator/SegmentGeneratorConfig.java | 3 +- .../segment/spi/memory/PinotByteBuffer.java | 2 +- 8 files changed, 121 insertions(+), 96 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java index 58db3628d8f5..78fa4c67562f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java @@ -33,15 +33,11 @@ public interface CLPStatsProvider { CLPStats getCLPStats(); class CLPStats { - private final EncodedMessage _clpEncodedMessage; - private final MessageEncoder _clpMessageEncoder; int _totalNumberOfDictVars = 0; int _totalNumberOfEncodedVars = 0; int _maxNumberOfEncodedVars = 0; private String[] _sortedLogTypeValues; private String[] _sortedDictVarValues; - private Set _logTypes = new ObjectOpenHashSet<>(AbstractColumnStatisticsCollector.INITIAL_HASH_SET_SIZE); - private Set _dictVars = new ObjectOpenHashSet<>(AbstractColumnStatisticsCollector.INITIAL_HASH_SET_SIZE); public CLPStats(String[] sortedLogTypeValues, String[] sortedDictVarValues, int totalNumberOfDictVars, int totalNumberOfEncodedVars, int maxNumberOfEncodedVars) { @@ -50,56 +46,6 @@ public CLPStats(String[] sortedLogTypeValues, String[] sortedDictVarValues, int _totalNumberOfDictVars = totalNumberOfDictVars; _totalNumberOfEncodedVars = totalNumberOfEncodedVars; _maxNumberOfEncodedVars = maxNumberOfEncodedVars; - _clpEncodedMessage = null; - _clpMessageEncoder = null; - } - - public CLPStats() { - _clpEncodedMessage = new EncodedMessage(); - _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, - BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); - } - - public void collect(String value) { - String logType; - String[] dictVars; - Long[] encodedVars; - - try { - _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); - logType = _clpEncodedMessage.getLogTypeAsString(); - dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); - encodedVars = _clpEncodedMessage.getEncodedVarsAsBoxedLongs(); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to encode message: " + value, e); - } - - if (logType == null) { - logType = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; - } - - if (dictVars == null) { - dictVars = new String[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING}; - } - - if (encodedVars == null) { - encodedVars = new Long[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_LONG}; - } - - _logTypes.add(logType); - _dictVars.addAll(Arrays.asList(dictVars)); - _totalNumberOfDictVars += dictVars.length; - _totalNumberOfEncodedVars += encodedVars.length; - _maxNumberOfEncodedVars = Math.max(_maxNumberOfEncodedVars, encodedVars.length); - } - - public void seal() { - _sortedLogTypeValues = _logTypes.toArray(new String[0]); - _logTypes = null; - Arrays.sort(_sortedLogTypeValues); - _sortedDictVarValues = _dictVars.toArray(new String[0]); - _dictVars = null; - Arrays.sort(_sortedDictVarValues); } public void clear() { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java index 98d7bb3a7d63..67277eba022c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java @@ -18,11 +18,17 @@ */ package org.apache.pinot.segment.local.segment.creator.impl.stats; +import com.google.common.annotations.VisibleForTesting; +import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; +import com.yscope.clp.compressorfrontend.EncodedMessage; +import com.yscope.clp.compressorfrontend.MessageEncoder; import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; +import java.io.IOException; import java.util.Arrays; import java.util.Set; import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; import org.apache.pinot.spi.config.table.FieldConfig; +import org.apache.pinot.spi.data.FieldSpec; import static java.nio.charset.StandardCharsets.UTF_8; @@ -34,12 +40,12 @@ public class StringColumnPreIndexStatsCollector extends AbstractColumnStatistics private int _maxRowLength = 0; private String[] _sortedValues; private boolean _sealed = false; - private CLPStats _clpStats; + private CLPStatsCollector _clpStatsCollector; public StringColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { super(column, statsCollectorConfig); if (_fieldConfig != null && _fieldConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { - _clpStats = new CLPStats(); + _clpStatsCollector = new CLPStatsCollector(); } } @@ -53,8 +59,8 @@ public void collect(Object entry) { for (Object obj : values) { String value = (String) obj; _values.add(value); - if (_clpStats != null) { - _clpStats.collect(value); + if (_clpStatsCollector != null) { + _clpStatsCollector.collect(value); } int length = value.getBytes(UTF_8).length; @@ -69,8 +75,8 @@ public void collect(Object entry) { } else { String value = (String) entry; addressSorted(value); - if (_clpStats != null) { - _clpStats.collect(value); + if (_clpStatsCollector != null) { + _clpStatsCollector.collect(value); } if (_values.add(value)) { if (isPartitionEnabled()) { @@ -87,7 +93,10 @@ public void collect(Object entry) { @Override public CLPStats getCLPStats() { - return _clpStats; + if (_sealed) { + return _clpStatsCollector.getCLPStats(); + } + throw new IllegalStateException("you must seal the collector first before asking for clp stats"); } @Override @@ -141,10 +150,77 @@ public void seal() { _sortedValues = _values.toArray(new String[0]); _values = null; Arrays.sort(_sortedValues); - if (_clpStats != null) { - _clpStats.seal(); + if (_clpStatsCollector != null) { + _clpStatsCollector.seal(); } _sealed = true; } } + + @VisibleForTesting + public static class CLPStatsCollector { + private final EncodedMessage _clpEncodedMessage; + private final MessageEncoder _clpMessageEncoder; + int _totalNumberOfDictVars = 0; + int _totalNumberOfEncodedVars = 0; + int _maxNumberOfEncodedVars = 0; + private Set _logTypes = new ObjectOpenHashSet<>(AbstractColumnStatisticsCollector.INITIAL_HASH_SET_SIZE); + private Set _dictVars = new ObjectOpenHashSet<>(AbstractColumnStatisticsCollector.INITIAL_HASH_SET_SIZE); + private CLPStats _clpStats; + + public CLPStatsCollector() { + _clpEncodedMessage = new EncodedMessage(); + _clpMessageEncoder = new MessageEncoder(BuiltInVariableHandlingRuleVersions.VariablesSchemaV2, + BuiltInVariableHandlingRuleVersions.VariableEncodingMethodsV1); + } + + public void collect(String value) { + String logType; + String[] dictVars; + Long[] encodedVars; + + try { + _clpMessageEncoder.encodeMessage(value, _clpEncodedMessage); + logType = _clpEncodedMessage.getLogTypeAsString(); + dictVars = _clpEncodedMessage.getDictionaryVarsAsStrings(); + encodedVars = _clpEncodedMessage.getEncodedVarsAsBoxedLongs(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to encode message: " + value, e); + } + + if (logType == null) { + logType = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; + } + + if (dictVars == null) { + dictVars = new String[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_STRING}; + } + + if (encodedVars == null) { + encodedVars = new Long[]{FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_LONG}; + } + + _logTypes.add(logType); + _dictVars.addAll(Arrays.asList(dictVars)); + _totalNumberOfDictVars += dictVars.length; + _totalNumberOfEncodedVars += encodedVars.length; + _maxNumberOfEncodedVars = Math.max(_maxNumberOfEncodedVars, encodedVars.length); + } + + public void seal() { + String[] sortedLogTypeValues = _logTypes.toArray(new String[0]); + _logTypes = null; + Arrays.sort(sortedLogTypeValues); + String[] sortedDictVarValues = _dictVars.toArray(new String[0]); + _dictVars = null; + Arrays.sort(sortedDictVarValues); + _clpStats = + new CLPStats(sortedLogTypeValues, sortedDictVarValues, _totalNumberOfDictVars, _totalNumberOfEncodedVars, + _maxNumberOfEncodedVars); + } + + public CLPStats getCLPStats() { + return _clpStats; + } + } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index 236d04ba1e1e..86a1038dad11 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -60,6 +60,13 @@ public static ForwardIndexReaderFactory getInstance() { return StandardIndexes.forward(); } + @Override + protected ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata, + ForwardIndexConfig indexConfig) + throws IndexReaderConstraintException { + return createIndexReader(dataBuffer, metadata); + } + public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata) { if (metadata.hasDictionary()) { if (metadata.isSingleValue()) { @@ -120,11 +127,4 @@ private static ForwardIndexReader createNonV4RawIndexReader(PinotDataBuffer data } } } - - @Override - protected ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata, - ForwardIndexConfig indexConfig) - throws IndexReaderConstraintException { - return createIndexReader(dataBuffer, metadata); - } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java index c9d36bd7e843..ba4375ee951c 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java @@ -65,11 +65,11 @@ public void testCLPWriter() "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); logLines.add( - "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12" - + ".15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); + "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 0.0" + + ".0.0 GET https://0.0.0.0:8443/health?checkType=liveness, content-type null status code 200 OK"); logLines.add( - "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 10.12" - + ".19.5 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 0.0" + + ".0.0 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + "application/json status code 200 OK"); Schema schema = new Schema(); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java index 417577a5245c..47cb494f0f18 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java @@ -24,6 +24,7 @@ import org.apache.pinot.segment.local.io.writer.impl.DirectMemoryManager; import org.apache.pinot.segment.local.realtime.impl.forward.CLPMutableForwardIndex; import org.apache.pinot.segment.local.segment.creator.impl.stats.CLPStatsProvider; +import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; import org.apache.pinot.spi.data.FieldSpec; import org.testng.Assert; @@ -59,18 +60,16 @@ public void testString() List logLines = new ArrayList<>(); logLines.add( "2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] " - + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: " - + "true"); + + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); logLines.add( "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " - + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: " - + "true"); + + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); logLines.add( - "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 10.12" - + ".15.1 GET https://10.12.15.10:8443/health?checkType=liveness, content-type null status code 200 OK"); + "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 0.0" + + ".0.0 GET https://0.0.0.0:8443/health?checkType=liveness, content-type null status code 200 OK"); logLines.add( - "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 10.12" - + ".19.5 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 0.0" + + ".0.0 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + "application/json status code 200 OK"); for (int i = 0; i < rows; i++) { @@ -82,11 +81,13 @@ public void testString() } // Verify clp stats - CLPStatsProvider.CLPStats stats = new CLPStatsProvider.CLPStats(); + StringColumnPreIndexStatsCollector.CLPStatsCollector statsCollector = + new StringColumnPreIndexStatsCollector.CLPStatsCollector(); for (int i = 0; i < rows; i++) { - stats.collect(logLines.get(i)); + statsCollector.collect(logLines.get(i)); } - stats.seal(); + statsCollector.seal(); + CLPStatsProvider.CLPStats stats = statsCollector.getCLPStats(); CLPStatsProvider.CLPStats mutableIndexStats = readerWriter.getCLPStats(); Assert.assertEquals(stats.getTotalNumberOfDictVars(), mutableIndexStats.getTotalNumberOfDictVars()); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java index 2ddfe5499d50..52df382efa2d 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java @@ -114,9 +114,8 @@ final class Builder { private boolean _textCommitOnClose; public Builder withColumnIndexCreationInfo(ColumnIndexCreationInfo columnIndexCreationInfo) { - return withLengthOfLongestEntry( - columnIndexCreationInfo.getLengthOfLongestEntry()).withMaxNumberOfMultiValueElements( - columnIndexCreationInfo.getMaxNumberOfMultiValueElements()) + return withLengthOfLongestEntry(columnIndexCreationInfo.getLengthOfLongestEntry()) + .withMaxNumberOfMultiValueElements(columnIndexCreationInfo.getMaxNumberOfMultiValueElements()) .withMaxRowLengthInBytes(columnIndexCreationInfo.getMaxRowLengthInBytes()) .withMinValue((Comparable) columnIndexCreationInfo.getMin()) .withMaxValue((Comparable) columnIndexCreationInfo.getMax()) @@ -124,7 +123,8 @@ public Builder withColumnIndexCreationInfo(ColumnIndexCreationInfo columnIndexCr .withSortedUniqueElementsArray(columnIndexCreationInfo.getSortedUniqueElementsArray()) .withColumnStatistics(columnIndexCreationInfo.getColumnStatistics()) .withCardinality(columnIndexCreationInfo.getDistinctValueCount()) - .withFixedLength(columnIndexCreationInfo.isFixedLength()).sorted(columnIndexCreationInfo.isSorted()); + .withFixedLength(columnIndexCreationInfo.isFixedLength()) + .sorted(columnIndexCreationInfo.isSorted()); } public Builder withColumnStatistics(ColumnStatistics columnStatistics) { @@ -268,11 +268,12 @@ final class Common implements IndexCreationContext { private final boolean _textCommitOnClose; private final ColumnStatistics _columnStatistics; - public Common(File indexDir, int lengthOfLongestEntry, int maxNumberOfMultiValueElements, int maxRowLengthInBytes, - boolean onHeap, FieldSpec fieldSpec, boolean sorted, int cardinality, int totalNumberOfEntries, int totalDocs, - boolean hasDictionary, Comparable minValue, Comparable maxValue, boolean forwardIndexDisabled, - Object sortedUniqueElementsArray, boolean optimizeDictionary, boolean fixedLength, boolean textCommitOnClose, - ColumnStatistics columnStatistics) { + public Common(File indexDir, int lengthOfLongestEntry, + int maxNumberOfMultiValueElements, int maxRowLengthInBytes, boolean onHeap, + FieldSpec fieldSpec, boolean sorted, int cardinality, int totalNumberOfEntries, + int totalDocs, boolean hasDictionary, Comparable minValue, Comparable maxValue, + boolean forwardIndexDisabled, Object sortedUniqueElementsArray, boolean optimizeDictionary, + boolean fixedLength, boolean textCommitOnClose, ColumnStatistics columnStatistics) { _indexDir = indexDir; _lengthOfLongestEntry = lengthOfLongestEntry; _maxNumberOfMultiValueElements = maxNumberOfMultiValueElements; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java index e7e1827efabf..6305dcd8522d 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java @@ -273,7 +273,8 @@ private void extractCompressionCodecConfigsFromTableConfig(TableConfig tableConf List fieldConfigList = tableConfig.getFieldConfigList(); if (fieldConfigList != null) { for (FieldConfig fieldConfig : fieldConfigList) { - if (fieldConfig.getEncodingType() == FieldConfig.EncodingType.RAW && fieldConfig.getCompressionCodec() != null + if (fieldConfig.getEncodingType() == FieldConfig.EncodingType.RAW + && fieldConfig.getCompressionCodec() != null && fieldConfig.getCompressionCodec().isApplicableToRawIndex()) { _rawIndexCreationColumns.add(fieldConfig.getName()); _rawIndexCompressionType.put(fieldConfig.getName(), diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java index aeee3ff98a58..1cb357ea442b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotByteBuffer.java @@ -60,7 +60,7 @@ static PinotByteBuffer mapFile(File file, boolean readOnly, long offset, int siz } } - public PinotByteBuffer(ByteBuffer buffer, boolean closeable, boolean flushable) { + private PinotByteBuffer(ByteBuffer buffer, boolean closeable, boolean flushable) { super(closeable); _buffer = buffer; _flushable = flushable; From 013ebc1f5af967b4ca3490513335fb6a18397284 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Wed, 13 Mar 2024 13:48:02 +0530 Subject: [PATCH 27/30] Lint --- .../tests/CLPEncodingRealtimeIntegrationTest.java | 7 ++++--- .../segment/creator/impl/stats/CLPStatsProvider.java | 10 ---------- .../forward/mutable/CLPMutableForwardIndexTest.java | 6 ++++-- 3 files changed, 8 insertions(+), 15 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java index c94f1f723f9c..c8b5d88646c8 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/CLPEncodingRealtimeIntegrationTest.java @@ -29,11 +29,10 @@ import org.apache.pinot.spi.config.table.ingestion.TransformConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.util.TestUtils; +import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.testng.Assert.assertEquals; - public class CLPEncodingRealtimeIntegrationTest extends BaseClusterIntegrationTestSet { private List _avroFiles; @@ -94,7 +93,9 @@ protected List getNoDictionaryColumns() { @Test public void testValues() throws Exception { - assertEquals(getCurrentCountStarResult(), getCountStarResult()); + Assert.assertEquals(getPinotConnection().execute( + "SELECT count(*) FROM " + getTableName() + " WHERE REGEXP_LIKE(logLine, '.*executor.*')").getResultSet(0) + .getLong(0), 53); } protected int getRealtimeSegmentFlushSize() { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java index 78fa4c67562f..b8611886206b 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/CLPStatsProvider.java @@ -18,16 +18,6 @@ */ package org.apache.pinot.segment.local.segment.creator.impl.stats; -import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; -import com.yscope.clp.compressorfrontend.EncodedMessage; -import com.yscope.clp.compressorfrontend.MessageEncoder; -import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; -import java.io.IOException; -import java.util.Arrays; -import java.util.Set; -import org.apache.pinot.spi.data.FieldSpec; - - public interface CLPStatsProvider { CLPStats getCLPStats(); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java index 47cb494f0f18..c01d256941dd 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java @@ -60,10 +60,12 @@ public void testString() List logLines = new ArrayList<>(); logLines.add( "2023/10/26 00:03:10.168 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32c_DEFAULT)] " - + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective: true"); + + "Event pinot::DEFAULT::4a02a32c_DEFAULT : Refreshed 35 property LiveInstance took 5 ms. Selective:" + + " true"); logLines.add( "2023/10/26 00:03:10.169 INFO [PropertyCache] [HelixController-pipeline-default-pinot-(4a02a32d_DEFAULT)] " - + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective: true"); + + "Event pinot::DEFAULT::4a02a32d_DEFAULT : Refreshed 81 property LiveInstance took 4 ms. Selective:" + + " true"); logLines.add( "2023/10/27 16:35:10.470 INFO [ControllerResponseFilter] [grizzly-http-server-2] Handled request from 0.0" + ".0.0 GET https://0.0.0.0:8443/health?checkType=liveness, content-type null status code 200 OK"); From 342ce5c9e4f3175426a19f76edb006973918bad5 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Tue, 19 Mar 2024 13:46:22 +0530 Subject: [PATCH 28/30] Fix context close --- .../readers/forward/CLPForwardIndexReaderV1.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index 135035bb978a..3720683b4481 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -159,9 +159,15 @@ public CLPReaderContext(FixedBitMVForwardIndexReader.Context dictVarsReaderConte @Override public void close() throws IOException { - _dictVarsReaderContext.close(); - _logTypeReaderContext.close(); - _encodedVarReaderContext.close(); + if (_dictVarsReaderContext != null) { + _dictVarsReaderContext.close(); + } + if (_logTypeReaderContext != null) { + _logTypeReaderContext.close(); + } + if (_encodedVarReaderContext != null) { + _encodedVarReaderContext.close(); + } } } } From e2e51800fbba512c6c948b4a67074be7120aafa8 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Tue, 19 Mar 2024 16:29:57 +0530 Subject: [PATCH 29/30] Add recordByteRanges support --- .../local/io/util/VarLengthValueReader.java | 11 +++++ .../forward/CLPForwardIndexReaderV1.java | 46 +++++++++++++++++++ 2 files changed, 57 insertions(+) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueReader.java index cd1346896a89..d9384bf16f0e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueReader.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/util/VarLengthValueReader.java @@ -18,6 +18,8 @@ */ package org.apache.pinot.segment.local.io.util; +import java.util.List; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; import static java.nio.charset.StandardCharsets.UTF_8; @@ -101,6 +103,15 @@ public String getUnpaddedString(int index, int numBytesPerValue, byte[] buffer) return new String(buffer, 0, length, UTF_8); } + public void recordOffsetRanges(int index, long baseOffset, List rangeList) { + int offsetPosition = _dataSectionStartOffSet + Integer.BYTES * index; + int startOffset = _dataBuffer.getInt(offsetPosition); + int endOffset = _dataBuffer.getInt(offsetPosition + Integer.BYTES); + rangeList.add(new ForwardIndexReader.ByteRange(offsetPosition + baseOffset, 2 * Integer.BYTES)); + int length = endOffset - startOffset; + rangeList.add(new ForwardIndexReader.ByteRange(startOffset + baseOffset, length)); + } + @Override public String getPaddedString(int index, int numBytesPerValue, byte[] buffer) { throw new UnsupportedOperationException(); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java index 3720683b4481..d68a80ae2069 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/CLPForwardIndexReaderV1.java @@ -21,6 +21,8 @@ import com.yscope.clp.compressorfrontend.BuiltInVariableHandlingRuleVersions; import com.yscope.clp.compressorfrontend.MessageDecoder; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.pinot.segment.local.io.util.PinotDataBitSet; import org.apache.pinot.segment.local.io.util.VarLengthValueReader; import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1; @@ -36,10 +38,15 @@ public class CLPForwardIndexReaderV1 implements ForwardIndexReader ranges) { + int logTypeDictId = _logTypeFwdIndexReader.getDictId(docId, context._logTypeReaderContext); + ranges.add(new ByteRange(_logTypeFwdIndexReaderStartOffset + _logTypeFwdIndexReader.getRawDataStartOffset() + + (long) _logTypeFwdIndexReader.getDocLength() * docId, _logTypeFwdIndexReader.getDocLength())); + _logTypeDictReader.recordOffsetRanges(logTypeDictId, _logTypeDictReaderStartOffset, ranges); + + int[] dictVarsDictIds = _dictVarsFwdIndexReader.getDictIdMV(docId, context._dictVarsReaderContext); + List fwdIndexByteRanges = new ArrayList<>(); + _dictVarsFwdIndexReader.recordDocIdByteRanges(docId, context._dictVarsReaderContext, fwdIndexByteRanges); + for (ByteRange range : fwdIndexByteRanges) { + ranges.add(new ByteRange(_dictVarsFwdIndexReaderStartOffset + range.getOffset(), range.getSizeInBytes())); + } + fwdIndexByteRanges.clear(); + + for (int dictVarsDictId : dictVarsDictIds) { + _dictVarsDictReader.recordOffsetRanges(dictVarsDictId, _dictVarsDictReaderStartOffset, ranges); + } + _encodedVarFwdIndexReader.recordDocIdByteRanges(docId, context._encodedVarReaderContext, fwdIndexByteRanges); + for (ByteRange range : fwdIndexByteRanges) { + ranges.add(new ByteRange(_encodedVarFwdIndexReaderStartOffset + range.getOffset(), range.getSizeInBytes())); + } + } + + @Override + public boolean isFixedOffsetMappingType() { + return false; + } + @Override public ChunkCompressionType getCompressionType() { return ChunkCompressionType.PASS_THROUGH; From 78532fe434c1192263c500fb2671bc81e6ec4b25 Mon Sep 17 00:00:00 2001 From: Saurabh Dubey Date: Thu, 21 Mar 2024 14:43:17 +0530 Subject: [PATCH 30/30] Review comments --- .../segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java | 2 +- .../creator/impl/stats/StringColumnPreIndexStatsCollector.java | 2 +- .../segment/index/forward/ForwardIndexCreatorFactory.java | 1 - .../segment/index/creator/CLPForwardIndexCreatorTest.java | 3 ++- .../index/forward/mutable/CLPMutableForwardIndexTest.java | 1 + .../java/org/apache/pinot/spi/config/table/FieldConfig.java | 2 +- 6 files changed, 6 insertions(+), 5 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java index 5a1a436d5780..c681265ffb9f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/CLPForwardIndexCreatorV1.java @@ -265,7 +265,7 @@ private void copyFileIntoBuffer(File file) throws IOException { @Override public void close() throws IOException { - // Delete all temp file + // Delete all temp files _dataFile.close(); FileUtils.deleteDirectory(_intermediateFilesDir); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java index 67277eba022c..d606c784d20f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/StringColumnPreIndexStatsCollector.java @@ -96,7 +96,7 @@ public CLPStats getCLPStats() { if (_sealed) { return _clpStatsCollector.getCLPStats(); } - throw new IllegalStateException("you must seal the collector first before asking for clp stats"); + throw new IllegalStateException("The collector must be sealed before calling getCLPStats"); } @Override diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java index 61ee335c0a4a..cd0417a25842 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java @@ -72,7 +72,6 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex // Dictionary disabled columns DataType storedType = fieldSpec.getDataType().getStoredType(); if (indexConfig.getCompressionCodec() == FieldConfig.CompressionCodec.CLP) { - // CLP compression codec return new CLPForwardIndexCreatorV1(indexDir, columnName, numTotalDocs, context.getColumnStatistics()); } ChunkCompressionType chunkCompressionType = indexConfig.getChunkCompressionType(); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java index ba4375ee951c..c97a1d318fb3 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/CLPForwardIndexCreatorTest.java @@ -71,6 +71,7 @@ public void testCLPWriter() "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 0.0" + ".0.0 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + "application/json status code 200 OK"); + logLines.add("null"); Schema schema = new Schema(); schema.addField(new DimensionFieldSpec("column1", FieldSpec.DataType.STRING, true)); @@ -92,7 +93,7 @@ public void testCLPWriter() File indexFile = new File(TEMP_DIR, "column1" + V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION); CLPForwardIndexCreatorV1 clpForwardIndexCreatorV1 = - new CLPForwardIndexCreatorV1(TEMP_DIR, "column1", 4, statsCollector); + new CLPForwardIndexCreatorV1(TEMP_DIR, "column1", logLines.size(), statsCollector); for (String logLine : logLines) { clpForwardIndexCreatorV1.putString(logLine); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java index c01d256941dd..a926b9a396ad 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/CLPMutableForwardIndexTest.java @@ -73,6 +73,7 @@ public void testString() "2023/10/27 16:35:10.607 INFO [ControllerResponseFilter] [grizzly-http-server-6] Handled request from 0.0" + ".0.0 GET https://pinot-pinot-broker-headless.managed.svc.cluster.local:8093/tables, content-type " + "application/json status code 200 OK"); + logLines.add("null"); for (int i = 0; i < rows; i++) { readerWriter.setString(i, logLines.get(i)); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java index e5708d1ab2eb..704cb2e01c64 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java @@ -127,7 +127,7 @@ public enum CompressionCodec { ZSTANDARD(true, false), LZ4(true, false), // CLP is a special type of compression codec that isn't generally applicable to all RAW columns and has a - // special handling (see {@link CLPForwardIndexCreatorV1}) + // special handling for log lines (see {@link CLPForwardIndexCreatorV1}) CLP(false, false), // For MV dictionary encoded forward index, add a second level dictionary encoding for the multi-value entries