mirror of https://github.com/apache/druid.git
1) Nicer logging
2) Announcer should actually delete its zk entries when it is told to unannounce something 3) Skip over complex columns that we don't have a proper deserializer for
This commit is contained in:
parent
b8ba9138ff
commit
9153cc94ac
|
@ -22,18 +22,15 @@ package com.metamx.druid.coordination;
|
||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.client.DruidServer;
|
|
||||||
import com.metamx.druid.curator.announcement.Announcer;
|
import com.metamx.druid.curator.announcement.Announcer;
|
||||||
import com.metamx.druid.initialization.ZkPathsConfig;
|
import com.metamx.druid.initialization.ZkPathsConfig;
|
||||||
import com.netflix.curator.utils.ZKPaths;
|
import com.netflix.curator.utils.ZKPaths;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
|
public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
|
||||||
{
|
{
|
||||||
|
@ -100,14 +97,16 @@ public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
|
||||||
|
|
||||||
public void announceSegment(DataSegment segment) throws IOException
|
public void announceSegment(DataSegment segment) throws IOException
|
||||||
{
|
{
|
||||||
log.info("Announcing realtime segment %s", segment.getIdentifier());
|
final String path = makeServedSegmentPath(segment);
|
||||||
announcer.announce(makeServedSegmentPath(segment), jsonMapper.writeValueAsBytes(segment));
|
log.info("Announcing segment[%s] to path[%s]", segment.getIdentifier(), path);
|
||||||
|
announcer.announce(path, jsonMapper.writeValueAsBytes(segment));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void unannounceSegment(DataSegment segment) throws IOException
|
public void unannounceSegment(DataSegment segment) throws IOException
|
||||||
{
|
{
|
||||||
log.info("Unannouncing realtime segment %s", segment.getIdentifier());
|
final String path = makeServedSegmentPath(segment);
|
||||||
announcer.unannounce(makeServedSegmentPath(segment));
|
log.info("Unannouncing segment[%s] at path[%s]", segment.getIdentifier(), path);
|
||||||
|
announcer.unannounce(path);
|
||||||
}
|
}
|
||||||
|
|
||||||
private String makeAnnouncementPath() {
|
private String makeAnnouncementPath() {
|
||||||
|
|
|
@ -190,6 +190,7 @@ public class Announcer
|
||||||
*/
|
*/
|
||||||
public void unannounce(String path)
|
public void unannounce(String path)
|
||||||
{
|
{
|
||||||
|
log.info("unannouncing [%s]", path);
|
||||||
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
|
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
|
||||||
final String parentPath = pathAndNode.getPath();
|
final String parentPath = pathAndNode.getPath();
|
||||||
|
|
||||||
|
@ -202,5 +203,12 @@ public class Announcer
|
||||||
if (subPaths.remove(pathAndNode.getNode()) == null) {
|
if (subPaths.remove(pathAndNode.getNode()) == null) {
|
||||||
throw new IAE("Path[%s] not announced, cannot unannounce.", path);
|
throw new IAE("Path[%s] not announced, cannot unannounce.", path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
curator.delete().guaranteed().forPath(path);
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package com.metamx.druid.curator.inventory;
|
package com.metamx.druid.curator.inventory;
|
||||||
|
|
||||||
|
import com.google.common.base.Charsets;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -30,10 +31,13 @@ import com.metamx.druid.curator.ShutdownNowIgnoringExecutorService;
|
||||||
import com.metamx.druid.curator.cache.PathChildrenCacheFactory;
|
import com.metamx.druid.curator.cache.PathChildrenCacheFactory;
|
||||||
import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory;
|
import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory;
|
||||||
import com.netflix.curator.framework.CuratorFramework;
|
import com.netflix.curator.framework.CuratorFramework;
|
||||||
|
import com.netflix.curator.framework.CuratorFrameworkFactory;
|
||||||
import com.netflix.curator.framework.recipes.cache.ChildData;
|
import com.netflix.curator.framework.recipes.cache.ChildData;
|
||||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
||||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
|
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
|
||||||
|
import com.netflix.curator.retry.RetryOneTime;
|
||||||
|
import com.netflix.curator.utils.ZKPaths;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
@ -189,39 +193,36 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
||||||
private class ContainerCacheListener implements PathChildrenCacheListener
|
private class ContainerCacheListener implements PathChildrenCacheListener
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public void childEvent(
|
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
|
||||||
CuratorFramework client, PathChildrenCacheEvent event
|
|
||||||
) throws Exception
|
|
||||||
{
|
{
|
||||||
final ChildData containerChild = event.getData();
|
final ChildData child = event.getData();
|
||||||
final String containerKey = containerChild.getPath().substring(config.getInventoryPath().length());
|
final String containerKey = ZKPaths.getNodeFromPath(child.getPath());
|
||||||
final ContainerClass container;
|
final ContainerClass container;
|
||||||
|
|
||||||
switch (event.getType()) {
|
switch (event.getType()) {
|
||||||
case CHILD_ADDED:
|
case CHILD_ADDED:
|
||||||
container = strategy.deserializeContainer(containerChild.getData());
|
container = strategy.deserializeContainer(child.getData());
|
||||||
|
|
||||||
// This would normally be a race condition, but the only thing that should be mutating the containers
|
// This would normally be a race condition, but the only thing that should be mutating the containers
|
||||||
// map is this listener, which should never run concurrently. If the same container is going to disappear
|
// map is this listener, which should never run concurrently. If the same container is going to disappear
|
||||||
// and come back, we expect a removed event in between.
|
// and come back, we expect a removed event in between.
|
||||||
if (containers.containsKey(containerKey)) {
|
if (containers.containsKey(containerKey)) {
|
||||||
log.error("New node[%s] but there was already one. That's not good, ignoring new one.", containerChild.getPath());
|
log.error("New node[%s] but there was already one. That's not good, ignoring new one.", child.getPath());
|
||||||
}
|
}
|
||||||
|
|
||||||
final String inventoryPath = String.format("%s/%s", config.getContainerPath(), containerKey);
|
final String inventoryPath = String.format("%s/%s", config.getInventoryPath(), containerKey);
|
||||||
log.info("Creating listener on inventory[%s]", inventoryPath);
|
PathChildrenCache inventoryCache = cacheFactory.make(curatorFramework, inventoryPath);
|
||||||
PathChildrenCache containerCache = cacheFactory.make(curatorFramework, inventoryPath);
|
inventoryCache.getListenable().addListener(new InventoryCacheListener(containerKey, inventoryPath));
|
||||||
containerCache.getListenable().addListener(new InventoryCacheListener(containerKey, inventoryPath));
|
|
||||||
|
|
||||||
containers.put(containerKey, new ContainerHolder(container, containerCache));
|
containers.put(containerKey, new ContainerHolder(container, inventoryCache));
|
||||||
|
|
||||||
containerCache.start();
|
inventoryCache.start();
|
||||||
|
|
||||||
break;
|
break;
|
||||||
case CHILD_REMOVED:
|
case CHILD_REMOVED:
|
||||||
final ContainerHolder removed = containers.remove(containerKey);
|
final ContainerHolder removed = containers.remove(containerKey);
|
||||||
if (removed == null) {
|
if (removed == null) {
|
||||||
log.warn("Container[%s] removed that wasn't a container!?", containerChild.getPath());
|
log.warn("Container[%s] removed that wasn't a container!?", child.getPath());
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -232,11 +233,11 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
||||||
|
|
||||||
break;
|
break;
|
||||||
case CHILD_UPDATED:
|
case CHILD_UPDATED:
|
||||||
container = strategy.deserializeContainer(containerChild.getData());
|
container = strategy.deserializeContainer(child.getData());
|
||||||
|
|
||||||
ContainerHolder oldContainer = containers.get(containerKey);
|
ContainerHolder oldContainer = containers.get(containerKey);
|
||||||
if (oldContainer == null) {
|
if (oldContainer == null) {
|
||||||
log.warn("Container update[%s], but the old container didn't exist!? Ignoring.", containerChild.getPath());
|
log.warn("Container update[%s], but the old container didn't exist!? Ignoring.", child.getPath());
|
||||||
}
|
}
|
||||||
else {
|
else {
|
||||||
synchronized (oldContainer) {
|
synchronized (oldContainer) {
|
||||||
|
@ -245,8 +246,6 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
||||||
}
|
}
|
||||||
|
|
||||||
break;
|
break;
|
||||||
default:
|
|
||||||
log.info("Got event[%s] at containerPath[%s]", config.getInventoryPath());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -262,24 +261,22 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void childEvent(
|
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
|
||||||
CuratorFramework client, PathChildrenCacheEvent event
|
|
||||||
) throws Exception
|
|
||||||
{
|
{
|
||||||
final ChildData inventoryChild = event.getData();
|
final ChildData child = event.getData();
|
||||||
final ContainerHolder holder = containers.get(containerKey);
|
final ContainerHolder holder = containers.get(containerKey);
|
||||||
|
|
||||||
if (holder == null) {
|
if (holder == null) {
|
||||||
log.error("Inventory[%s] change for non-existent container[%s]!?", inventoryChild.getPath());
|
log.error("Inventory[%s] change for non-existent container[%s]!?", child.getPath());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final String inventoryKey = inventoryChild.getPath().substring(inventoryPath.length());
|
final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath());
|
||||||
|
|
||||||
switch (event.getType()) {
|
switch (event.getType()) {
|
||||||
case CHILD_ADDED:
|
case CHILD_ADDED:
|
||||||
case CHILD_UPDATED:
|
case CHILD_UPDATED:
|
||||||
final InventoryClass inventory = strategy.deserializeInventory(inventoryChild.getData());
|
final InventoryClass inventory = strategy.deserializeInventory(child.getData());
|
||||||
|
|
||||||
synchronized (holder) {
|
synchronized (holder) {
|
||||||
holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, inventory));
|
holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, inventory));
|
||||||
|
|
|
@ -75,6 +75,6 @@ public class ComplexColumnPartSerde implements ColumnPartSerde
|
||||||
@Override
|
@Override
|
||||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
|
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
|
||||||
{
|
{
|
||||||
return serde.deserializeColumn(buffer, builder);
|
return serde == null ? this : serde.deserializeColumn(buffer, builder);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue