cryptoe commented on code in PR #14985:
URL: https://github.com/apache/druid/pull/14985#discussion_r1360499361
##########
.github/workflows/revised-its.yml:
##########
@@ -50,7 +50,7 @@ jobs:
matrix:
#jdk: [8, 11, 17]
jdk: [8]
- it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex,
MultiStageQueryWithMM, InputSource, InputFormat]
+ it: [MultiStageQuery, BatchIndex]
#indexer: [indexer, middleManager]
Review Comment:
lets revert this change before commit. I guess these are accidental.
##########
benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java:
##########
@@ -71,26 +68,21 @@ public class DruidSchemaInternRowSignatureBenchmark
{
private SegmentMetadataCacheForBenchmark cache;
- private static class SegmentMetadataCacheForBenchmark extends
SegmentMetadataCache
+ private static class SegmentMetadataCacheForBenchmark extends
CoordinatorSegmentMetadataCache
Review Comment:
Lets change the original one to the brokerSegmentMetadataCache and add a new
one for CoordinatorSegmentMetadataCache and lets compare the results.
##########
docs/operations/metrics.md:
##########
@@ -358,6 +361,9 @@ These metrics are for the Druid Coordinator and are reset
each time the Coordina
|`serverview/init/time`|Time taken to initialize the coordinator server
view.||Depends on the number of segments.|
|`serverview/sync/healthy`|Sync status of the Coordinator with a
segment-loading server such as a Historical or Peon. Emitted only when
[HTTP-based server view](../configuration/index.md#segment-management) is
enabled. You can use this metric in conjunction with
`serverview/sync/unstableTime` to debug slow startup of the
Coordinator.|`server`, `tier`|1 for fully synced servers, 0 otherwise|
|`serverview/sync/unstableTime`|Time in milliseconds for which the Coordinator
has been failing to sync with a segment-loading server. Emitted only when
[HTTP-based server view](../configuration/index.md#segment-management) is
enabled.|`server`, `tier`|Not emitted for synced servers.|
+|`metadatacache/init/time`|Time taken to initialize the coordinator segment
metadata cache.|`dataSource`|Depends on the number of segments.|
+|`metadatacache/refresh/count`|Number of segments to refresh in coordinator
segment metadata cache.|`dataSource`|
+|`metadatacache/refresh/time`|Time taken to refresh segments in coordinator
segment metadata cache.|`dataSource`|
Review Comment:
Please also add the user facing docs for this feature.
##########
server/src/main/java/org/apache/druid/client/CoordinatorServerView.java:
##########
@@ -50,28 +57,33 @@ public class CoordinatorServerView implements InventoryView
private static final Logger log = new Logger(CoordinatorServerView.class);
private final Object lock = new Object();
-
private final Map<SegmentId, SegmentLoadInfo> segmentLoadInfos;
private final Map<String, VersionedIntervalTimeline<String,
SegmentLoadInfo>> timelines;
-
+ private final ConcurrentMap<String, QueryRunner> serverQueryRunners;
Review Comment:
Can you please add java docs as to why this is a concurrent data structure.
##########
server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java:
##########
@@ -59,6 +61,15 @@ public ImmutableSegmentLoadInfo toImmutableSegmentLoadInfo()
return new ImmutableSegmentLoadInfo(segment, servers);
}
+ /**
+ * Randomly return one server from the sets of {@code servers}
+ */
+ public DruidServerMetadata pickOne()
+ {
+ synchronized (this) {
Review Comment:
Why is this call made thread safe ?
##########
server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java:
##########
@@ -59,6 +61,15 @@ public ImmutableSegmentLoadInfo toImmutableSegmentLoadInfo()
return new ImmutableSegmentLoadInfo(segment, servers);
}
+ /**
+ * Randomly return one server from the sets of {@code servers}
+ */
+ public DruidServerMetadata pickOne()
+ {
+ synchronized (this) {
Review Comment:
I guess since this object is used in `DataSourceResouce` we need to make it
threadSafe.
##########
docs/operations/metrics.md:
##########
@@ -358,6 +361,9 @@ These metrics are for the Druid Coordinator and are reset
each time the Coordina
|`serverview/init/time`|Time taken to initialize the coordinator server
view.||Depends on the number of segments.|
|`serverview/sync/healthy`|Sync status of the Coordinator with a
segment-loading server such as a Historical or Peon. Emitted only when
[HTTP-based server view](../configuration/index.md#segment-management) is
enabled. You can use this metric in conjunction with
`serverview/sync/unstableTime` to debug slow startup of the
Coordinator.|`server`, `tier`|1 for fully synced servers, 0 otherwise|
|`serverview/sync/unstableTime`|Time in milliseconds for which the Coordinator
has been failing to sync with a segment-loading server. Emitted only when
[HTTP-based server view](../configuration/index.md#segment-management) is
enabled.|`server`, `tier`|Not emitted for synced servers.|
+|`metadatacache/init/time`|Time taken to initialize the coordinator segment
metadata cache.|`dataSource`|Depends on the number of segments.|
Review Comment:
I guess all of this should not contain data source as a dimension.
##########
server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.segment.metadata;
+
+import com.google.common.collect.Sets;
+import com.google.inject.Inject;
+import org.apache.druid.client.CoordinatorServerView;
+import org.apache.druid.client.InternalQueryConfig;
+import org.apache.druid.client.ServerView;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.security.Escalator;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+
+import java.io.IOException;
+import java.util.Set;
+
+/**
+ * Coordinator-side cache of segment metadata that combines segments to build
+ * datasources. The cache provides metadata about a datasource, see {@link
DataSourceInformation}.
+ */
+@ManageLifecycle
+public class CoordinatorSegmentMetadataCache extends
AbstractSegmentMetadataCache<DataSourceInformation>
+{
+ private static final EmittingLogger log = new
EmittingLogger(CoordinatorSegmentMetadataCache.class);
+
+ @Inject
+ public CoordinatorSegmentMetadataCache(
+ QueryLifecycleFactory queryLifecycleFactory,
+ CoordinatorServerView serverView,
+ SegmentMetadataCacheConfig config,
+ Escalator escalator,
+ InternalQueryConfig internalQueryConfig,
+ ServiceEmitter emitter
+ )
+ {
+ super(queryLifecycleFactory, config, escalator, internalQueryConfig,
emitter);
+
+ initServerViewTimelineCallback(serverView);
+ }
+
+ private void initServerViewTimelineCallback(final CoordinatorServerView
serverView)
+ {
+ serverView.registerTimelineCallback(
+ callbackExec,
+ new TimelineServerView.TimelineCallback()
+ {
+ @Override
+ public ServerView.CallbackAction timelineInitialized()
+ {
+ synchronized (lock) {
+ isServerViewInitialized = true;
+ lock.notifyAll();
+ }
+
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction segmentAdded(final
DruidServerMetadata server, final DataSegment segment)
+ {
+ addSegment(server, segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction segmentRemoved(final DataSegment
segment)
+ {
+ removeSegment(segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction serverSegmentRemoved(
+ final DruidServerMetadata server,
+ final DataSegment segment
+ )
+ {
+ removeServerSegment(server, segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+ }
+ );
+ }
+
+ /**
+ * Fires SegmentMetadataQuery to fetch schema information for each segment
in the refresh list.
+ * The schema information for individual segments is combined to construct a
table schema, which is then cached.
+ *
+ * @param segmentsToRefresh segments for which the schema might have
changed
+ * @param dataSourcesToRebuild datasources for which the schema might have
changed
+ * @throws IOException when querying segment from data nodes and
tasks
+ */
+ @Override
+ public void refresh(final Set<SegmentId> segmentsToRefresh, final
Set<String> dataSourcesToRebuild) throws IOException
+ {
+ // Refresh the segments.
+ final Set<SegmentId> refreshed = refreshSegments(segmentsToRefresh);
+
+ synchronized (lock) {
+ // Add missing segments back to the refresh list.
+ segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh,
refreshed));
+
+ // Compute the list of datasources to rebuild tables for.
+ dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild);
+ refreshed.forEach(segment ->
dataSourcesToRebuild.add(segment.getDataSource()));
+ dataSourcesNeedingRebuild.clear();
+ }
+
+ // Rebuild the datasources.
+ for (String dataSource : dataSourcesToRebuild) {
+ final RowSignature rowSignature = buildDruidTable(dataSource);
+ if (rowSignature == null) {
+ log.info("datasource [%s] no longer exists, all metadata removed.",
dataSource);
Review Comment:
Lets change this info comment to rowSignature is empty for this data source
hence removing the data source.
##########
docs/operations/metrics.md:
##########
@@ -72,6 +72,9 @@ Most metric values reset each emission period, as specified
in `druid.monitoring
|`metadatacache/init/time`|Time taken to initialize the broker segment
metadata cache. Useful to detect if brokers are taking too long to
start||Depends on the number of segments.|
|`metadatacache/refresh/count`|Number of segments to refresh in broker segment
metadata cache.|`dataSource`|
|`metadatacache/refresh/time`|Time taken to refresh segments in broker segment
metadata cache.|`dataSource`|
+|`metadatacache/schemaPoll/count`|Number of coordinator polls to fetch
datasource schema.|`dataSource`|
Review Comment:
Why is this at a data source level .
##########
server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java:
##########
@@ -808,7 +792,7 @@ private Set<SegmentId> refreshSegmentsForDataSource(final
String dataSource, fin
@VisibleForTesting
@Nullable
- DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String
dataSource)
+ public RowSignature buildDruidTable(final String dataSource)
Review Comment:
lets rename this to buildRowSignatureForDataSource.
Please add javadocs.
##########
server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java:
##########
@@ -96,40 +87,37 @@
import java.util.stream.StreamSupport;
/**
- * Broker-side cache of segment metadata which combines segments to identify
- * datasources which become "tables" in Calcite. This cache provides the
"physical"
- * metadata about a datasource which is blended with catalog "logical" metadata
- * to provide the final user-view of each datasource.
+ * An abstract class that listens for segment change events and caches segment
metadata. It periodically refreshes
+ * the segments, by fetching their metadata which includes schema information
from sources like
+ * data nodes, db (the logic is specificed in the child class) and builds
table schema.
+ *
+ * <p>This class is generic and is parameterized by a type {@code T} that
extends {@link DataSourceInformation}.</p>
+ *
+ * <p>This class has an abstract method {@link #refresh(Set, Set)} which the
child class must override
+ * with the logic to build and cache table schema.</p>
+ *
+ * @param <T> The type of information associated with the data source, which
must extend {@link DataSourceInformation}.
Review Comment:
```suggestion
* An abstract class that listens for segment change events and caches
segment metadata in memory. It periodically refreshes
* the segments, by fetching their metadata from sources like
* data nodes, metadata db and builds table schema.
*
*
* <p>This class has an abstract method {@link #refresh(Set, Set)} which the
child class must override
* with the logic to build and cache table schema.</p>
*
* @param <T> The type of information associated with the data source which
must extend {@link DataSourceInformation}.
```
##########
processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java:
##########
@@ -52,26 +54,40 @@ public class SegmentStatusInCluster implements
Comparable<SegmentStatusInCluster
*/
@JsonUnwrapped
private final DataSegment dataSegment;
+ /**
+ * Number of rows in the segment.
+ */
+ private final Long numRows;
+ /**
+ * If the segment is realtime.
+ */
+ private final boolean realtime;
Review Comment:
@adarshsanjeev Since we added replication factor to this class for the
`cold` segments, can we instead not pass a tristate enum with values
`REALTIME,HISTORICAL,COLD` and remove this boolean. Names are not final though
?
##########
server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java:
##########
@@ -808,7 +792,7 @@ private Set<SegmentId> refreshSegmentsForDataSource(final
String dataSource, fin
@VisibleForTesting
@Nullable
- DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String
dataSource)
+ public RowSignature buildDruidTable(final String dataSource)
Review Comment:
We should also think of getting the `aggFactories` here since it will be
required for auto compaction improvements where in we can launch the job
without downloading the segments if the index spec is not provided.
##########
server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java:
##########
@@ -59,6 +61,15 @@ public ImmutableSegmentLoadInfo toImmutableSegmentLoadInfo()
return new ImmutableSegmentLoadInfo(segment, servers);
}
+ /**
+ * Randomly return one server from the sets of {@code servers}
+ */
+ public DruidServerMetadata pickOne()
+ {
+ synchronized (this) {
Review Comment:
We should take a lock in the adding/remove as well.
##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.base.Stopwatch;
+import com.google.common.collect.Sets;
+import com.google.inject.Inject;
+import org.apache.druid.client.InternalQueryConfig;
+import org.apache.druid.client.ServerView;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.security.Escalator;
+import
org.apache.druid.sql.calcite.table.DatasourceTable.PhysicalDatasourceMetadata;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Broker-side cache of segment metadata that combines segments to build
+ * datasources which become "tables" in Calcite. This cache provides the
"physical"
+ * metadata about a dataSource which is blended with catalog "logical" metadata
+ * to provide the final user-view of each dataSource.
+ * <p>
+ * This class extends {@link AbstractSegmentMetadataCache} and introduces
following changes,
+ * <ul>
+ * <li>The refresh mechanism includes polling the coordinator for datasource
schema,
+ * and falling back to running {@link
org.apache.druid.query.metadata.metadata.SegmentMetadataQuery}.</li>
+ * <li>It builds and caches {@link PhysicalDatasourceMetadata} object for
the table schema</li>
+ * </ul>
+ */
+@ManageLifecycle
+public class BrokerSegmentMetadataCache extends
AbstractSegmentMetadataCache<PhysicalDatasourceMetadata>
+{
+ private static final EmittingLogger log = new
EmittingLogger(BrokerSegmentMetadataCache.class);
+
+ private final PhysicalDatasourceMetadataFactory dataSourceMetadataFactory;
+ private final CoordinatorClient coordinatorClient;
+
+ private final BrokerSegmentMetadataCacheConfig config;
+
+ @Inject
+ public BrokerSegmentMetadataCache(
+ final QueryLifecycleFactory queryLifecycleFactory,
+ final TimelineServerView serverView,
+ final BrokerSegmentMetadataCacheConfig config,
+ final Escalator escalator,
+ final InternalQueryConfig internalQueryConfig,
+ final ServiceEmitter emitter,
+ final PhysicalDatasourceMetadataFactory dataSourceMetadataFactory,
+ final CoordinatorClient coordinatorClient
+ )
+ {
+ super(
+ queryLifecycleFactory,
+ config,
+ escalator,
+ internalQueryConfig,
+ emitter
+ );
+ this.dataSourceMetadataFactory = dataSourceMetadataFactory;
+ this.coordinatorClient = coordinatorClient;
+ this.config = config;
+ initServerViewTimelineCallback(serverView);
+ }
+
+ private void initServerViewTimelineCallback(final TimelineServerView
serverView)
+ {
+ serverView.registerTimelineCallback(
+ callbackExec,
+ new TimelineServerView.TimelineCallback()
+ {
+ @Override
+ public ServerView.CallbackAction timelineInitialized()
+ {
+ synchronized (lock) {
+ isServerViewInitialized = true;
+ lock.notifyAll();
+ }
+
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction segmentAdded(final
DruidServerMetadata server, final DataSegment segment)
+ {
+ addSegment(server, segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction segmentRemoved(final DataSegment
segment)
+ {
+ removeSegment(segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction serverSegmentRemoved(
+ final DruidServerMetadata server,
+ final DataSegment segment
+ )
+ {
+ removeServerSegment(server, segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+ }
+ );
+ }
+
+ /**
+ * Refreshes the set of segments in two steps:
+ * <ul>
+ * <li>Polls the coordinator for the datasource schema.</li>
+ * <li>Refreshes the remaining set of segments by executing a
SegmentMetadataQuery and
+ * builds datasource schema by combining segment schema.</li>
+ * </ul>
+ *
+ * @param segmentsToRefresh segments for which the schema might have
changed
+ * @param dataSourcesToRebuild datasources for which the schema might have
changed
+ * @throws IOException when querying segment schema from data nodes
and tasks
+ */
+ @Override
+ public void refresh(final Set<SegmentId> segmentsToRefresh, final
Set<String> dataSourcesToRebuild) throws IOException
+ {
+ // query all the datasource schema, which includes,
Review Comment:
```suggestion
// query schema for all the data sources which includes
```
##########
server/src/main/java/org/apache/druid/client/CachingClusteredClient.java:
##########
@@ -844,23 +845,23 @@ private byte[] computeQueryCacheKeyWithJoin()
}
}
- private static class TimelineConverter implements
UnaryOperator<TimelineLookup<String, ServerSelector>>
+ public static class TimelineConverter<ObjectType extends
Overshadowable<ObjectType>> implements UnaryOperator<TimelineLookup<String,
ObjectType>>
Review Comment:
Can you please add java docs for this since this is a public class now.
##########
docs/configuration/index.md:
##########
@@ -2002,7 +2003,7 @@ The Druid SQL server is configured through the following
properties on the Broke
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN
queries](../querying/topnquery.md) when a SQL query could be expressed as such.
If false, exact [GroupBy queries](../querying/groupbyquery.md) will be used
instead.|true|
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have
filter conditions on __time column so that all generated native queries will
have user specified intervals. If true, all queries without filter condition on
__time column will fail|false|
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server,
which will affect how time functions and timestamp literals behave. Should be a
time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
-|`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.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.|true|
Review Comment:
Lets move this as part of another PR.
##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.base.Stopwatch;
+import com.google.common.collect.Sets;
+import com.google.inject.Inject;
+import org.apache.druid.client.InternalQueryConfig;
+import org.apache.druid.client.ServerView;
+import org.apache.druid.client.TimelineServerView;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.security.Escalator;
+import
org.apache.druid.sql.calcite.table.DatasourceTable.PhysicalDatasourceMetadata;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Broker-side cache of segment metadata that combines segments to build
+ * datasources which become "tables" in Calcite. This cache provides the
"physical"
+ * metadata about a dataSource which is blended with catalog "logical" metadata
+ * to provide the final user-view of each dataSource.
+ * <p>
+ * This class extends {@link AbstractSegmentMetadataCache} and introduces
following changes,
+ * <ul>
+ * <li>The refresh mechanism includes polling the coordinator for datasource
schema,
+ * and falling back to running {@link
org.apache.druid.query.metadata.metadata.SegmentMetadataQuery}.</li>
+ * <li>It builds and caches {@link PhysicalDatasourceMetadata} object for
the table schema</li>
+ * </ul>
+ */
+@ManageLifecycle
+public class BrokerSegmentMetadataCache extends
AbstractSegmentMetadataCache<PhysicalDatasourceMetadata>
+{
+ private static final EmittingLogger log = new
EmittingLogger(BrokerSegmentMetadataCache.class);
+
+ private final PhysicalDatasourceMetadataFactory dataSourceMetadataFactory;
+ private final CoordinatorClient coordinatorClient;
+
+ private final BrokerSegmentMetadataCacheConfig config;
+
+ @Inject
+ public BrokerSegmentMetadataCache(
+ final QueryLifecycleFactory queryLifecycleFactory,
+ final TimelineServerView serverView,
+ final BrokerSegmentMetadataCacheConfig config,
+ final Escalator escalator,
+ final InternalQueryConfig internalQueryConfig,
+ final ServiceEmitter emitter,
+ final PhysicalDatasourceMetadataFactory dataSourceMetadataFactory,
+ final CoordinatorClient coordinatorClient
+ )
+ {
+ super(
+ queryLifecycleFactory,
+ config,
+ escalator,
+ internalQueryConfig,
+ emitter
+ );
+ this.dataSourceMetadataFactory = dataSourceMetadataFactory;
+ this.coordinatorClient = coordinatorClient;
+ this.config = config;
+ initServerViewTimelineCallback(serverView);
+ }
+
+ private void initServerViewTimelineCallback(final TimelineServerView
serverView)
+ {
+ serverView.registerTimelineCallback(
+ callbackExec,
+ new TimelineServerView.TimelineCallback()
+ {
+ @Override
+ public ServerView.CallbackAction timelineInitialized()
+ {
+ synchronized (lock) {
+ isServerViewInitialized = true;
+ lock.notifyAll();
+ }
+
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction segmentAdded(final
DruidServerMetadata server, final DataSegment segment)
+ {
+ addSegment(server, segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction segmentRemoved(final DataSegment
segment)
+ {
+ removeSegment(segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+
+ @Override
+ public ServerView.CallbackAction serverSegmentRemoved(
+ final DruidServerMetadata server,
+ final DataSegment segment
+ )
+ {
+ removeServerSegment(server, segment);
+ return ServerView.CallbackAction.CONTINUE;
+ }
+ }
+ );
+ }
+
+ /**
+ * Refreshes the set of segments in two steps:
+ * <ul>
+ * <li>Polls the coordinator for the datasource schema.</li>
+ * <li>Refreshes the remaining set of segments by executing a
SegmentMetadataQuery and
+ * builds datasource schema by combining segment schema.</li>
+ * </ul>
+ *
+ * @param segmentsToRefresh segments for which the schema might have
changed
+ * @param dataSourcesToRebuild datasources for which the schema might have
changed
+ * @throws IOException when querying segment schema from data nodes
and tasks
+ */
+ @Override
+ public void refresh(final Set<SegmentId> segmentsToRefresh, final
Set<String> dataSourcesToRebuild) throws IOException
+ {
+ // query all the datasource schema, which includes,
+ // datasources explicitly marked for rebuilding
+ // datasources for the segments to be refreshed
+ // prebuilt datasources
+ final Set<String> dataSourcesToQuery = new HashSet<>(dataSourcesToRebuild);
+
+ segmentsToRefresh.forEach(segment ->
dataSourcesToQuery.add(segment.getDataSource()));
+
+ dataSourcesToQuery.addAll(tables.keySet());
+
+ // Fetch datasource information from the Coordinator
+ Map<String, PhysicalDatasourceMetadata> polledDataSourceMetadata =
queryDataSourceInformation(dataSourcesToQuery);
+
+ // update datasource metadata in the cache
+ polledDataSourceMetadata.forEach(this::updateDSMetadata);
+
+ // Remove segments of the datasource from refresh list for which we
received schema from the Coordinator.
+ segmentsToRefresh.removeIf(segmentId ->
polledDataSourceMetadata.containsKey(segmentId.getDataSource()));
+
+ Set<SegmentId> refreshed = new HashSet<>();
+
+ // Refresh the remaining segments.
+ if (!config.isDisableSegmentMetadataQueries()) {
+ refreshed = refreshSegments(segmentsToRefresh);
+ }
+
+ synchronized (lock) {
+ // Add missing segments back to the refresh list.
+ segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh,
refreshed));
+
+ // Compute the list of datasources to rebuild tables for.
+ dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild);
+ refreshed.forEach(segment ->
dataSourcesToRebuild.add(segment.getDataSource()));
+
+ // Remove those datasource for which we received schema from the
Coordinator.
+ dataSourcesToRebuild.removeAll(polledDataSourceMetadata.keySet());
+ dataSourcesNeedingRebuild.clear();
+ }
+
+ // Rebuild the datasources.
+ for (String dataSource : dataSourcesToRebuild) {
+ final RowSignature rowSignature = buildDruidTable(dataSource);
+ if (rowSignature == null) {
+ log.info("datasource [%s] no longer exists, all metadata removed.",
dataSource);
+ tables.remove(dataSource);
+ return;
+ }
+
+ final PhysicalDatasourceMetadata physicalDatasourceMetadata =
dataSourceMetadataFactory.build(dataSource, rowSignature);
+ updateDSMetadata(dataSource, physicalDatasourceMetadata);
+ }
+ }
+
+ private Map<String, PhysicalDatasourceMetadata>
queryDataSourceInformation(Set<String> dataSourcesToQuery)
+ {
+ final Map<String, PhysicalDatasourceMetadata> polledDataSourceMetadata =
new HashMap<>();
+
+ Stopwatch stopwatch = Stopwatch.createStarted();
+
+ try {
+ emitter.emit(ServiceMetricEvent.builder().setMetric(
+ "metadatacache/schemaPoll/count", 1));
+
FutureUtils.getUnchecked(coordinatorClient.fetchDataSourceInformation(dataSourcesToQuery),
true)
+ .forEach(dataSourceInformation ->
polledDataSourceMetadata.put(
+ dataSourceInformation.getDataSource(),
+ dataSourceMetadataFactory.build(
+ dataSourceInformation.getDataSource(),
+ dataSourceInformation.getRowSignature()
+ )
+ ));
+ }
+ catch (Exception e) {
+ log.warn(e, "Failed to query datasource information from the
Coordinator.");
+ emitter.emit(ServiceMetricEvent.builder().setMetric(
+ "metadatacache/schemaPoll/failed", 1));
+ }
+
+ emitter.emit(ServiceMetricEvent.builder().setMetric(
Review Comment:
poll time should be before we iterate over the results.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]