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/11/30 13:13:53 UTC

[GitHub] [druid] abhishekagarwal87 commented on a change in pull request #12001: Intern RowSignature in DruidSchema to reduce its memory footprint

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