You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/08/15 06:14:30 UTC

[GitHub] [druid] jihoonson opened a new pull request #11597: Segments table performance enhancement

jihoonson opened a new pull request #11597:
URL: https://github.com/apache/druid/pull/11597


   ### Description
   
   This PR improves the performance of the segments table by 1) exploiting pre-sorted data structure and 2) caching results to avoid repeated execution of expensive operations. For 1), when the published segments caching is enabled, the cached published segments in `MetadataSegmentView` is sorted in the same order used to sort the available segments in `DruidSchema`. When a query scans segments, the query engine now can merge published and available segments using the merge sorted algorithm as they are sorted in the same order. The previous hash-based merge algorithm is still used when the caching is disabled. Also, `druid.sql.planner.forceHashBasedMergeForSegmentsTable` is added to force the hash-based merge even when the caching is enabled, which is off by default (merge-sorted is on by default).
   
   For 2), scanning the segments table requires to convert timestamps to strings and serialize other non-primitive types to JSON strings. Since string conversion and JSON serialization are expensive operations, a cache is added to avoid calling those operations for the same object repeatedly. The cache can be GCed once the scan is finished.
   
   The benchmark results are in the below. This might be a bit biased since all segments had the same metadata, where the technique used for 2) can show the best performance. In reality, since segments can have different segment granularity, dimensions, and metrics, the performance gain could be lower than what is shown here.
   
   ```
   master
   
    Benchmark                            (availableSegmentsInterval)  (forceHashBasedMerge)  (numSegmentsPerInterval)  (publishedSegmentsInterval)  (segmentGranularity)  (sql)  Mode  Cnt     Score    Error  Units
    SystemSchemaBenchmark.segmentsTable               2021-01-01/P3Y                   true                        10               2021-01-02/P3Y                   DAY      0  avgt   10    30.596 ±  0.399  ms/op
    SystemSchemaBenchmark.segmentsTable               2021-01-01/P3Y                   true                        10               2021-01-02/P3Y                   DAY      1  avgt   10    39.703 ±  0.238  ms/op
    SystemSchemaBenchmark.segmentsTable               2021-01-01/P3Y                   true                       100               2021-01-02/P3Y                   DAY      0  avgt   10   382.867 ±  6.577  ms/op
    SystemSchemaBenchmark.segmentsTable               2021-01-01/P3Y                   true                       100               2021-01-02/P3Y                   DAY      1  avgt   10   387.911 ±  3.867  ms/op
    SystemSchemaBenchmark.segmentsTable               2021-01-01/P3Y                   true                      1000               2021-01-02/P3Y                   DAY      0  avgt   10  3971.577 ± 70.304  ms/op
    SystemSchemaBenchmark.segmentsTable               2021-01-01/P3Y                   true                      1000               2021-01-02/P3Y                   DAY      1  avgt   10  5334.702 ± 44.773  ms/op
   ```
   
   ```
   PR
   
    * Benchmark                             (availableSegmentsInterval)  (forceHashBasedMerge)  (numSegmentsPerInterval)  (publishedSegmentsInterval)  (segmentGranularity)  (sql)  Mode  Cnt     Score    Error  Units
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                        10               2021-01-02/P3Y                   DAY      0  avgt   10    14.914 ±  0.305  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                        10               2021-01-02/P3Y                   DAY      1  avgt   10    21.591 ±  0.139  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                       100               2021-01-02/P3Y                   DAY      0  avgt   10   199.416 ±  2.208  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                       100               2021-01-02/P3Y                   DAY      1  avgt   10   203.366 ±  4.706  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                      1000               2021-01-02/P3Y                   DAY      0  avgt   10  2110.735 ± 65.326  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                      1000               2021-01-02/P3Y                   DAY      1  avgt   10  2615.910 ± 69.914  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                        10               2021-01-02/P3Y                   DAY      0  avgt   10    11.967 ±  0.308  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                        10               2021-01-02/P3Y                   DAY      1  avgt   10    18.917 ±  0.220  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                       100               2021-01-02/P3Y                   DAY      0  avgt   10   123.630 ±  0.839  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                       100               2021-01-02/P3Y                   DAY      1  avgt   10   132.013 ±  0.664  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                      1000               2021-01-02/P3Y                   DAY      0  avgt   10  1303.304 ± 40.063  ms/op
    * SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                      1000               2021-01-02/P3Y                   DAY      1  avgt   10  1799.447 ± 23.659  ms/op
   ```
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `MetadataSegmentView`
    * `SegmentsTableRow`
    * `SystemSchema`
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [x] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



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


[GitHub] [druid] jihoonson commented on a change in pull request #11597: Segments table performance enhancement

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #11597:
URL: https://github.com/apache/druid/pull/11597#discussion_r725439798



##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentsTableBenchamrkBase.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.BrokerSegmentWatcherConfig;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.data.input.impl.DimensionSchema;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.discovery.DruidLeaderClient;
+import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.NonnullPair;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.QuerySegmentWalker;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.segment.loading.SegmentLoader;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.security.AuthTestUtils;
+import org.apache.druid.server.security.AuthenticationResult;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.sql.calcite.planner.Calcites;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerFactory;
+import org.apache.druid.sql.calcite.planner.SegmentsTableConfig;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeDruidNodeDiscoveryProvider;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeHttpClient;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeServerInventoryView;
+import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
+import org.apache.druid.sql.calcite.util.TestServerInventoryView;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.Interval;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+public abstract class SegmentsTableBenchamrkBase

Review comment:
       Thanks. I renamed it to `SegmentsTableQueryTestSuite`.

##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentsTableBenchmarkQueryTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.druid.common.guava.SettableSupplier;
+import org.apache.druid.sql.calcite.planner.DruidPlanner;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerResult;
+import org.apache.druid.sql.calcite.planner.SegmentsTableConfig;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class SegmentsTableBenchmarkQueryTest extends SegmentsTableBenchamrkBase

Review comment:
       Hmm, my intention was that this test verifies the results of the queries used for the segments table benchmark. Is `SegmentsTableBenchmarkQueriesTest` better?

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/ObjectStringCache.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import org.apache.druid.common.guava.SettableSupplier;
+
+import javax.annotation.Nullable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * A simple cache to hold string representation of the given keys.
+ * This cache can be used to avoid repetitive calls of expensive {@link Object#toString()}
+ * or JSON serializations.
+ *
+ * This cache uses a simple caching policy that keeps only the first N entries.
+ * The cache size is limited by the {@link #cacheSize} constructor parameter.
+ *
+ * @param <K> cache key type
+ *
+ * @see SegmentsTableRow#toObjectArray
+ */
+class ObjectStringCache<K>

Review comment:
       Good question. I didn't use the caffeine at first because 1) the strategy of caching first N entries will likely work well for the access pattern for segments table columns in most cases and 2) I wasn't sure about the overhead in caffeine for concurrent access support. For 1), I still think caching first N would probably be OK in most cases, but perhaps caffeine's Window TinyLfu could be better in some edge cases. For 2), I ran the benchmark again with a caffeine-based cache with 1MB of cache size and didn't see any difference in the benchmark results as seen below (numbers are slightly larger than the one in the PR description, but I think it's my computer as I got similarly slightly larger numbers with the original map-based cache). So I switched the cache to using caffeine instead. 
   
   ```
   Benchmark                             (availableSegmentsInterval)  (forceHashBasedMerge)  (numSegmentsPerInterval)  (publishedSegmentsInterval)  (segmentGranularity)  (sql)  Mode  Cnt     Score    Error  Units
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                        10               2021-01-02/P3Y                   DAY      0  avgt   10    18.890 ±  1.291  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                        10               2021-01-02/P3Y                   DAY      1  avgt   10    25.546 ±  0.764  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                       100               2021-01-02/P3Y                   DAY      0  avgt   10   201.993 ±  2.534  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                       100               2021-01-02/P3Y                   DAY      1  avgt   10   208.421 ±  2.847  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                      1000               2021-01-02/P3Y                   DAY      0  avgt   10  2195.274 ± 70.780  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                   true                      1000               2021-01-02/P3Y                   DAY      1  avgt   10  2743.012 ± 88.182  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                        10               2021-01-02/P3Y                   DAY      0  avgt   10    15.990 ±  0.355  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                        10               2021-01-02/P3Y                   DAY      1  avgt   10    22.791 ±  0.625  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                       100               2021-01-02/P3Y                   DAY      0  avgt   10   135.999 ±  1.466  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                       100               2021-01-02/P3Y                   DAY      1  avgt   10   145.376 ±  1.334  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                      1000               2021-01-02/P3Y                   DAY      0  avgt   10  1254.524 ± 39.122  ms/op
   SegmentsTableBenchmark.segmentsTable               2021-01-01/P3Y                  false                      1000               2021-01-02/P3Y                   DAY      1  avgt   10  1891.175 ± 25.529  ms/op
   ```




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



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


[GitHub] [druid] clintropolis commented on a change in pull request #11597: Segments table performance enhancement

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #11597:
URL: https://github.com/apache/druid/pull/11597#discussion_r723045248



##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentsTableBenchamrkBase.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.BrokerSegmentWatcherConfig;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.data.input.impl.DimensionSchema;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.discovery.DruidLeaderClient;
+import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.NonnullPair;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.QuerySegmentWalker;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.segment.loading.SegmentLoader;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.security.AuthTestUtils;
+import org.apache.druid.server.security.AuthenticationResult;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.sql.calcite.planner.Calcites;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerFactory;
+import org.apache.druid.sql.calcite.planner.SegmentsTableConfig;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeDruidNodeDiscoveryProvider;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeHttpClient;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeServerInventoryView;
+import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
+import org.apache.druid.sql.calcite.util.TestServerInventoryView;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.Interval;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+public abstract class SegmentsTableBenchamrkBase

Review comment:
       typo: `SegmentsTableBenchamrkBase` -> `SegmentsTableBenchmarkBase`
   
   Side thought/nitpick, `Benchmark` doesn't seem like quite the right name since it is just setting up a set of queries to do stuff with, and happens to be used by some actual benchmarks alongside tests. But, i'm not really sure what would be a better name.. something query suite? naming is the worst, and since this is just a test class it doesn't really matter...

##########
File path: docs/configuration/index.md
##########
@@ -1764,6 +1764,8 @@ The Druid SQL server is configured through the following properties on the Broke
 |`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST API will be invoked when broker needs published segments info.|false|
 |`druid.sql.planner.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000|
 |`druid.sql.planner.authorizeSystemTablesDirectly`|If true, Druid authorizes queries against any of the system schema tables (`sys` in SQL) as `SYSTEM_TABLE` resources which require `READ` access, in addition to permissions based content filtering.|false|
+|`druid.sql.sys.segments.forceHashBasedMerge`|When `druid.sql.planner.metadataSegmentCacheEnable` is set to true, the broker uses a sort-based algorithm by default for merging the sets of published segments and available segments to process the `sys.segments` table. You can force using a hash-based merge algorithm by setting this config even when the cache is enabled. The sort-based algorithm is supposed to be faster than the hash-based algorithm in most cases.|false|

Review comment:
       ```suggestion
   |`druid.sql.sys.segments.forceHashBasedMerge`|When `druid.sql.planner.metadataSegmentCacheEnable` is set to true, the broker uses a sort-based algorithm by default for merging the sets of published and available segments to process the `sys.segments` table. Use of the hash-based merge algorithm can be forced by setting this to true, even when the cache is enabled. The sort-based algorithm should be faster than the hash-based algorithm in most cases.|false|
   ```

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/ObjectStringCache.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import org.apache.druid.common.guava.SettableSupplier;
+
+import javax.annotation.Nullable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * A simple cache to hold string representation of the given keys.
+ * This cache can be used to avoid repetitive calls of expensive {@link Object#toString()}
+ * or JSON serializations.
+ *
+ * This cache uses a simple caching policy that keeps only the first N entries.
+ * The cache size is limited by the {@link #cacheSize} constructor parameter.
+ *
+ * @param <K> cache key type
+ *
+ * @see SegmentsTableRow#toObjectArray
+ */
+class ObjectStringCache<K>

Review comment:
       maybe this class deserves its own tests?
   
   Also I wonder if using caffeine for a cache so to evict items not recently used might be better behavior than caching the first 'n'?

##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentsTableBenchmarkQueryTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.druid.common.guava.SettableSupplier;
+import org.apache.druid.sql.calcite.planner.DruidPlanner;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerResult;
+import org.apache.druid.sql.calcite.planner.SegmentsTableConfig;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class SegmentsTableBenchmarkQueryTest extends SegmentsTableBenchamrkBase

Review comment:
       similar nit about naming since this isn't testing the benchmark




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



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


[GitHub] [druid] clintropolis commented on a change in pull request #11597: Segments table performance enhancement

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #11597:
URL: https://github.com/apache/druid/pull/11597#discussion_r723045248



##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentsTableBenchamrkBase.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.BrokerSegmentWatcherConfig;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.data.input.impl.DimensionSchema;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.discovery.DruidLeaderClient;
+import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.NonnullPair;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.QuerySegmentWalker;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.segment.loading.SegmentLoader;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.security.AuthTestUtils;
+import org.apache.druid.server.security.AuthenticationResult;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.sql.calcite.planner.Calcites;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerFactory;
+import org.apache.druid.sql.calcite.planner.SegmentsTableConfig;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeDruidNodeDiscoveryProvider;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeHttpClient;
+import org.apache.druid.sql.calcite.util.CalciteTests.FakeServerInventoryView;
+import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
+import org.apache.druid.sql.calcite.util.TestServerInventoryView;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.Interval;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+public abstract class SegmentsTableBenchamrkBase

Review comment:
       typo: `SegmentsTableBenchamrkBase` -> `SegmentsTableBenchmarkBase`
   
   Side thought/nitpick, `Benchmark` doesn't seem like quite the right name since it is just setting up a set of queries to do stuff with, and happens to be used by some actual benchmarks alongside tests. But, i'm not really sure what would be a better name.. something query suite? naming is the worst, and since this is just a test class it doesn't really matter...

##########
File path: docs/configuration/index.md
##########
@@ -1764,6 +1764,8 @@ The Druid SQL server is configured through the following properties on the Broke
 |`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST API will be invoked when broker needs published segments info.|false|
 |`druid.sql.planner.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000|
 |`druid.sql.planner.authorizeSystemTablesDirectly`|If true, Druid authorizes queries against any of the system schema tables (`sys` in SQL) as `SYSTEM_TABLE` resources which require `READ` access, in addition to permissions based content filtering.|false|
+|`druid.sql.sys.segments.forceHashBasedMerge`|When `druid.sql.planner.metadataSegmentCacheEnable` is set to true, the broker uses a sort-based algorithm by default for merging the sets of published segments and available segments to process the `sys.segments` table. You can force using a hash-based merge algorithm by setting this config even when the cache is enabled. The sort-based algorithm is supposed to be faster than the hash-based algorithm in most cases.|false|

Review comment:
       ```suggestion
   |`druid.sql.sys.segments.forceHashBasedMerge`|When `druid.sql.planner.metadataSegmentCacheEnable` is set to true, the broker uses a sort-based algorithm by default for merging the sets of published and available segments to process the `sys.segments` table. Use of the hash-based merge algorithm can be forced by setting this to true, even when the cache is enabled. The sort-based algorithm should be faster than the hash-based algorithm in most cases.|false|
   ```

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/ObjectStringCache.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import org.apache.druid.common.guava.SettableSupplier;
+
+import javax.annotation.Nullable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * A simple cache to hold string representation of the given keys.
+ * This cache can be used to avoid repetitive calls of expensive {@link Object#toString()}
+ * or JSON serializations.
+ *
+ * This cache uses a simple caching policy that keeps only the first N entries.
+ * The cache size is limited by the {@link #cacheSize} constructor parameter.
+ *
+ * @param <K> cache key type
+ *
+ * @see SegmentsTableRow#toObjectArray
+ */
+class ObjectStringCache<K>

Review comment:
       maybe this class deserves its own tests?
   
   Also I wonder if using caffeine for a cache so to evict items not recently used might be better behavior than caching the first 'n'?

##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentsTableBenchmarkQueryTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.druid.sql.calcite.schema;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.druid.common.guava.SettableSupplier;
+import org.apache.druid.sql.calcite.planner.DruidPlanner;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerResult;
+import org.apache.druid.sql.calcite.planner.SegmentsTableConfig;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class SegmentsTableBenchmarkQueryTest extends SegmentsTableBenchamrkBase

Review comment:
       similar nit about naming since this isn't testing the benchmark




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



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