findingrish commented on code in PR #14985:
URL: https://github.com/apache/druid/pull/14985#discussion_r1372579988


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.segment.metadata.DataSourceInformation;
+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.List;
+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 schema for all datasources, 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);

Review Comment:
   That was the plan initially, but it seemed that having 2 different configs 
to enable the feature would be confusing. 
   On the other hand, the latency of querying the Coordinator is pretty low. 
   



-- 
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]

Reply via email to