more code review comments

This commit is contained in:
fjy 2014-05-19 17:52:34 -07:00
parent 6126dd4231
commit 1addea4dbe
4 changed files with 13 additions and 43 deletions

View File

@ -101,29 +101,4 @@ public class ImmutableDruidServer
{ {
return segments; return segments;
} }
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ImmutableDruidServer that = (ImmutableDruidServer) o;
if (metadata != null ? !metadata.equals(that.metadata) : that.metadata != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
return metadata != null ? metadata.hashCode() : 0;
}
} }

View File

@ -51,7 +51,6 @@ import java.sql.SQLException;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -125,7 +124,7 @@ public class DatabaseRuleManager
private final Supplier<DatabaseRuleManagerConfig> config; private final Supplier<DatabaseRuleManagerConfig> config;
private final Supplier<DbTablesConfig> dbTables; private final Supplier<DbTablesConfig> dbTables;
private final IDBI dbi; private final IDBI dbi;
private final AtomicReference<ConcurrentHashMap<String, List<Rule>>> rules; private final AtomicReference<ImmutableMap<String, List<Rule>>> rules;
private volatile ScheduledExecutorService exec; private volatile ScheduledExecutorService exec;
@ -146,8 +145,8 @@ public class DatabaseRuleManager
this.dbTables = dbTables; this.dbTables = dbTables;
this.dbi = dbi; this.dbi = dbi;
this.rules = new AtomicReference<ConcurrentHashMap<String, List<Rule>>>( this.rules = new AtomicReference<>(
new ConcurrentHashMap<String, List<Rule>>() ImmutableMap.<String, List<Rule>>of()
); );
} }
@ -188,7 +187,7 @@ public class DatabaseRuleManager
return; return;
} }
rules.set(new ConcurrentHashMap<String, List<Rule>>()); rules.set(ImmutableMap.<String, List<Rule>>of());
started = false; started = false;
exec.shutdownNow(); exec.shutdownNow();
@ -199,7 +198,7 @@ public class DatabaseRuleManager
public void poll() public void poll()
{ {
try { try {
ConcurrentHashMap<String, List<Rule>> newRules = new ConcurrentHashMap<String, List<Rule>>( ImmutableMap<String, List<Rule>> newRules = ImmutableMap.copyOf(
dbi.withHandle( dbi.withHandle(
new HandleCallback<Map<String, List<Rule>>>() new HandleCallback<Map<String, List<Rule>>>()
{ {
@ -309,12 +308,6 @@ public class DatabaseRuleManager
} }
} }
); );
ConcurrentHashMap<String, List<Rule>> existingRules = rules.get();
if (existingRules == null) {
existingRules = new ConcurrentHashMap<String, List<Rule>>();
}
existingRules.put(dataSource, newRules);
} }
catch (Exception e) { catch (Exception e) {
log.error(e, String.format("Exception while overriding rule for %s", dataSource)); log.error(e, String.format("Exception while overriding rule for %s", dataSource));

View File

@ -350,6 +350,10 @@ public class DruidCoordinator
) )
{ {
try { try {
if (fromServer.getMetadata().equals(toServer.getMetadata())) {
throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentName, fromServer.getName());
}
final DataSegment segment = fromServer.getSegment(segmentName); final DataSegment segment = fromServer.getSegment(segmentName);
if (segment == null) { if (segment == null) {
throw new IAE("Unable to find segment [%s] on server [%s]", segmentName, fromServer.getName()); throw new IAE("Unable to find segment [%s] on server [%s]", segmentName, fromServer.getName());

View File

@ -94,10 +94,8 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
} }
if (!foundMatchingRule) { if (!foundMatchingRule) {
log.makeAlert( log.makeAlert("Unable to find a matching rule!")
"Unable to find a matching rule for dataSource[%s]", .addData("dataSource", segment.getDataSource())
segment.getDataSource()
)
.addData("segment", segment.getIdentifier()) .addData("segment", segment.getIdentifier())
.emit(); .emit();
} }