Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
e8b0a23
Add published segment cache in broker
Jan 22, 2019
6ec4911
Change the DataSegment interner so it's not based on DataSEgment's eq…
Jan 27, 2019
8b6d453
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 27, 2019
281600d
Use separate interner for realtime and historical segments
Jan 28, 2019
cdc751e
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 28, 2019
8fed80a
Remove trueEquals as it's not used anymore, change log message
Jan 28, 2019
0d0f89f
PR comments
Jan 29, 2019
8fc84b3
PR comments
Jan 29, 2019
adf133f
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 29, 2019
d46edc5
Fix tests
Jan 29, 2019
92601d0
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 29, 2019
a993482
PR comments
Jan 30, 2019
a4cbcfc
Few more modification to
Jan 30, 2019
e376df3
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 30, 2019
0fbb48c
minor changes
Jan 30, 2019
8df2d96
PR comments
Jan 31, 2019
d5b7d79
PR comments
Jan 31, 2019
2bfa396
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 31, 2019
632d741
Make the segment cache in broker off by default
Feb 1, 2019
bcc6513
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
c41f085
Add doc for new planner config
Feb 1, 2019
07a80b1
Update documentation
Feb 1, 2019
d83eb33
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
a440c0c
PR comments
Feb 1, 2019
b385345
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
981b080
some more changes
Feb 1, 2019
3a94cae
PR comments
Feb 1, 2019
0032a31
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
ad28458
fix test
Feb 1, 2019
33751a4
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
2ddb7a1
remove unintentional change, whether to synchronize on lifecycleLock …
Feb 1, 2019
1b37493
minor changes
Feb 1, 2019
60dbf41
some changes to initialization
Feb 2, 2019
8860053
use pollPeriodInMS
Feb 2, 2019
ca21779
Add boolean cachePopulated to check if first poll succeeds
Feb 2, 2019
3a70f39
Remove poll from start()
Feb 2, 2019
e2a9af7
take the log message out of condition in stop()
Feb 2, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ public void setup()
final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index);
final PlannerFactory plannerFactory = new PlannerFactory(
druidSchema,
Expand Down
2 changes: 2 additions & 0 deletions docs/content/querying/sql.md
Original file line number Diff line number Diff line change
Expand Up @@ -698,6 +698,8 @@ The Druid SQL server is configured through the following properties on the Broke
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|`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 wihout 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.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000|

## SQL Metrics

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ public void setUp() throws Exception

final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new BloomFilterSqlAggregator()),
ImmutableSet.of()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ public void setUp() throws Exception

final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new QuantileSqlAggregator()),
ImmutableSet.of()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* 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.client;

import com.google.common.collect.Interner;
import com.google.common.collect.Interners;
import org.apache.druid.timeline.DataSegment;

/**
* Interns the DataSegment object in order to share the reference for same DataSegment.
* It uses two separate interners for realtime and historical segments to prevent
* overwriting the size of a segment which was served by a historical and later served
* by another realtime server, since realtime server always publishes with size 0.
*/
public class DataSegmentInterner
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would you please add a javadoc about what this is doing and why we need two interners?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added docs

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Interners with weak references shouldn't be used: #7395

{
private static final Interner<DataSegment> REALTIME_INTERNER = Interners.newWeakInterner();
private static final Interner<DataSegment> HISTORICAL_INTERNER = Interners.newWeakInterner();

private DataSegmentInterner()
{
//No instantiation
}

public static DataSegment intern(DataSegment segment)
{
// A segment learns it's size and dimensions when it moves from a relatime to historical server
// for that reason, we are using it's size as the indicator to decide whether to use REALTIME or
// HISTORICAL interner.
return segment.getSize() > 0 ? HISTORICAL_INTERNER.intern(segment) : REALTIME_INTERNER.intern(segment);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's worth to comment why size can be an indicator of a realtime or historical segment.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added a comment

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.druid.client.selector;

import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap;
import org.apache.druid.client.DataSegmentInterner;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment;
Expand Down Expand Up @@ -50,7 +51,7 @@ public ServerSelector(
TierSelectorStrategy strategy
)
{
this.segment = new AtomicReference<>(segment);
this.segment = new AtomicReference<>(DataSegmentInterner.intern(segment));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure segment still needs to be an AtomicReference.. Let's figure it out in #6952.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

okay

this.strategy = strategy;
this.historicalServers = new Int2ObjectRBTreeMap<>(strategy.getComparator());
this.realtimeServers = new Int2ObjectRBTreeMap<>(strategy.getComparator());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;

/**
Expand Down Expand Up @@ -148,14 +149,22 @@ public Response getDatabaseSegmentDataSource(@PathParam("dataSourceName") final
@GET
@Path("/segments")
@Produces(MediaType.APPLICATION_JSON)
public Response getDatabaseSegments(@Context final HttpServletRequest req)
public Response getDatabaseSegments(
@Context final HttpServletRequest req,
@QueryParam("datasources") final Set<String> datasources
)
{
final Collection<ImmutableDruidDataSource> druidDataSources = metadataSegmentManager.getDataSources();
Collection<ImmutableDruidDataSource> druidDataSources = metadataSegmentManager.getDataSources();
if (datasources != null && !datasources.isEmpty()) {
druidDataSources = druidDataSources.stream()
.filter(src -> datasources.contains(src.getName()))
.collect(Collectors.toSet());
Copy link
Member

@leventov leventov Mar 14, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@surekhasaharan we don't want to collect ImmutableDruidDataSource objects, which are already known to be unique here, into a set. It induces expensive O(n) hash code computations (which is also totally unnecessary; ImmutableDruidDataSource.hashCode() should be cheaper. But that's another question.)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks @leventov for catching this, will change to list in a follow up PR.

}
final Stream<DataSegment> metadataSegments = druidDataSources
.stream()
.flatMap(t -> t.getSegments().stream());

Function<DataSegment, Iterable<ResourceAction>> raGenerator = segment -> Collections.singletonList(
final Function<DataSegment, Iterable<ResourceAction>> raGenerator = segment -> Collections.singletonList(
AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));

final Iterable<DataSegment> authorizedSegments =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2213,13 +2213,13 @@ private List<Map<DruidServer, ServerExpectations>> populateTimeline(
expectedResults.get(k).get(j)
);
serverExpectations.get(lastServer).addExpectation(expectation);

EasyMock.expect(mockSegment.getSize()).andReturn(0L).anyTimes();
EasyMock.replay(mockSegment);
ServerSelector selector = new ServerSelector(
expectation.getSegment(),
new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())
);
selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), selector.getSegment());

final ShardSpec shardSpec;
if (numChunks == 1) {
shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0);
Expand All @@ -2234,6 +2234,7 @@ private List<Map<DruidServer, ServerExpectations>> populateTimeline(
}
shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j);
}
EasyMock.reset(mockSegment);
EasyMock.expect(mockSegment.getShardSpec())
.andReturn(shardSpec)
.anyTimes();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,21 @@ public class PlannerConfig
private DateTimeZone sqlTimeZone = DateTimeZone.UTC;

@JsonProperty
private boolean metadataSegmentCacheEnable = false;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would you please add a doc for this configuration?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, added


@JsonProperty
private long metadataSegmentPollPeriod = 60000;

public long getMetadataSegmentPollPeriod()
{
return metadataSegmentPollPeriod;
}

public boolean isMetadataSegmentCacheEnable()
{
return metadataSegmentCacheEnable;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add to equals, toString, hashCode methods

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added

}

private boolean serializeComplexValues = true;

public Period getMetadataRefreshPeriod()
Expand Down Expand Up @@ -159,6 +174,8 @@ public PlannerConfig withOverrides(final Map<String, Object> context)
newConfig.requireTimeCondition = isRequireTimeCondition();
newConfig.sqlTimeZone = getSqlTimeZone();
newConfig.awaitInitializationOnStart = isAwaitInitializationOnStart();
newConfig.metadataSegmentCacheEnable = isMetadataSegmentCacheEnable();
newConfig.metadataSegmentPollPeriod = getMetadataSegmentPollPeriod();
newConfig.serializeComplexValues = shouldSerializeComplexValues();
return newConfig;
}
Expand Down Expand Up @@ -200,6 +217,8 @@ public boolean equals(final Object o)
useFallback == that.useFallback &&
requireTimeCondition == that.requireTimeCondition &&
awaitInitializationOnStart == that.awaitInitializationOnStart &&
metadataSegmentCacheEnable == that.metadataSegmentCacheEnable &&
metadataSegmentPollPeriod == that.metadataSegmentPollPeriod &&
serializeComplexValues == that.serializeComplexValues &&
Objects.equals(metadataRefreshPeriod, that.metadataRefreshPeriod) &&
Objects.equals(sqlTimeZone, that.sqlTimeZone);
Expand All @@ -221,6 +240,8 @@ public int hashCode()
requireTimeCondition,
awaitInitializationOnStart,
sqlTimeZone,
metadataSegmentCacheEnable,
metadataSegmentPollPeriod,
serializeComplexValues
);
}
Expand All @@ -239,6 +260,8 @@ public String toString()
", useFallback=" + useFallback +
", requireTimeCondition=" + requireTimeCondition +
", awaitInitializationOnStart=" + awaitInitializationOnStart +
", metadataSegmentCacheEnable=" + metadataSegmentCacheEnable +
", metadataSegmentPollPeriod=" + metadataSegmentPollPeriod +
", sqlTimeZone=" + sqlTimeZone +
", serializeComplexValues=" + serializeComplexValues +
'}';
Expand Down
Loading