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/12/07 17:24:25 UTC

[GitHub] [druid] LakshSingla opened a new pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

LakshSingla opened a new pull request #12001:
URL: https://github.com/apache/druid/pull/12001


   ### Description
   
   `DruidSchema` consists of a concurrent HashMap of `DataSource` -> `Segement` -> `AvailableSegmentMetadata`. `AvailableSegmentMetadata` contains `RowSignature` of the segment, and for each segment, a new object is getting created. RowSignature is an immutable class, and hence it can be interned, and this can lead to huge savings of memory being used in broker, since a lot of the segments of a table would potentially have same RowSignature.
   
   Following are some heap dumps of the broker attached for some experiments. The data being ingested is the quickstart `wikipedia` data, but the secondary partition is dynamic with 1 max row per segment, to generate a lot of segments which the `DruidSchema` would have to cache. Also, the configuration used is `start-single-server-small`. 
   For the normal testing `apache-druid-0.22.0` was used while the patch was tested on top of the `master` branch (shouldn't cause much difference).
   
   1.
   No additional columns were appended to the existing columns of the wiki data.
   #### Non Interned
   ![Screenshot 2021-11-30 at 1 19 07 AM](https://user-images.githubusercontent.com/30999375/143933629-63366e17-1288-4ba7-8ba7-8eab811443c2.png)
   
   ![Screenshot 2021-11-30 at 1 19 45 AM](https://user-images.githubusercontent.com/30999375/143933700-c4a88d0e-64f5-4c2b-b970-b355378c3569.png)
   
   #### Interned
   ![Screenshot 2021-11-30 at 1 20 04 AM](https://user-images.githubusercontent.com/30999375/143933765-b888b5fa-92ba-4dea-a9c0-850e5dab914a.png)
   
   ![Screenshot 2021-11-30 at 1 20 19 AM](https://user-images.githubusercontent.com/30999375/143933804-26b362d8-a091-4f71-a2d6-4e914355e01b.png)
   
   2.
   51 addititonal columns were appended to the original columns with names "random-#" to increase the memory footprint of the RowSignature.
   #### Non Interned
   ![Screenshot 2021-11-30 at 1 20 42 AM](https://user-images.githubusercontent.com/30999375/143933860-78c483d5-7aef-44ba-959c-93162c2f8f00.png)
   
   ![Screenshot 2021-11-30 at 1 20 54 AM](https://user-images.githubusercontent.com/30999375/143933892-035c6023-5365-465f-afcc-500061c3bedb.png)
    
   #### Interned
   
   ![Screenshot 2021-11-30 at 1 21 15 AM](https://user-images.githubusercontent.com/30999375/143933921-4b14e2c3-abbc-497b-9b13-9031d5ed26ba.png)
   
   ![Screenshot 2021-11-30 at 1 21 33 AM](https://user-images.githubusercontent.com/30999375/143933941-986d69c4-2e0b-40a9-8763-b06ec2cd767a.png)
   
   This change precomputes the hash of the RowSignature, since its an immutable object, as well as uses an interner in the DruidSchema to generate the AvailableSegmentMetadata objects.
   
   Benchmark was added to see the effect of interning on the time required to create the DruidSchema object. It can be removed in the end, if not required. Also, a couple of methods had to be moved from package-private to protected for use in the benchmark, which might be an argument for punting it in the final changes.
   
   Synthetic test scenario used to measure the impact, so it might differ in a real use case, but I think the memory savings should still be present in real use cases. 
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `DruidSchema` - Intern the RowSignature
    * `RowSignature` - Precompute the hash function
   
   <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.)
   - [ ] 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.
   - [ ] 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.
   - [x] 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] LakshSingla commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -791,7 +795,7 @@ DruidTable buildDruidTable(final String dataSource)
     } else {
       tableDataSource = new TableDataSource(dataSource);
     }
-    return new DruidTable(tableDataSource, builder.build(), null, isJoinable, isBroadcast);
+    return new DruidTable(tableDataSource, ROW_SIGNATURE_INTERNER.intern(builder.build()), null, isJoinable, isBroadcast);

Review comment:
       👍 Updated




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

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] LakshSingla commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -791,7 +795,7 @@ DruidTable buildDruidTable(final String dataSource)
     } else {
       tableDataSource = new TableDataSource(dataSource);
     }
-    return new DruidTable(tableDataSource, builder.build(), null, isJoinable, isBroadcast);
+    return new DruidTable(tableDataSource, ROW_SIGNATURE_INTERNER.intern(builder.build()), null, isJoinable, isBroadcast);

Review comment:
       Since there's no advantage or disadvantage  of updating it, should I change it to maintain consistency in the class or revert it since it provides no benefit.




-- 
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] LakshSingla commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 3)
+@Measurement(iterations = 15)
+public class DruidSchemaInternRowSignatureBenchmark
+{
+
+  @Param({"10000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+    @Override
+    public Set<SegmentId> refreshSegments(final Set<SegmentId> segments) throws IOException
+    {
+      return super.refreshSegments(segments);
+    }
+
+
+    @Override
+    public void addSegment(final DruidServerMetadata server, final DataSegment segment)
+    {
+      super.addSegment(server, segment);
+    }
+
+    @Override
+    protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(Iterable<SegmentId> segments)
+    {
+      final int numColumns = 1000;
+      Map<String, ColumnAnalysis> columnToAnalysisMap = new HashMap<>();
+      for (int i = 0; i < numColumns; ++i) {
+        columnToAnalysisMap.put(
+            "col" + i,
+            new ColumnAnalysis(
+                ColumnType.STRING,
+                null,
+                false,
+                false,
+                40,
+                null,
+                null,
+                null,
+                null
+            )
+        );
+      }
+      return Sequences.simple(
+          Lists.transform(
+              Lists.newArrayList(segments),
+              (segment) -> new SegmentAnalysis(
+                  segment.toString(),
+                  ImmutableList.of(segment.getInterval()),
+                  columnToAnalysisMap,
+                  40,
+                  40,
+                  null,
+                  null,
+                  null,
+                  false
+              )
+          )
+      );
+    }
+  }
+
+  @State(Scope.Thread)
+  public static class MyState
+  {
+    SegmentId segmentId;
+
+    @Setup(Level.Iteration)
+    public void setup()
+    {
+      segmentId = SegmentId.of("dummy", Intervals.ETERNITY, "1", new LinearShardSpec(0));
+    }
+
+    @TearDown(Level.Iteration)
+    public void teardown()
+    {
+      segmentId = null;
+    }
+  }
+
+  @Setup
+  public void setup()
+  {
+
+    druidSchema = new DruidSchemaForBenchmark(
+        EasyMock.mock(QueryLifecycleFactory.class),
+        EasyMock.mock(TimelineServerView.class),
+        null,
+        null,
+        EasyMock.mock(PlannerConfig.class),
+        null,
+        null
+    );
+    serverMetadata = new DruidServerMetadata(
+        "dummy",
+        "dummy",
+        "dummy",
+        42,
+        ServerType.HISTORICAL,
+        "tier-0",
+        0
+    );
+
+    dataSegment = DataSegment.builder()
+                             .dataSource("dummy")
+                             .shardSpec(new LinearShardSpec(0))
+                             .dimensions(ImmutableList.of("col1", "col2", "col3", "col4"))
+                             .version("1")
+                             .interval(Intervals.ETERNITY)
+                             .size(0)
+                             .build();
+  }
+
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.MICROSECONDS)
+  public void addSegments(MyState state, Blackhole blackhole) throws IOException
+  {
+    druidSchema.addSegment(serverMetadata, dataSegment);

Review comment:
       Updated the benchmark to call updates on a large set of segments at once, and reduced the number of iterations that it does this over. 




-- 
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] LakshSingla commented on pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on pull request #12001:
URL: https://github.com/apache/druid/pull/12001#issuecomment-986780710


   I have updated the latest benchmark to call refreshSegment repeatedly on an already present segment id. Also using JMH's Blackhole and State objects to consume and provide the parameters to the `refreshSegment` call. Following are the results:
   
   With interning:
   ```
   Benchmark                                   (iterations)  (numColumns)  Mode  Cnt        Score        Error  Units
   DruidSchemaAddSegmentBenchmark.addSegments         10000          1000  avgt   15  2170791.575 ± 364899.773  us/op
   ```
   
   Without interning:
   ```
   Benchmark                                   (iterations)  (numColumns)  Mode  Cnt        Score        Error  Units
   DruidSchemaAddSegmentBenchmark.addSegments         10000          1000  avgt   15  1714452.523 ± 126620.724  us/op
   ```
   
   The rows has 1000 columns, and the call to refresh segments is made 10000 times.


-- 
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] abhishekagarwal87 merged pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 merged pull request #12001:
URL: https://github.com/apache/druid/pull/12001


   


-- 
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] LakshSingla commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaAddSegmentBenchmark.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 10)
+@Measurement(iterations = 25)
+public class DruidSchemaAddSegmentBenchmark
+{
+  @Param({"1000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+
+    @Override
+    public void addSegment(final DruidServerMetadata server, final DataSegment segment)

Review comment:
       To provide the initial segment, I have added this method. It won't be called repeatedly in the benchmark however.




-- 
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] LakshSingla edited a comment on pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

Posted by GitBox <gi...@apache.org>.
LakshSingla edited a comment on pull request #12001:
URL: https://github.com/apache/druid/pull/12001#issuecomment-986780710


   I have updated the latest benchmark to call refreshSegment repeatedly on an already present segment id. Till I don't use the Blackhole and State objects, the result of the benchmark is coming out to be similar with both interned and non interned usage. Therefore using JMH's Blackhole and State objects to consume and provide the parameters to the `refreshSegment` call. Following are the results:
   
   With interning:
   ```
   Benchmark                                   (iterations)  (numColumns)  Mode  Cnt        Score        Error  Units
   DruidSchemaAddSegmentBenchmark.addSegments         10000          1000  avgt   15  2170791.575 ± 364899.773  us/op
   ```
   
   Without interning:
   ```
   Benchmark                                   (iterations)  (numColumns)  Mode  Cnt        Score        Error  Units
   DruidSchemaAddSegmentBenchmark.addSegments         10000          1000  avgt   15  1714452.523 ± 126620.724  us/op
   ```
   
   The rows has 1000 columns, and the call to refresh segments is made 10000 times.
   
   


-- 
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] LakshSingla commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 3)
+@Measurement(iterations = 15)
+public class DruidSchemaInternRowSignatureBenchmark
+{
+
+  @Param({"10000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+    @Override
+    public Set<SegmentId> refreshSegments(final Set<SegmentId> segments) throws IOException

Review comment:
       Overriding since I am changing the access type to public here so that it can be called explicitly.




-- 
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] abhishekagarwal87 commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 3)
+@Measurement(iterations = 15)
+public class DruidSchemaInternRowSignatureBenchmark
+{
+
+  @Param({"10000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+    @Override
+    public Set<SegmentId> refreshSegments(final Set<SegmentId> segments) throws IOException

Review comment:
       do we need to override these methods since we are just delegating the operation to super class

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -791,7 +795,7 @@ DruidTable buildDruidTable(final String dataSource)
     } else {
       tableDataSource = new TableDataSource(dataSource);
     }
-    return new DruidTable(tableDataSource, builder.build(), null, isJoinable, isBroadcast);
+    return new DruidTable(tableDataSource, ROW_SIGNATURE_INTERNER.intern(builder.build()), null, isJoinable, isBroadcast);

Review comment:
       I will suggest reverting it since it is simpler that way. 

##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 3)
+@Measurement(iterations = 15)
+public class DruidSchemaInternRowSignatureBenchmark
+{
+
+  @Param({"10000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+    @Override
+    public Set<SegmentId> refreshSegments(final Set<SegmentId> segments) throws IOException
+    {
+      return super.refreshSegments(segments);
+    }
+
+
+    @Override
+    public void addSegment(final DruidServerMetadata server, final DataSegment segment)
+    {
+      super.addSegment(server, segment);
+    }
+
+    @Override
+    protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(Iterable<SegmentId> segments)
+    {
+      final int numColumns = 1000;
+      Map<String, ColumnAnalysis> columnToAnalysisMap = new HashMap<>();
+      for (int i = 0; i < numColumns; ++i) {
+        columnToAnalysisMap.put(
+            "col" + i,
+            new ColumnAnalysis(
+                ColumnType.STRING,
+                null,
+                false,
+                false,
+                40,
+                null,
+                null,
+                null,
+                null
+            )
+        );
+      }
+      return Sequences.simple(
+          Lists.transform(
+              Lists.newArrayList(segments),
+              (segment) -> new SegmentAnalysis(
+                  segment.toString(),
+                  ImmutableList.of(segment.getInterval()),
+                  columnToAnalysisMap,
+                  40,
+                  40,
+                  null,
+                  null,
+                  null,
+                  false
+              )
+          )
+      );
+    }
+  }
+
+  @State(Scope.Thread)
+  public static class MyState
+  {
+    SegmentId segmentId;
+
+    @Setup(Level.Iteration)
+    public void setup()
+    {
+      segmentId = SegmentId.of("dummy", Intervals.ETERNITY, "1", new LinearShardSpec(0));
+    }
+
+    @TearDown(Level.Iteration)
+    public void teardown()
+    {
+      segmentId = null;
+    }
+  }
+
+  @Setup
+  public void setup()
+  {
+
+    druidSchema = new DruidSchemaForBenchmark(
+        EasyMock.mock(QueryLifecycleFactory.class),
+        EasyMock.mock(TimelineServerView.class),
+        null,
+        null,
+        EasyMock.mock(PlannerConfig.class),
+        null,
+        null
+    );
+    serverMetadata = new DruidServerMetadata(
+        "dummy",
+        "dummy",
+        "dummy",
+        42,
+        ServerType.HISTORICAL,
+        "tier-0",
+        0
+    );
+
+    dataSegment = DataSegment.builder()
+                             .dataSource("dummy")
+                             .shardSpec(new LinearShardSpec(0))
+                             .dimensions(ImmutableList.of("col1", "col2", "col3", "col4"))
+                             .version("1")
+                             .interval(Intervals.ETERNITY)
+                             .size(0)
+                             .build();
+  }
+
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.MICROSECONDS)
+  public void addSegments(MyState state, Blackhole blackhole) throws IOException
+  {
+    druidSchema.addSegment(serverMetadata, dataSegment);

Review comment:
       I feel we should rather pass a list of like 10K segments than calling the same method with single segment 10K times. Former is a more closer to how it will be called in the real-world. 




-- 
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] LakshSingla commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaAddSegmentBenchmark.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 10)
+@Measurement(iterations = 25)
+public class DruidSchemaAddSegmentBenchmark

Review comment:
       Thanks for the review. Updated the PR and added some comments.




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

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] abhishekagarwal87 closed pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 closed pull request #12001:
URL: https://github.com/apache/druid/pull/12001


   


-- 
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] abhishekagarwal87 commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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



##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaAddSegmentBenchmark.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 10)
+@Measurement(iterations = 25)
+public class DruidSchemaAddSegmentBenchmark
+{
+  @Param({"1000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+
+    @Override
+    public void addSegment(final DruidServerMetadata server, final DataSegment segment)

Review comment:
       is this method needed? 

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -877,7 +881,7 @@ int getTotalSegments()
 
   private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis)
   {
-    final RowSignature.Builder rowSignatureBuilder = RowSignature.builder();
+    final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); // Intern here

Review comment:
       what do we need to intern here? 

##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaAddSegmentBenchmark.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 10)
+@Measurement(iterations = 25)
+public class DruidSchemaAddSegmentBenchmark

Review comment:
       We should be benchmarking refreshSegments method since that is where we run segment metadata queries and build row signature. 

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -691,7 +695,7 @@ private long recomputeIsRealtime(ImmutableSet<DruidServerMetadata> servers)
         if (segmentId == null) {
           log.warn("Got analysis for segment[%s] we didn't ask for, ignoring.", analysis.getId());
         } else {
-          final RowSignature rowSignature = analysisToRowSignature(analysis);
+          final RowSignature rowSignature = analysisToRowSignature(analysis); // Intern here

Review comment:
       ```suggestion
             final RowSignature rowSignature = analysisToRowSignature(analysis);
   ```

##########
File path: benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaAddSegmentBenchmark.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.benchmark;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.BrokerInternalQueryConfig;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
+import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.SegmentManager;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.easymock.EasyMock;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 10)
+@Measurement(iterations = 25)
+public class DruidSchemaAddSegmentBenchmark
+{
+  @Param({"1000"})
+  private int iterations;
+
+  private DruidSchemaForBenchmark druidSchema;
+  private DruidServerMetadata serverMetadata;
+  private DataSegment dataSegment;
+
+  private static class DruidSchemaForBenchmark extends DruidSchema
+  {
+    public DruidSchemaForBenchmark(
+        final QueryLifecycleFactory queryLifecycleFactory,
+        final TimelineServerView serverView,
+        final SegmentManager segmentManager,
+        final JoinableFactory joinableFactory,
+        final PlannerConfig config,
+        final Escalator escalator,
+        final BrokerInternalQueryConfig brokerInternalQueryConfig
+    )
+    {
+      super(
+          queryLifecycleFactory,
+          serverView,
+          segmentManager,
+          joinableFactory,
+          config,
+          escalator,
+          brokerInternalQueryConfig
+      );
+    }
+
+
+    @Override
+    public void addSegment(final DruidServerMetadata server, final DataSegment segment)
+    {
+      super.addSegment(server, segment);
+    }
+
+    @Override
+    protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(Iterable<SegmentId> segments)
+    {
+      return Sequences.simple(
+          Lists.transform(
+              Lists.newArrayList(segments),
+              (segment) -> new SegmentAnalysis(
+                  segment.toString(),
+                  ImmutableList.of(segment.getInterval()),
+                  ImmutableMap.of(
+                      "col1",
+                      new ColumnAnalysis(
+                          ColumnType.STRING,
+                          null,
+                          false,
+                          false,
+                          40,
+                          null,
+                          null,
+                          null,
+                          null
+                      ),
+                      "col2",
+                      new ColumnAnalysis(ColumnType.LONG, null, false, false, 40, null, null, null, null),
+                      "col2",

Review comment:
       this is the same key that has been used before

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -893,7 +897,8 @@ private static RowSignature analysisToRowSignature(final SegmentAnalysis analysi
 
       rowSignatureBuilder.add(entry.getKey(), valueType);
     }
-    return rowSignatureBuilder.build();
+    return ROW_SIGNATURE_INTERNER.intern(rowSignatureBuilder.build());
+    // return rowSignatureBuilder.build();

Review comment:
       this should be removed. 

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java
##########
@@ -791,7 +795,7 @@ DruidTable buildDruidTable(final String dataSource)
     } else {
       tableDataSource = new TableDataSource(dataSource);
     }
-    return new DruidTable(tableDataSource, builder.build(), null, isJoinable, isBroadcast);
+    return new DruidTable(tableDataSource, ROW_SIGNATURE_INTERNER.intern(builder.build()), null, isJoinable, isBroadcast);

Review comment:
       we are unlikely to see much advantage of interning this signature since it is stored one per table. 




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

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