surekhasaharan commented on a change in pull request #6901: Introduce published segment cache in broker URL: https://github.com/apache/incubator-druid/pull/6901#discussion_r271479675
########## File path: sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java ########## @@ -0,0 +1,255 @@ +/* + * 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.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import org.apache.druid.client.BrokerSegmentWatcherConfig; +import org.apache.druid.client.DataSegmentInterner; +import org.apache.druid.client.JsonParserIterator; +import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.concurrent.LifecycleLock; +import org.apache.druid.discovery.DruidLeaderClient; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; +import org.apache.druid.sql.calcite.planner.PlannerConfig; +import org.apache.druid.timeline.DataSegment; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class polls the coordinator in background to keep the latest published segments. + * Provides {@link #getPublishedSegments()} for others to get segments in metadata store. + */ +@ManageLifecycle +public class MetadataSegmentView +{ + + private static final EmittingLogger log = new EmittingLogger(MetadataSegmentView.class); + + private final DruidLeaderClient coordinatorDruidLeaderClient; + private final ObjectMapper jsonMapper; + private final BytesAccumulatingResponseHandler responseHandler; + private final BrokerSegmentWatcherConfig segmentWatcherConfig; + + private final boolean isCacheEnabled; + @Nullable + private final ConcurrentMap<DataSegment, DateTime> publishedSegments; + private final ScheduledExecutorService scheduledExec; + private final long pollPeriodInMS; + private final LifecycleLock lifecycleLock = new LifecycleLock(); + private final AtomicBoolean cachePopulated = new AtomicBoolean(false); + + @Inject + public MetadataSegmentView( + final @Coordinator DruidLeaderClient druidLeaderClient, + final ObjectMapper jsonMapper, + final BytesAccumulatingResponseHandler responseHandler, + final BrokerSegmentWatcherConfig segmentWatcherConfig, + final PlannerConfig plannerConfig + ) + { + Preconditions.checkNotNull(plannerConfig, "plannerConfig"); + this.coordinatorDruidLeaderClient = druidLeaderClient; + this.jsonMapper = jsonMapper; + this.responseHandler = responseHandler; + this.segmentWatcherConfig = segmentWatcherConfig; + this.isCacheEnabled = plannerConfig.isMetadataSegmentCacheEnable(); + this.pollPeriodInMS = plannerConfig.getMetadataSegmentPollPeriod(); + this.publishedSegments = isCacheEnabled ? new ConcurrentHashMap<>(1000) : null; + this.scheduledExec = Execs.scheduledSingleThreaded("MetadataSegmentView-Cache--%d"); + } + + @LifecycleStart + public void start() + { + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + try { + if (isCacheEnabled) { + scheduledExec.schedule(new PollTask(), pollPeriodInMS, TimeUnit.MILLISECONDS); + } + lifecycleLock.started(); + log.info("MetadataSegmentView Started."); + } + finally { + lifecycleLock.exitStart(); + } + } + + @LifecycleStop + public void stop() + { + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); + } + log.info("MetadataSegmentView is stopping."); + if (isCacheEnabled) { + scheduledExec.shutdown(); + } + log.info("MetadataSegmentView Stopped."); + } + + private void poll() + { + log.info("polling published segments from coordinator"); + final JsonParserIterator<DataSegment> metadataSegments = getMetadataSegments( + coordinatorDruidLeaderClient, + jsonMapper, + responseHandler, + segmentWatcherConfig.getWatchedDataSources() + ); + + final DateTime timestamp = DateTimes.nowUtc(); + while (metadataSegments.hasNext()) { + final DataSegment interned = DataSegmentInterner.intern(metadataSegments.next()); + // timestamp is used to filter deleted segments + publishedSegments.put(interned, timestamp); + } + // filter the segments from cache whose timestamp is not equal to latest timestamp stored, + // since the presence of a segment with an earlier timestamp indicates that + // "that" segment is not returned by coordinator in latest poll, so it's + // likely deleted and therefore we remove it from publishedSegments + // Since segments are not atomically replaced because it can cause high + // memory footprint due to large number of published segments, so + // we are incrementally removing deleted segments from the map + // This means publishedSegments will be eventually consistent with + // the segments in coordinator + publishedSegments.entrySet().removeIf(e -> e.getValue() != timestamp); + cachePopulated.set(true); + } + + public Iterator<DataSegment> getPublishedSegments() + { + if (isCacheEnabled) { + Preconditions.checkState( + lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS) && cachePopulated.get(), Review comment: yeah, client would get `IllegalStateException` with the error message `hold on, still syncing published segments`, and they'll have to retry. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org For additional commands, e-mail: commits-h...@druid.apache.org