Skip to content

Make realtimes available for loading segments#4148

Merged
leventov merged 11 commits intoapache:masterfrom
jihoonson:zk-coordinator-for-realtimes
May 18, 2017
Merged

Make realtimes available for loading segments#4148
leventov merged 11 commits intoapache:masterfrom
jihoonson:zk-coordinator-for-realtimes

Conversation

@jihoonson
Copy link
Copy Markdown
Contributor

@jihoonson jihoonson commented Apr 5, 2017

Part of #4032.
In this patch, I added SegmentManager which is separated from ServerManager and is responsible for loading and dropping segments for a node. This SegmentManager is added to both historicals and realtimes.

#4077 introduces BroadcastRule for join processing. I'll extend BroadcastRules to be applied for realtimes after it is merged.


This change is Reviewable

@fjy fjy added the Feature label Apr 5, 2017
@fjy fjy added this to the 0.10.1 milestone Apr 5, 2017
{
return "ImmutableDruidDataSource{"
+ "name='" + name
+ "', segments='" + segmentsHolder
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Usual Druid's toString pattern doesn't include '

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Hmm, actually I followed some other toString() implementaions like DataSegment or SegmentDescriptor. It seems we need to make a standard for toString() first.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

+ "name='" + name
+ "', segments='" + segmentsHolder
+ "', properties='" + properties
+ "'}";
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Doesn't include partitionNames on purpose?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yes, partitionNames usually includes a lot of partitions.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Please leave a comment

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added a comment.

public String toString()
{
return "ImmutableDruidServer{"
+ "meta='" + metadata
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Same

+ "meta='" + metadata
+ "', size='" + currSize
+ "', sources='" + dataSources
+ "'}";
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Doesn't include segments on purpose?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yes, a server usually holds a lot of segments.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Please leave a comment

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added a comment.

*/
private final Map<String, Map<Integer, FireChief>> chiefs;

private final SegmentManager segmentManager;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This field is created but not used

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ah, I added this field for the future use, but it would be fine to add later. I removed for now.


private final Object lock = new Object();
private final SegmentLoader segmentLoader;
private final Map<String, VersionedIntervalTimeline<String, ReferenceCountingSegment>> dataSources = new HashMap<>();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

There could be one ConcurrentHashMap<String, DataSourceState>, where DataSourceState includes VersionedIntervalTimeline<String, ReferenceCountingSegment>, size and count. Then all synchronization could be delegated to ConcurrentHashMap methods, no explicit synchronized and locks are needed in SegmentManager. Also ConcurrentHashMap's concurrency is better than synchronization on a single object.

Copy link
Copy Markdown
Contributor Author

@jihoonson jihoonson May 16, 2017

Choose a reason for hiding this comment

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

SegmentManager is simply separated from ServerManager. Your comments around SegmentManager and ServerManager look good, but I think it is not a part of this pr. Maybe better to raise a new issue after this pr.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Ok but would be nice if you could do this as part of this PR, it's 15 min of work, another issue and a separate PR will eat more of everybody's attention

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

For the issues which are less related to the original issues, I think it would be fine to fix them in a single PR if the changes are truly little and intuitive, but otherwise, it's better to separate to several PRs even though some PRs will be quite small. This is because

  1. Authors and reviewers can focus on the original issues of the PR. This will increase review speed.
  2. As you know, most changes require to add tests even though they are simple. This can be a burden for authors which makes the development slow down.
  3. I think getting other people's attention will be good because they can review PRs from other points of view.

Besides, even in the world you mentioned, a lock is needed anyway for synchronization for accessing ConcurrentHashMap and accessing/mutating DataSourceState taken from ConcurrentHashMap (please see loadSegment()). For example,

private final ConcurrentHashMap<String, DataSourceState> dataSources;
...
public boolean loadSegment(final DataSegment segment) throws SegmentLoadingException
{
  synchronized (lock) {
    final DataSourceState dataSourceState = dataSources.get(dataSource);
    final PartitionHolder<ReferenceCountingSegment> entry = dataSourceState.findEntry(segment.getInterval(), segment.getVersion);
    ...
    dataSourceState.add(
        segment.getInterval(),
        segment.getVersion(),
        segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter))
    );
    ...
  }
}

For this issue, I prefer to investigate first what the exact requirements are.

Copy link
Copy Markdown
Member

@leventov leventov May 16, 2017

Choose a reason for hiding this comment

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

Ok for not including refactoring into this PR.

Synchronization that you mentioned still needed could be managed by ConcurrentHashMap:

dataSources.compute(dataSource, dataSourceState -> {
  if (dataSourceState == null) ...
  else {
    dataSourceState.add(...);
    return dataSourceState;
  }
});

ConcurrentHashMap guarantees that executions of lambdas provided to compute(), computeIfAbsent(), merge() etc. are linearizable. Internally it is implemented via the same intrinsic locks that you use explicitly, but striped over the ConcurrentHashMap entries.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ah, right. That way should work.
Thanks for your understanding. I'll raise a new PR after this PR.

String dataSourceName = getDataSourceName(dataSource);

final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = dataSources.get(dataSourceName);
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = segmentManager.getDataSources()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

getDataSources() creates a copy, please add a method to SegmentManager to extract VersionedIntervalTimeline by dataSource.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Please address

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Thanks. I added a getTimeline() method.

final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = dataSources.get(
dataSourceName
);
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = segmentManager.getDataSources()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Same

*
* @see io.druid.server.coordinator.rules.LoadRule
*/
boolean segmentReplicatable()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Maybe isSegmentReplicationTarget()?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Changed.

*
* @return true if it is available for broadcast.
*/
boolean segmentBroadcastable()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Maybe isSegmentBroadcastTarget()?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Changed.

@gianm gianm removed this from the 0.10.1 milestone May 16, 2017
@leventov
Copy link
Copy Markdown
Member

@jihoonson please resolve conflicts. Going to merge this PR tomorrow unless somebody else wants to review.

@jihoonson
Copy link
Copy Markdown
Contributor Author

@leventov thanks. Resolved conflicts.

@leventov leventov merged commit 5c0a7ad into apache:master May 18, 2017
@gianm gianm added this to the 0.10.1 milestone May 18, 2017
return cluster.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
return historicals.values().stream()
.flatMap(Collection::stream)
.collect(() -> realtimes, Set::add, Set::addAll);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Why getAllServers() adds elements to realtimes collection?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ah it's a bug. I raised a pr #4500.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Thanks for catching it!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants