/
PulsarSplitManager.java
457 lines (404 loc) · 21.8 KB
/
PulsarSplitManager.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
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.pulsar.sql.presto;
import static com.google.common.base.Preconditions.checkArgument;
import static io.prestosql.spi.StandardErrorCode.QUERY_REJECTED;
import static java.util.Objects.requireNonNull;
import static org.apache.bookkeeper.mledger.ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries;
import static org.apache.pulsar.sql.presto.PulsarConnectorUtils.restoreNamespaceDelimiterIfNeeded;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicate;
import io.airlift.log.Logger;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.connector.ColumnHandle;
import io.prestosql.spi.connector.ConnectorSession;
import io.prestosql.spi.connector.ConnectorSplitManager;
import io.prestosql.spi.connector.ConnectorSplitSource;
import io.prestosql.spi.connector.ConnectorTableLayoutHandle;
import io.prestosql.spi.connector.ConnectorTransactionHandle;
import io.prestosql.spi.connector.FixedSplitSource;
import io.prestosql.spi.predicate.Domain;
import io.prestosql.spi.predicate.Range;
import io.prestosql.spi.predicate.TupleDomain;
import java.io.IOException;
import java.sql.Timestamp;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import javax.inject.Inject;
import lombok.Data;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
import org.apache.bookkeeper.mledger.ReadOnlyCursor;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl;
import org.apache.pulsar.common.schema.SchemaInfo;
/**
* The class helping to manage splits.
*/
public class PulsarSplitManager implements ConnectorSplitManager {
private final String connectorId;
private final PulsarConnectorConfig pulsarConnectorConfig;
private final PulsarAdmin pulsarAdmin;
private static final Logger log = Logger.get(PulsarSplitManager.class);
private ObjectMapper objectMapper = new ObjectMapper();
@Inject
public PulsarSplitManager(PulsarConnectorId connectorId, PulsarConnectorConfig pulsarConnectorConfig) {
this.connectorId = requireNonNull(connectorId, "connectorId is null").toString();
this.pulsarConnectorConfig = requireNonNull(pulsarConnectorConfig, "pulsarConnectorConfig is null");
try {
this.pulsarAdmin = pulsarConnectorConfig.getPulsarAdmin();
} catch (PulsarClientException e) {
log.error(e);
throw new RuntimeException(e);
}
}
@Override
public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHandle, ConnectorSession session,
ConnectorTableLayoutHandle layout,
ConnectorSplitManager.SplitSchedulingStrategy splitSchedulingStrategy) {
int numSplits = this.pulsarConnectorConfig.getTargetNumSplits();
PulsarTableLayoutHandle layoutHandle = (PulsarTableLayoutHandle) layout;
PulsarTableHandle tableHandle = layoutHandle.getTable();
TupleDomain<ColumnHandle> tupleDomain = layoutHandle.getTupleDomain();
String namespace = restoreNamespaceDelimiterIfNeeded(tableHandle.getSchemaName(), pulsarConnectorConfig);
TopicName topicName = TopicName.get("persistent", NamespaceName.get(namespace), tableHandle.getTopicName());
SchemaInfo schemaInfo;
try {
schemaInfo = this.pulsarAdmin.schemas().getSchemaInfo(
String.format("%s/%s", namespace, tableHandle.getTopicName()));
} catch (PulsarAdminException e) {
if (e.getStatusCode() == 401) {
throw new PrestoException(QUERY_REJECTED,
String.format("Failed to get pulsar topic schema for topic %s/%s: Unauthorized",
namespace, tableHandle.getTopicName()));
} else if (e.getStatusCode() == 404) {
schemaInfo = PulsarSqlSchemaInfoProvider.defaultSchema();
} else {
throw new RuntimeException("Failed to get pulsar topic schema for topic "
+ String.format("%s/%s", namespace, tableHandle.getTopicName())
+ ": " + ExceptionUtils.getRootCause(e).getLocalizedMessage(), e);
}
}
Collection<PulsarSplit> splits;
try {
OffloadPoliciesImpl offloadPolicies = (OffloadPoliciesImpl) this.pulsarAdmin.namespaces()
.getOffloadPolicies(topicName.getNamespace());
if (offloadPolicies != null) {
offloadPolicies.setOffloadersDirectory(pulsarConnectorConfig.getOffloadersDirectory());
offloadPolicies.setManagedLedgerOffloadMaxThreads(
pulsarConnectorConfig.getManagedLedgerOffloadMaxThreads());
}
if (!PulsarConnectorUtils.isPartitionedTopic(topicName, this.pulsarAdmin)) {
splits = getSplitsNonPartitionedTopic(
numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies);
log.debug("Splits for non-partitioned topic %s: %s", topicName, splits);
} else {
splits = getSplitsPartitionedTopic(
numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies);
log.debug("Splits for partitioned topic %s: %s", topicName, splits);
}
} catch (Exception e) {
log.error(e, "Failed to get splits");
throw new RuntimeException(e);
}
return new FixedSplitSource(splits);
}
@VisibleForTesting
Collection<PulsarSplit> getSplitsPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle
tableHandle, SchemaInfo schemaInfo, TupleDomain<ColumnHandle> tupleDomain,
OffloadPoliciesImpl offloadPolicies) throws Exception {
List<Integer> predicatedPartitions = getPredicatedPartitions(topicName, tupleDomain);
if (log.isDebugEnabled()) {
log.debug("Partition filter result %s", predicatedPartitions);
}
int actualNumSplits = Math.max(predicatedPartitions.size(), numSplits);
int splitsPerPartition = actualNumSplits / predicatedPartitions.size();
int splitRemainder = actualNumSplits % predicatedPartitions.size();
PulsarConnectorCache pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig);
ManagedLedgerFactory managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory();
ManagedLedgerConfig managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig(
topicName.getNamespaceObject(), offloadPolicies, pulsarConnectorConfig);
List<PulsarSplit> splits = new LinkedList<>();
for (int i = 0; i < predicatedPartitions.size(); i++) {
int splitsForThisPartition = (splitRemainder > i) ? splitsPerPartition + 1 : splitsPerPartition;
splits.addAll(
getSplitsForTopic(
topicName.getPartition(predicatedPartitions.get(i)).getPersistenceNamingEncoding(),
managedLedgerFactory,
managedLedgerConfig,
splitsForThisPartition,
tableHandle,
schemaInfo,
topicName.getPartition(predicatedPartitions.get(i)).getLocalName(),
tupleDomain,
offloadPolicies));
}
return splits;
}
private List<Integer> getPredicatedPartitions(TopicName topicName, TupleDomain<ColumnHandle> tupleDomain) {
int numPartitions;
try {
numPartitions = (this.pulsarAdmin.topics().getPartitionedTopicMetadata(topicName.toString())).partitions;
} catch (PulsarAdminException e) {
if (e.getStatusCode() == 401) {
throw new PrestoException(QUERY_REJECTED,
String.format("Failed to get metadata for partitioned topic %s: Unauthorized", topicName));
}
throw new RuntimeException("Failed to get metadata for partitioned topic "
+ topicName + ": " + ExceptionUtils.getRootCause(e).getLocalizedMessage(), e);
}
List<Integer> predicatePartitions = new ArrayList<>();
if (tupleDomain.getDomains().isPresent()) {
Domain domain = tupleDomain.getDomains().get().get(PulsarInternalColumn.PARTITION
.getColumnHandle(connectorId, false));
if (domain != null) {
domain.getValues().getValuesProcessor().consume(
ranges -> domain.getValues().getRanges().getOrderedRanges().forEach(range -> {
Integer low = 0;
Integer high = numPartitions;
if (!range.getLow().isLowerUnbounded() && range.getLow().getValueBlock().isPresent()) {
low = range.getLow().getValueBlock().get().getInt(0, 0);
}
if (!range.getHigh().isLowerUnbounded() && range.getHigh().getValueBlock().isPresent()) {
high = range.getHigh().getValueBlock().get().getInt(0, 0);
}
for (int i = low; i <= high; i++) {
predicatePartitions.add(i);
}
}),
discreteValues -> {},
allOrNone -> {});
} else {
for (int i = 0; i < numPartitions; i++) {
predicatePartitions.add(i);
}
}
} else {
for (int i = 0; i < numPartitions; i++) {
predicatePartitions.add(i);
}
}
return predicatePartitions;
}
@VisibleForTesting
Collection<PulsarSplit> getSplitsNonPartitionedTopic(int numSplits, TopicName topicName,
PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain<ColumnHandle> tupleDomain,
OffloadPoliciesImpl offloadPolicies) throws Exception {
PulsarConnectorCache pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig);
ManagedLedgerFactory managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory();
ManagedLedgerConfig managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig(
topicName.getNamespaceObject(), offloadPolicies, pulsarConnectorConfig);
return getSplitsForTopic(
topicName.getPersistenceNamingEncoding(),
managedLedgerFactory,
managedLedgerConfig,
numSplits,
tableHandle,
schemaInfo,
topicName.getLocalName(),
tupleDomain,
offloadPolicies);
}
@VisibleForTesting
Collection<PulsarSplit> getSplitsForTopic(String topicNamePersistenceEncoding,
ManagedLedgerFactory managedLedgerFactory,
ManagedLedgerConfig managedLedgerConfig,
int numSplits,
PulsarTableHandle tableHandle,
SchemaInfo schemaInfo, String tableName,
TupleDomain<ColumnHandle> tupleDomain,
OffloadPoliciesImpl offloadPolicies)
throws ManagedLedgerException, InterruptedException, IOException {
ReadOnlyCursor readOnlyCursor = null;
try {
readOnlyCursor = managedLedgerFactory.openReadOnlyCursor(
topicNamePersistenceEncoding,
PositionImpl.earliest, managedLedgerConfig);
long numEntries = readOnlyCursor.getNumberOfEntries();
if (numEntries <= 0) {
return Collections.EMPTY_LIST;
}
PredicatePushdownInfo predicatePushdownInfo = PredicatePushdownInfo.getPredicatePushdownInfo(
this.connectorId,
tupleDomain,
managedLedgerFactory,
managedLedgerConfig,
topicNamePersistenceEncoding,
numEntries);
PositionImpl initialStartPosition;
if (predicatePushdownInfo != null) {
numEntries = predicatePushdownInfo.getNumOfEntries();
initialStartPosition = predicatePushdownInfo.getStartPosition();
} else {
initialStartPosition = (PositionImpl) readOnlyCursor.getReadPosition();
}
readOnlyCursor.close();
readOnlyCursor = managedLedgerFactory.openReadOnlyCursor(
topicNamePersistenceEncoding,
initialStartPosition, new ManagedLedgerConfig());
long remainder = numEntries % numSplits;
long avgEntriesPerSplit = numEntries / numSplits;
List<PulsarSplit> splits = new LinkedList<>();
for (int i = 0; i < numSplits; i++) {
long entriesForSplit = (remainder > i) ? avgEntriesPerSplit + 1 : avgEntriesPerSplit;
PositionImpl startPosition = (PositionImpl) readOnlyCursor.getReadPosition();
readOnlyCursor.skipEntries(Math.toIntExact(entriesForSplit));
PositionImpl endPosition = (PositionImpl) readOnlyCursor.getReadPosition();
PulsarSplit pulsarSplit = new PulsarSplit(i, this.connectorId,
restoreNamespaceDelimiterIfNeeded(tableHandle.getSchemaName(), pulsarConnectorConfig),
schemaInfo.getName(),
tableName,
entriesForSplit,
new String(schemaInfo.getSchema(), "ISO8859-1"),
schemaInfo.getType(),
startPosition.getEntryId(),
endPosition.getEntryId(),
startPosition.getLedgerId(),
endPosition.getLedgerId(),
tupleDomain,
objectMapper.writeValueAsString(schemaInfo.getProperties()),
offloadPolicies);
splits.add(pulsarSplit);
}
return splits;
} finally {
if (readOnlyCursor != null) {
try {
readOnlyCursor.close();
} catch (Exception e) {
log.error(e);
}
}
}
}
@Data
private static class PredicatePushdownInfo {
private PositionImpl startPosition;
private PositionImpl endPosition;
private long numOfEntries;
private PredicatePushdownInfo(PositionImpl startPosition, PositionImpl endPosition, long numOfEntries) {
this.startPosition = startPosition;
this.endPosition = endPosition;
this.numOfEntries = numOfEntries;
}
public static PredicatePushdownInfo getPredicatePushdownInfo(String connectorId,
TupleDomain<ColumnHandle> tupleDomain,
ManagedLedgerFactory managedLedgerFactory,
ManagedLedgerConfig managedLedgerConfig,
String topicNamePersistenceEncoding,
long totalNumEntries) throws
ManagedLedgerException, InterruptedException {
ReadOnlyCursor readOnlyCursor = null;
try {
readOnlyCursor = managedLedgerFactory.openReadOnlyCursor(
topicNamePersistenceEncoding,
PositionImpl.earliest, managedLedgerConfig);
if (tupleDomain.getDomains().isPresent()) {
Domain domain = tupleDomain.getDomains().get().get(PulsarInternalColumn.PUBLISH_TIME
.getColumnHandle(connectorId, false));
if (domain != null) {
// TODO support arbitrary number of ranges
// only worry about one range for now
if (domain.getValues().getRanges().getRangeCount() == 1) {
checkArgument(domain.getType().isOrderable(), "Domain type must be orderable");
Long upperBoundTs = null;
Long lowerBoundTs = null;
Range range = domain.getValues().getRanges().getOrderedRanges().get(0);
if (!range.getHigh().isUpperUnbounded()) {
upperBoundTs = new Timestamp(range.getHigh().getValueBlock().get()
.getLong(0, 0)).getTime();
}
if (!range.getLow().isLowerUnbounded()) {
lowerBoundTs = new Timestamp(range.getLow().getValueBlock().get()
.getLong(0, 0)).getTime();
}
PositionImpl overallStartPos;
if (lowerBoundTs == null) {
overallStartPos = (PositionImpl) readOnlyCursor.getReadPosition();
} else {
overallStartPos = findPosition(readOnlyCursor, lowerBoundTs);
if (overallStartPos == null) {
overallStartPos = (PositionImpl) readOnlyCursor.getReadPosition();
}
}
PositionImpl overallEndPos;
if (upperBoundTs == null) {
readOnlyCursor.skipEntries(Math.toIntExact(totalNumEntries));
overallEndPos = (PositionImpl) readOnlyCursor.getReadPosition();
} else {
overallEndPos = findPosition(readOnlyCursor, upperBoundTs);
if (overallEndPos == null) {
overallEndPos = overallStartPos;
}
}
// Just use a close bound since presto can always filter out the extra entries even if
// the bound
// should be open or a mixture of open and closed
com.google.common.collect.Range<PositionImpl> posRange =
com.google.common.collect.Range.range(overallStartPos,
com.google.common.collect.BoundType.CLOSED,
overallEndPos, com.google.common.collect.BoundType.CLOSED);
long numOfEntries = readOnlyCursor.getNumberOfEntries(posRange) - 1;
PredicatePushdownInfo predicatePushdownInfo =
new PredicatePushdownInfo(overallStartPos, overallEndPos, numOfEntries);
log.debug("Predicate pushdown optimization calculated: %s", predicatePushdownInfo);
return predicatePushdownInfo;
}
}
}
} finally {
if (readOnlyCursor != null) {
readOnlyCursor.close();
}
}
return null;
}
}
private static PositionImpl findPosition(ReadOnlyCursor readOnlyCursor, long timestamp) throws
ManagedLedgerException,
InterruptedException {
return (PositionImpl) readOnlyCursor.findNewestMatching(SearchAllAvailableEntries, new Predicate<Entry>() {
@Override
public boolean apply(Entry entry) {
try {
long entryTimestamp = MessageImpl.getEntryTimestamp(entry.getDataBuffer());
return entryTimestamp <= timestamp;
} catch (Exception e) {
log.error(e, "Failed To deserialize message when finding position with error: %s", e);
} finally {
entry.release();
}
return false;
}
});
}
}