/
RaptorSplitManager.java
200 lines (174 loc) · 8.25 KB
/
RaptorSplitManager.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.raptor;
import com.facebook.presto.raptor.backup.BackupService;
import com.facebook.presto.raptor.metadata.ShardManager;
import com.facebook.presto.raptor.metadata.ShardNodes;
import com.facebook.presto.raptor.util.CloseableIterator;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorPartition;
import com.facebook.presto.spi.ConnectorPartitionResult;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorSplit;
import com.facebook.presto.spi.ConnectorSplitManager;
import com.facebook.presto.spi.ConnectorSplitSource;
import com.facebook.presto.spi.ConnectorTableHandle;
import com.facebook.presto.spi.HostAddress;
import com.facebook.presto.spi.Node;
import com.facebook.presto.spi.NodeManager;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.TupleDomain;
import com.google.common.collect.ImmutableList;
import javax.annotation.PreDestroy;
import javax.inject.Inject;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import static com.facebook.presto.raptor.RaptorErrorCode.RAPTOR_NO_HOST_FOR_SHARD;
import static com.facebook.presto.raptor.util.Types.checkType;
import static com.facebook.presto.spi.StandardErrorCode.NO_NODES_AVAILABLE;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.collect.Iterables.getOnlyElement;
import static com.google.common.collect.Iterators.limit;
import static com.google.common.collect.Iterators.transform;
import static com.google.common.collect.Maps.uniqueIndex;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.lang.String.format;
import static java.util.concurrent.CompletableFuture.supplyAsync;
import static java.util.concurrent.Executors.newCachedThreadPool;
public class RaptorSplitManager
implements ConnectorSplitManager
{
private final String connectorId;
private final NodeManager nodeManager;
private final ShardManager shardManager;
private final boolean backupAvailable;
private final ExecutorService executor;
@Inject
public RaptorSplitManager(RaptorConnectorId connectorId, NodeManager nodeManager, ShardManager shardManager, BackupService backupService)
{
this(connectorId, nodeManager, shardManager, checkNotNull(backupService, "backupService is null").isBackupAvailable());
}
public RaptorSplitManager(RaptorConnectorId connectorId, NodeManager nodeManager, ShardManager shardManager, boolean backupAvailable)
{
this.connectorId = checkNotNull(connectorId, "connectorId is null").toString();
this.nodeManager = checkNotNull(nodeManager, "nodeManager is null");
this.shardManager = checkNotNull(shardManager, "shardManager is null");
this.backupAvailable = backupAvailable;
this.executor = newCachedThreadPool(daemonThreadsNamed("raptor-split-" + connectorId + "-%s"));
}
@PreDestroy
public void destroy()
{
executor.shutdownNow();
}
@Override
public ConnectorPartitionResult getPartitions(ConnectorSession session, ConnectorTableHandle tableHandle, TupleDomain<ColumnHandle> tupleDomain)
{
RaptorTableHandle handle = checkType(tableHandle, RaptorTableHandle.class, "table");
ConnectorPartition partition = new RaptorPartition(handle.getTableId(), tupleDomain);
return new ConnectorPartitionResult(ImmutableList.of(partition), tupleDomain);
}
@Override
public ConnectorSplitSource getPartitionSplits(ConnectorSession session, ConnectorTableHandle tableHandle, List<ConnectorPartition> partitions)
{
RaptorTableHandle raptorTableHandle = checkType(tableHandle, RaptorTableHandle.class, "tableHandle");
checkArgument(partitions.size() == 1, "expected exactly one partition");
RaptorPartition partition = checkType(getOnlyElement(partitions), RaptorPartition.class, "partition");
TupleDomain<RaptorColumnHandle> effectivePredicate = toRaptorTupleDomain(partition.getEffectivePredicate());
return new RaptorSplitSource(raptorTableHandle.getTableId(), effectivePredicate);
}
private static List<HostAddress> getAddressesForNodes(Map<String, Node> nodeMap, Iterable<String> nodeIdentifiers)
{
ImmutableList.Builder<HostAddress> nodes = ImmutableList.builder();
for (String id : nodeIdentifiers) {
Node node = nodeMap.get(id);
if (node != null) {
nodes.add(node.getHostAndPort());
}
}
return nodes.build();
}
@SuppressWarnings("unchecked")
private static TupleDomain<RaptorColumnHandle> toRaptorTupleDomain(TupleDomain<ColumnHandle> tupleDomain)
{
return tupleDomain.transform(handle -> checkType(handle, RaptorColumnHandle.class, "columnHandle"));
}
private static <T> T selectRandom(Iterable<T> elements)
{
List<T> list = ImmutableList.copyOf(elements);
return list.get(ThreadLocalRandom.current().nextInt(list.size()));
}
private class RaptorSplitSource
implements ConnectorSplitSource
{
private final Map<String, Node> nodesById = uniqueIndex(nodeManager.getActiveNodes(), Node::getNodeIdentifier);
private final long tableId;
private final TupleDomain<RaptorColumnHandle> effectivePredicate;
private final CloseableIterator<ShardNodes> iterator;
public RaptorSplitSource(long tableId, TupleDomain<RaptorColumnHandle> effectivePredicate)
{
this.tableId = tableId;
this.effectivePredicate = checkNotNull(effectivePredicate, "effectivePredicate is null");
this.iterator = shardManager.getShardNodes(tableId, effectivePredicate);
}
@Override
public String getDataSourceName()
{
return connectorId;
}
@Override
public CompletableFuture<List<ConnectorSplit>> getNextBatch(int maxSize)
{
return supplyAsync(() -> ImmutableList.copyOf(transform(limit(iterator, maxSize), this::createSplit)), executor);
}
@Override
public void close()
{
iterator.close();
}
@Override
public boolean isFinished()
{
return !iterator.hasNext();
}
private ConnectorSplit createSplit(ShardNodes shard)
{
UUID shardId = shard.getShardUuid();
Collection<String> nodeIds = shard.getNodeIdentifiers();
List<HostAddress> addresses = getAddressesForNodes(nodesById, nodeIds);
if (addresses.isEmpty()) {
if (!backupAvailable) {
throw new PrestoException(RAPTOR_NO_HOST_FOR_SHARD, format("No host for shard %s found: %s", shardId, nodeIds));
}
// Pick a random node and optimistically assign the shard to it.
// That node will restore the shard from the backup location.
Set<Node> availableNodes = nodeManager.getActiveDatasourceNodes(connectorId);
if (availableNodes.isEmpty()) {
throw new PrestoException(NO_NODES_AVAILABLE, "No nodes available to run query");
}
Node node = selectRandom(availableNodes);
shardManager.assignShard(tableId, shardId, node.getNodeIdentifier());
addresses = ImmutableList.of(node.getHostAndPort());
}
return new RaptorSplit(shardId, addresses, effectivePredicate);
}
}
}