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/06 17:53: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_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