Skip to content

Commit 21098e3

Browse files
sohamiHanumathRao
authored andcommitted
UUID support for DrillbitEndpoint
RMState Blobs definition, serialization and deserialization, Zookeeper client support for transactions ZookeeperPersistentTransactional Store and RMStateBlobManager to do updates under lock Protect running and waiting queries map in WorkerBee
1 parent 963cf38 commit 21098e3

35 files changed

+2047
-214
lines changed

exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -691,6 +691,7 @@ private ExecConstants() {
691691
new OptionDescription("Indicates how long a query can wait in queue before the query fails. Range: 0-9223372036854775807"));
692692

693693
// New Smart RM boot time configs
694+
public static final String RM_WAIT_THREAD_INTERVAL = "exec.rm.wait_thread_interval";
694695
public static final String RM_QUERY_TAGS_KEY = "exec.rm.queryTags";
695696
public static final StringValidator RM_QUERY_TAGS_VALIDATOR = new StringValidator(RM_QUERY_TAGS_KEY,
696697
new OptionDescription("Allows user to set coma separated list of tags for all the queries submitted over a session"));

exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java

Lines changed: 15 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,16 +17,17 @@
1717
*/
1818
package org.apache.drill.exec.coord;
1919

20-
import java.util.Collection;
21-
import java.util.Set;
22-
import java.util.concurrent.ConcurrentHashMap;
23-
2420
import org.apache.drill.exec.coord.store.TransientStore;
2521
import org.apache.drill.exec.coord.store.TransientStoreConfig;
2622
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
2723
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.State;
2824
import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
2925

26+
import java.util.Collection;
27+
import java.util.Map;
28+
import java.util.Set;
29+
import java.util.concurrent.ConcurrentHashMap;
30+
3031
/**
3132
* Pluggable interface built to manage cluster coordination. Allows Drillbit or DrillClient to register its capabilities
3233
* as well as understand other node's existence and capabilities.
@@ -60,6 +61,10 @@ public abstract class ClusterCoordinator implements AutoCloseable {
6061
*/
6162
public abstract Collection<DrillbitEndpoint> getAvailableEndpoints();
6263

64+
public Map<String, DrillbitEndpoint> getAvailableEndpointsUUID() {
65+
throw new UnsupportedOperationException("Only supported by Zookeeper Cluster Coordinator outside YARN");
66+
}
67+
6368
/**
6469
* Get a collection of ONLINE drillbit endpoints by excluding the drillbits
6570
* that are in QUIESCENT state (drillbits that are shutting down). Primarily used by the planner
@@ -70,6 +75,10 @@ public abstract class ClusterCoordinator implements AutoCloseable {
7075

7176
public abstract Collection<DrillbitEndpoint> getOnlineEndPoints();
7277

78+
public Map<DrillbitEndpoint, String> getOnlineEndpointsUUID() {
79+
throw new UnsupportedOperationException("Only supported by Zookeeper Cluster Coordinator outside YARN");
80+
}
81+
7382
public abstract RegistrationHandle update(RegistrationHandle handle, State state);
7483

7584
public interface RegistrationHandle {
@@ -79,6 +88,8 @@ public interface RegistrationHandle {
7988
*/
8089
public abstract DrillbitEndpoint getEndPoint();
8190

91+
public abstract String getId();
92+
8293
public abstract void setEndPoint(DrillbitEndpoint endpoint);
8394
}
8495

exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java

Lines changed: 34 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,6 @@
1717
*/
1818
package org.apache.drill.exec.coord.local;
1919

20-
import java.util.ArrayList;
21-
import java.util.Collection;
22-
import java.util.Map;
23-
import java.util.UUID;
24-
import java.util.concurrent.ConcurrentHashMap;
25-
import java.util.concurrent.ConcurrentMap;
26-
import java.util.concurrent.Semaphore;
27-
import java.util.concurrent.TimeUnit;
28-
2920
import org.apache.drill.exec.coord.ClusterCoordinator;
3021
import org.apache.drill.exec.coord.DistributedSemaphore;
3122
import org.apache.drill.exec.coord.store.CachingTransientStoreFactory;
@@ -34,9 +25,17 @@
3425
import org.apache.drill.exec.coord.store.TransientStoreFactory;
3526
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
3627
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.State;
37-
3828
import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
3929

30+
import java.util.Collection;
31+
import java.util.HashMap;
32+
import java.util.Map;
33+
import java.util.UUID;
34+
import java.util.concurrent.ConcurrentHashMap;
35+
import java.util.concurrent.ConcurrentMap;
36+
import java.util.concurrent.Semaphore;
37+
import java.util.concurrent.TimeUnit;
38+
4039
public class LocalClusterCoordinator extends ClusterCoordinator {
4140
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalClusterCoordinator.class);
4241

@@ -103,7 +102,15 @@ public RegistrationHandle update(RegistrationHandle handle, State state) {
103102

104103
@Override
105104
public Collection<DrillbitEndpoint> getAvailableEndpoints() {
106-
return endpoints.values();
105+
return getAvailableEndpointsUUID().values();
106+
}
107+
108+
public Map<String, DrillbitEndpoint> getAvailableEndpointsUUID() {
109+
Map<String, DrillbitEndpoint> availableEndpointsUUID = new HashMap<>();
110+
for (Map.Entry<RegistrationHandle, DrillbitEndpoint> entry : endpoints.entrySet()) {
111+
availableEndpointsUUID.put(entry.getKey().getId(), entry.getValue());
112+
}
113+
return availableEndpointsUUID;
107114
}
108115

109116
/**
@@ -115,19 +122,27 @@ public Collection<DrillbitEndpoint> getAvailableEndpoints() {
115122
*/
116123
@Override
117124
public Collection<DrillbitEndpoint> getOnlineEndPoints() {
118-
Collection<DrillbitEndpoint> runningEndPoints = new ArrayList<>();
119-
for (DrillbitEndpoint endpoint: endpoints.values()){
120-
if(isDrillbitInState(endpoint, State.ONLINE)) {
121-
runningEndPoints.add(endpoint);
125+
return getOnlineEndpointsUUID().keySet();
126+
}
127+
128+
public Map<DrillbitEndpoint, String> getOnlineEndpointsUUID() {
129+
Map<DrillbitEndpoint, String> onlineEndpointsUUID = new HashMap<>();
130+
for (Map.Entry<RegistrationHandle, DrillbitEndpoint> entry : endpoints.entrySet()) {
131+
if(isDrillbitInState(entry.getValue(), State.ONLINE)) {
132+
onlineEndpointsUUID.put(entry.getValue(), entry.getKey().getId());
122133
}
123134
}
124-
return runningEndPoints;
135+
return onlineEndpointsUUID;
125136
}
126137

127138
private class Handle implements RegistrationHandle {
128139
private final UUID id = UUID.randomUUID();
129140
private DrillbitEndpoint drillbitEndpoint;
130141

142+
private Handle(DrillbitEndpoint data) {
143+
drillbitEndpoint = data;
144+
}
145+
131146
/**
132147
* Get the drillbit endpoint associated with the registration handle
133148
* @return drillbit endpoint
@@ -140,8 +155,9 @@ public void setEndPoint(DrillbitEndpoint endpoint) {
140155
this.drillbitEndpoint = endpoint;
141156
}
142157

143-
private Handle(DrillbitEndpoint data) {
144-
drillbitEndpoint = data;
158+
@Override
159+
public String getId() {
160+
return id.toString();
145161
}
146162

147163
@Override

exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java

Lines changed: 56 additions & 47 deletions
Original file line numberDiff line numberDiff line change
@@ -17,26 +17,11 @@
1717
*/
1818
package org.apache.drill.exec.coord.zk;
1919

20-
import static org.apache.drill.shaded.guava.com.google.common.collect.Collections2.transform;
21-
import java.io.IOException;
22-
import java.util.Collection;
23-
import java.util.Collections;
24-
import java.util.ArrayList;
25-
import java.util.Set;
26-
import java.util.HashSet;
27-
import java.util.concurrent.ConcurrentHashMap;
28-
import java.util.concurrent.CountDownLatch;
29-
import java.util.concurrent.TimeUnit;
30-
import java.util.regex.Matcher;
31-
import java.util.regex.Pattern;
32-
33-
import org.apache.curator.framework.imps.DefaultACLProvider;
34-
import org.apache.drill.shaded.guava.com.google.common.base.Throwables;
35-
import org.apache.commons.collections.keyvalue.MultiKey;
3620
import org.apache.curator.RetryPolicy;
3721
import org.apache.curator.framework.CuratorFramework;
3822
import org.apache.curator.framework.CuratorFrameworkFactory;
3923
import org.apache.curator.framework.api.ACLProvider;
24+
import org.apache.curator.framework.imps.DefaultACLProvider;
4025
import org.apache.curator.framework.state.ConnectionState;
4126
import org.apache.curator.framework.state.ConnectionStateListener;
4227
import org.apache.curator.retry.RetryNTimes;
@@ -57,7 +42,21 @@
5742
import org.apache.drill.exec.coord.store.TransientStoreFactory;
5843
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
5944
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.State;
60-
import org.apache.drill.shaded.guava.com.google.common.base.Function;
45+
import org.apache.drill.shaded.guava.com.google.common.base.Throwables;
46+
47+
import java.io.IOException;
48+
import java.util.Collection;
49+
import java.util.Collections;
50+
import java.util.HashMap;
51+
import java.util.HashSet;
52+
import java.util.Map;
53+
import java.util.Set;
54+
import java.util.concurrent.ConcurrentHashMap;
55+
import java.util.concurrent.CountDownLatch;
56+
import java.util.concurrent.TimeUnit;
57+
import java.util.regex.Matcher;
58+
import java.util.regex.Pattern;
59+
import java.util.stream.Collectors;
6160

6261
/**
6362
* Manages cluster coordination utilizing zookeeper. *
@@ -74,8 +73,8 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
7473
private ServiceCache<DrillbitEndpoint> serviceCache;
7574
private DrillbitEndpoint endpoint;
7675

77-
// endpointsMap maps Multikey( comprises of endoint address and port) to Drillbit endpoints
78-
private ConcurrentHashMap<MultiKey, DrillbitEndpoint> endpointsMap = new ConcurrentHashMap<MultiKey,DrillbitEndpoint>();
76+
// endpointsMap maps String UUID to Drillbit endpoints
77+
private ConcurrentHashMap<String, DrillbitEndpoint> endpointsMap = new ConcurrentHashMap<>();
7978
private static final Pattern ZK_COMPLEX_STRING = Pattern.compile("(^.*?)/(.*)/([^/]*)$");
8079

8180
public ZKClusterCoordinator(DrillConfig config, String connect) {
@@ -237,7 +236,12 @@ public RegistrationHandle update(RegistrationHandle handle, State state) {
237236

238237
@Override
239238
public Collection<DrillbitEndpoint> getAvailableEndpoints() {
240-
return this.endpoints;
239+
return getAvailableEndpointsUUID().values();
240+
}
241+
242+
@Override
243+
public Map<String, DrillbitEndpoint> getAvailableEndpointsUUID() {
244+
return this.endpointsMap;
241245
}
242246

243247
/*
@@ -249,14 +253,19 @@ public Collection<DrillbitEndpoint> getAvailableEndpoints() {
249253
*/
250254
@Override
251255
public Collection<DrillbitEndpoint> getOnlineEndPoints() {
252-
Collection<DrillbitEndpoint> runningEndPoints = new ArrayList<>();
253-
for (DrillbitEndpoint endpoint: endpoints){
254-
if(isDrillbitInState(endpoint, State.ONLINE)) {
255-
runningEndPoints.add(endpoint);
256+
return getOnlineEndpointsUUID().keySet();
257+
}
258+
259+
@Override
260+
public Map<DrillbitEndpoint, String> getOnlineEndpointsUUID() {
261+
Map<DrillbitEndpoint, String> onlineEndpointsUUID = new HashMap<>();
262+
for (Map.Entry<String, DrillbitEndpoint> endpointEntry : endpointsMap.entrySet()) {
263+
if (isDrillbitInState(endpointEntry.getValue(), State.ONLINE)) {
264+
onlineEndpointsUUID.put(endpointEntry.getValue(), endpointEntry.getKey());
256265
}
257266
}
258-
logger.debug("Online endpoints in ZK are" + runningEndPoints.toString());
259-
return runningEndPoints;
267+
logger.debug("Online endpoints in ZK are" + onlineEndpointsUUID.keySet().toString());
268+
return onlineEndpointsUUID;
260269
}
261270

262271
@Override
@@ -273,14 +282,11 @@ public <V> TransientStore<V> getOrCreateTransientStore(final TransientStoreConfi
273282
private synchronized void updateEndpoints() {
274283
try {
275284
// All active bits in the Zookeeper
276-
Collection<DrillbitEndpoint> newDrillbitSet =
277-
transform(discovery.queryForInstances(serviceName),
278-
new Function<ServiceInstance<DrillbitEndpoint>, DrillbitEndpoint>() {
279-
@Override
280-
public DrillbitEndpoint apply(ServiceInstance<DrillbitEndpoint> input) {
281-
return input.getPayload();
282-
}
283-
});
285+
final Map<String, DrillbitEndpoint> activeEndpointsUUID = discovery.queryForInstances(serviceName).stream()
286+
.collect(Collectors.toMap(ServiceInstance::getId, ServiceInstance::getPayload));
287+
288+
final Map<DrillbitEndpoint, String> UUIDtoEndpoints = activeEndpointsUUID.entrySet().stream()
289+
.collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
284290

285291
// set of newly dead bits : original bits - new set of active bits.
286292
Set<DrillbitEndpoint> unregisteredBits = new HashSet<>();
@@ -290,29 +296,32 @@ public DrillbitEndpoint apply(ServiceInstance<DrillbitEndpoint> input) {
290296

291297
// Updates the endpoints map if there is a change in state of the endpoint or with the addition
292298
// of new drillbit endpoints. Registered endpoints is set to newly live drillbit endpoints.
293-
for ( DrillbitEndpoint endpoint : newDrillbitSet) {
294-
String endpointAddress = endpoint.getAddress();
295-
int endpointPort = endpoint.getUserPort();
296-
if (! endpointsMap.containsKey(new MultiKey(endpointAddress, endpointPort))) {
297-
registeredBits.add(endpoint);
298-
}
299-
endpointsMap.put(new MultiKey(endpointAddress, endpointPort),endpoint);
299+
for (Map.Entry<String, DrillbitEndpoint> endpointToUUID : activeEndpointsUUID.entrySet()) {
300+
endpointsMap.put(endpointToUUID.getKey(), endpointToUUID.getValue());
300301
}
302+
301303
// Remove all the endpoints that are newly dead
302-
for ( MultiKey key: endpointsMap.keySet()) {
303-
if(!newDrillbitSet.contains(endpointsMap.get(key))) {
304-
unregisteredBits.add(endpointsMap.get(key));
305-
endpointsMap.remove(key);
304+
for ( String bitUUID: endpointsMap.keySet()) {
305+
if (!activeEndpointsUUID.containsKey(bitUUID)) {
306+
final DrillbitEndpoint unregisteredBit = endpointsMap.get(bitUUID);
307+
unregisteredBits.add(unregisteredBit);
308+
309+
if (UUIDtoEndpoints.containsKey(unregisteredBit)) {
310+
logger.info("Drillbit registered again with different UUID. [Details: Address: {}, UserPort: {}," +
311+
" PreviousUUID: {}, CurrentUUID: {}", unregisteredBit.getAddress(), unregisteredBit.getUserPort(),
312+
bitUUID, UUIDtoEndpoints.get(unregisteredBit));
313+
}
314+
endpointsMap.remove(bitUUID);
306315
}
307316
}
308317
endpoints = endpointsMap.values();
309318
if (logger.isDebugEnabled()) {
310319
StringBuilder builder = new StringBuilder();
311320
builder.append("Active drillbit set changed. Now includes ");
312-
builder.append(newDrillbitSet.size());
321+
builder.append(activeEndpointsUUID.size());
313322
builder.append(" total bits. New active drillbits:\n");
314323
builder.append("Address | User Port | Control Port | Data Port | Version | State\n");
315-
for (DrillbitEndpoint bit: newDrillbitSet) {
324+
for (DrillbitEndpoint bit: activeEndpointsUUID.values()) {
316325
builder.append(bit.getAddress()).append(" | ");
317326
builder.append(bit.getUserPort()).append(" | ");
318327
builder.append(bit.getControlPort()).append(" | ");

exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKRegistrationHandle.java

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -26,19 +26,24 @@ public class ZKRegistrationHandle implements RegistrationHandle {
2626
public final String id;
2727
public DrillbitEndpoint endpoint;
2828

29+
public ZKRegistrationHandle(String id, DrillbitEndpoint endpoint) {
30+
super();
31+
this.id = id;
32+
this.endpoint = endpoint;
33+
}
34+
2935
public DrillbitEndpoint getEndPoint() {
3036
return endpoint;
3137
}
3238

39+
public String getId() {
40+
return id;
41+
}
42+
3343
@Override
3444
public void setEndPoint(DrillbitEndpoint endpoint) {
3545
this.endpoint = endpoint;
3646
}
3747

38-
public ZKRegistrationHandle(String id, DrillbitEndpoint endpoint) {
39-
super();
40-
this.id = id;
41-
this.endpoint = endpoint;
42-
}
4348

4449
}

0 commit comments

Comments
 (0)