Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -156,14 +156,7 @@ public void inventoryInitialized()
{
log.info("Inventory Initialized");
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentViewInitialized();
}
}
(SegmentCallback input) -> input.segmentViewInitialized()
);
}
}
Expand Down Expand Up @@ -232,15 +225,10 @@ protected void runSegmentCallbacks(
{
for (final Map.Entry<SegmentCallback, Executor> entry : segmentCallbacks.entrySet()) {
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) {
segmentCallbackRemoved(entry.getKey());
segmentCallbacks.remove(entry.getKey());
}
() -> {
if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) {
segmentCallbackRemoved(entry.getKey());
segmentCallbacks.remove(entry.getKey());
}
}
);
Expand All @@ -251,14 +239,9 @@ private void runServerRemovedCallbacks(final DruidServer server)
{
for (final Map.Entry<ServerRemovedCallback, Executor> entry : serverRemovedCallbacks.entrySet()) {
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) {
serverRemovedCallbacks.remove(entry.getKey());
}
() -> {
if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) {
serverRemovedCallbacks.remove(entry.getKey());
}
}
);
Expand All @@ -285,14 +268,7 @@ protected void addSingleInventory(
container.addDataSegment(inventory.getIdentifier(), inventory);

runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentAdded(container.getMetadata(), inventory);
}
}
(input) -> input.segmentAdded(container.getMetadata(), inventory)
);
}

Expand All @@ -314,14 +290,7 @@ protected void removeSingleInventory(final DruidServer container, String invento
container.removeDataSegment(inventoryKey);

runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentRemoved(container.getMetadata(), segment);
}
}
(input) -> input.segmentRemoved(container.getMetadata(), segment)
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

package io.druid.curator.inventory;

import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.MapMaker;
import com.google.common.collect.Sets;
Expand All @@ -39,7 +38,9 @@
import java.io.IOException;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;

/**
Expand Down Expand Up @@ -69,6 +70,8 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>

private volatile PathChildrenCache childrenCache;

private final CountDownLatch latch = new CountDownLatch(1);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

can we call this initializationDoneLatch or something like that ? also , I think doneInitializing boolean variable can be removed and latch.await(1 ms) can be used to do the boolean check wherever necessary.


public CuratorInventoryManager(
CuratorFramework curatorFramework,
InventoryManagerConfig config,
Expand Down Expand Up @@ -106,7 +109,8 @@ public void start() throws Exception
childrenCache = cacheFactory.make(curatorFramework, config.getContainerPath());
}

childrenCache.getListenable().addListener(new ContainerCacheListener());
ContainerCacheListener containerCacheListener = new ContainerCacheListener();
childrenCache.getListenable().addListener(containerCacheListener);

try {
childrenCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT);
Expand All @@ -122,6 +126,13 @@ public void start() throws Exception
}
throw e;
}

// Block start() till PathChildrenCache is fully loaded
while (!latch.await(1, TimeUnit.MINUTES)) {
log.info("Waiting for PathChildrenCache to be completely loaded.");
}

log.info("PathChildrenCache has been loaded.");
}

@LifecycleStop
Expand Down Expand Up @@ -164,14 +175,7 @@ public Iterable<ContainerClass> getInventory()
{
return Iterables.transform(
containers.values(),
new Function<ContainerHolder, ContainerClass>()
{
@Override
public ContainerClass apply(ContainerHolder input)
{
return input.getContainer();
}
}
(ContainerHolder input) -> input.getContainer()
);
}

Expand All @@ -197,7 +201,7 @@ private class ContainerHolder
PathChildrenCache cache
)
{
this.container = new AtomicReference<ContainerClass>(container);
this.container = new AtomicReference<>(container);
this.cache = cache;
}

Expand Down Expand Up @@ -351,6 +355,7 @@ private void maybeDoneInitializing()
// only fire if we are done initializing the parent PathChildrenCache
if (containersInitialized && uninitializedInventory.isEmpty()) {
doneInitializing = true;
latch.countDown();
strategy.inventoryInitialized();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.api.CuratorListener;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.Watcher;
import org.junit.After;
Expand Down Expand Up @@ -62,7 +61,7 @@ public void tearDown() throws Exception
public void testSanity() throws Exception
{
final MapStrategy strategy = new MapStrategy();
CuratorInventoryManager<Map<String, Integer>, Integer> manager = new CuratorInventoryManager<Map<String, Integer>, Integer>(
CuratorInventoryManager<Map<String, Integer>, Integer> manager = new CuratorInventoryManager<>(
curator, new StringInventoryManagerConfig("/container", "/inventory"), exec, strategy
);

Expand Down Expand Up @@ -114,14 +113,10 @@ curator, new StringInventoryManagerConfig("/container", "/inventory"), exec, str

final CountDownLatch latch = new CountDownLatch(1);
curator.getCuratorListenable().addListener(
new CuratorListener() {
@Override
public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
{
if (event.getType() == CuratorEventType.WATCHED
&& event.getWatchedEvent().getState() == Watcher.Event.KeeperState.Disconnected) {
latch.countDown();
}
(CuratorFramework client, CuratorEvent event) -> {
if (event.getType() == CuratorEventType.WATCHED
&& event.getWatchedEvent().getState() == Watcher.Event.KeeperState.Disconnected) {
latch.countDown();
}
}
);
Expand Down Expand Up @@ -247,11 +242,6 @@ private void setNewContainerLatch(CountDownLatch newContainerLatch)
this.newContainerLatch = newContainerLatch;
}

private void setDeadContainerLatch(CountDownLatch deadContainerLatch)
{
this.deadContainerLatch = deadContainerLatch;
}

private void setNewInventoryLatch(CountDownLatch newInventoryLatch)
{
this.newInventoryLatch = newInventoryLatch;
Expand Down