PulsarSplitManager.java 18.9 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20
/**
 * 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;

21
import com.facebook.presto.spi.ColumnHandle;
22 23 24 25
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorSplitSource;
import com.facebook.presto.spi.ConnectorTableLayoutHandle;
import com.facebook.presto.spi.FixedSplitSource;
26
import com.facebook.presto.spi.PrestoException;
27 28
import com.facebook.presto.spi.connector.ConnectorSplitManager;
import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
29 30 31
import com.facebook.presto.spi.predicate.Domain;
import com.facebook.presto.spi.predicate.Range;
import com.facebook.presto.spi.predicate.TupleDomain;
32 33
import com.google.common.annotations.VisibleForTesting;
import io.airlift.log.Logger;
34 35
import lombok.Data;
import org.apache.bookkeeper.mledger.Entry;
36 37 38 39 40 41
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.ManagedLedgerFactoryImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
42
import org.apache.commons.lang3.exception.ExceptionUtils;
43 44 45
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.PulsarClientException;
46
import org.apache.pulsar.client.impl.MessageImpl;
47 48 49
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.schema.SchemaInfo;
50 51
import com.google.common.base.Predicate;
import org.apache.bookkeeper.conf.ClientConfiguration;
52 53

import javax.inject.Inject;
54
import java.sql.Timestamp;
55 56 57 58 59
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;

60
import static com.facebook.presto.spi.StandardErrorCode.QUERY_REJECTED;
61
import static com.google.common.base.Preconditions.checkArgument;
62
import static java.util.Objects.requireNonNull;
63
import static org.apache.bookkeeper.mledger.ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries;
64
import static org.apache.pulsar.sql.presto.PulsarConnectorUtils.restoreNamespaceDelimiterIfNeeded;
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

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);

    @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,
                                          SplitSchedulingStrategy splitSchedulingStrategy) {

        int numSplits = this.pulsarConnectorConfig.getTargetNumSplits();

        PulsarTableLayoutHandle layoutHandle = (PulsarTableLayoutHandle) layout;
        PulsarTableHandle tableHandle = layoutHandle.getTable();
97
        TupleDomain<ColumnHandle> tupleDomain = layoutHandle.getTupleDomain();
98

99 100
        String namespace = restoreNamespaceDelimiterIfNeeded(tableHandle.getSchemaName(), pulsarConnectorConfig);
        TopicName topicName = TopicName.get("persistent", NamespaceName.get(namespace),
101 102 103
                tableHandle.getTableName());

        SchemaInfo schemaInfo;
104

105 106
        try {
            schemaInfo = this.pulsarAdmin.schemas().getSchemaInfo(
107
                    String.format("%s/%s", namespace, tableHandle.getTableName()));
108
        } catch (PulsarAdminException e) {
109 110 111 112 113 114 115
            if (e.getStatusCode() == 401) {
                throw new PrestoException(QUERY_REJECTED,
                        String.format("Failed to get pulsar topic schema for topic %s/%s: Unauthorized",
                                namespace, tableHandle.getTableName()));
            }

            throw new RuntimeException("Failed to get pulsar topic schema for topic "
116
                    + String.format("%s/%s", namespace, tableHandle.getTableName())
117
                    + ": " + ExceptionUtils.getRootCause(e).getLocalizedMessage(), e);
118 119 120 121 122
        }

        Collection<PulsarSplit> splits;
        try {
            if (!PulsarConnectorUtils.isPartitionedTopic(topicName, this.pulsarAdmin)) {
123
                splits = getSplitsNonPartitionedTopic(numSplits, topicName, tableHandle, schemaInfo, tupleDomain);
124 125
                log.debug("Splits for non-partitioned topic %s: %s", topicName, splits);
            } else {
126
                splits = getSplitsPartitionedTopic(numSplits, topicName, tableHandle, schemaInfo, tupleDomain);
127 128 129 130 131 132 133 134 135 136 137 138 139
                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
    ManagedLedgerFactory getManagedLedgerFactory() throws Exception {
        ClientConfiguration bkClientConfiguration = new ClientConfiguration()
                .setZkServers(this.pulsarConnectorConfig.getZookeeperUri())
140
                .setClientTcpNoDelay(false)
141
                .setStickyReadsEnabled(false)
142
                .setUseV2WireProtocol(true);
143 144 145 146 147
        return new ManagedLedgerFactoryImpl(bkClientConfiguration);
    }

    @VisibleForTesting
    Collection<PulsarSplit> getSplitsPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle
148
            tableHandle, SchemaInfo schemaInfo, TupleDomain<ColumnHandle> tupleDomain) throws Exception {
149 150 151 152
        int numPartitions;
        try {
            numPartitions = (this.pulsarAdmin.topics().getPartitionedTopicMetadata(topicName.toString())).partitions;
        } catch (PulsarAdminException e) {
153 154 155 156 157
            if (e.getStatusCode() == 401) {
                throw new PrestoException(QUERY_REJECTED,
                        String.format("Failed to get metadata for partitioned topic %s: Unauthorized", topicName));
            }

158 159
            throw new RuntimeException("Failed to get metadata for partitioned topic "
                    + topicName + ": " + ExceptionUtils.getRootCause(e).getLocalizedMessage(),e);
160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181
        }

        int actualNumSplits = Math.max(numPartitions, numSplits);

        int splitsPerPartition = actualNumSplits / numPartitions;

        int splitRemainder = actualNumSplits % numPartitions;

        ManagedLedgerFactory managedLedgerFactory = getManagedLedgerFactory();

        try {
            List<PulsarSplit> splits = new LinkedList<>();
            for (int i = 0; i < numPartitions; i++) {

                int splitsForThisPartition = (splitRemainder > i) ? splitsPerPartition + 1 : splitsPerPartition;
                splits.addAll(
                        getSplitsForTopic(
                                topicName.getPartition(i).getPersistenceNamingEncoding(),
                                managedLedgerFactory,
                                splitsForThisPartition,
                                tableHandle,
                                schemaInfo,
182 183
                                topicName.getPartition(i).getLocalName(),
                                tupleDomain)
184 185 186 187 188 189 190 191 192 193 194 195 196 197
                );
            }
            return splits;
        } finally {
            if (managedLedgerFactory != null) {
                try {
                    managedLedgerFactory.shutdown();
                } catch (Exception e) {
                    log.error(e);
                }
            }
        }
    }

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
    @VisibleForTesting
    Collection<PulsarSplit> getSplitsNonPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle
            tableHandle, SchemaInfo schemaInfo, TupleDomain<ColumnHandle> tupleDomain) throws Exception {
        ManagedLedgerFactory managedLedgerFactory = null;
        try {
            managedLedgerFactory = getManagedLedgerFactory();

            return getSplitsForTopic(
                    topicName.getPersistenceNamingEncoding(),
                    managedLedgerFactory,
                    numSplits,
                    tableHandle,
                    schemaInfo,
                    tableHandle.getTableName(), tupleDomain);
        } finally {
            if (managedLedgerFactory != null) {
                try {
                    managedLedgerFactory.shutdown();
                } catch (Exception e) {
                    log.error(e);
                }
            }
        }
    }

223 224 225 226 227
    @VisibleForTesting
    Collection<PulsarSplit> getSplitsForTopic(String topicNamePersistenceEncoding,
                                              ManagedLedgerFactory managedLedgerFactory,
                                              int numSplits,
                                              PulsarTableHandle tableHandle,
228 229
                                              SchemaInfo schemaInfo, String tableName,
                                              TupleDomain<ColumnHandle> tupleDomain)
230 231 232 233 234 235 236
            throws ManagedLedgerException, InterruptedException {

        ReadOnlyCursor readOnlyCursor = null;
        try {
            readOnlyCursor = managedLedgerFactory.openReadOnlyCursor(
                    topicNamePersistenceEncoding,
                    PositionImpl.earliest, new ManagedLedgerConfig());
237

238 239 240 241
            long numEntries = readOnlyCursor.getNumberOfEntries();
            if (numEntries <= 0) {
                return Collections.EMPTY_LIST;
            }
242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263

            PredicatePushdownInfo predicatePushdownInfo = PredicatePushdownInfo.getPredicatePushdownInfo(
                    this.connectorId,
                    tupleDomain,
                    managedLedgerFactory,
                    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());

264 265 266 267 268 269 270 271 272 273 274 275
            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();

                splits.add(new PulsarSplit(i, this.connectorId,
276
                        restoreNamespaceDelimiterIfNeeded(tableHandle.getSchemaName(), pulsarConnectorConfig),
277 278 279 280 281 282 283
                        tableName,
                        entriesForSplit,
                        new String(schemaInfo.getSchema()),
                        schemaInfo.getType(),
                        startPosition.getEntryId(),
                        endPosition.getEntryId(),
                        startPosition.getLedgerId(),
284
                        endPosition.getLedgerId(),
285 286
                        tupleDomain,
                        schemaInfo.getProperties()));
287 288 289 290 291 292 293 294 295 296 297 298 299
            }
            return splits;
        } finally {
            if (readOnlyCursor != null) {
                try {
                    readOnlyCursor.close();
                } catch (Exception e) {
                    log.error(e);
                }
            }
        }
    }

300 301 302 303 304
    @Data
    private static class PredicatePushdownInfo {
        private PositionImpl startPosition;
        private PositionImpl endPosition;
        private long numOfEntries;
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
        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,
                                                                     String topicNamePersistenceEncoding,
                                                                     long totalNumEntries) throws
                ManagedLedgerException, InterruptedException {

            ReadOnlyCursor readOnlyCursor = null;
            try {
                readOnlyCursor = managedLedgerFactory.openReadOnlyCursor(
                        topicNamePersistenceEncoding,
                        PositionImpl.earliest, new 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()) {
341 342
                                upperBoundTs = new Timestamp(range.getHigh().getValueBlock().get()
                                        .getLong(0, 0)).getTime();
343 344 345
                            }

                            if (!range.getLow().isLowerUnbounded()) {
346 347
                                lowerBoundTs = new Timestamp(range.getLow().getValueBlock().get()
                                        .getLong(0, 0)).getTime();
348 349 350 351 352 353 354
                            }

                            PositionImpl overallStartPos;
                            if (lowerBoundTs == null) {
                                overallStartPos = (PositionImpl) readOnlyCursor.getReadPosition();
                            } else {
                                overallStartPos = findPosition(readOnlyCursor, lowerBoundTs);
355 356 357
                                if (overallStartPos == null) {
                                    overallStartPos = (PositionImpl) readOnlyCursor.getReadPosition();
                                }
358 359 360 361 362 363 364 365
                            }

                            PositionImpl overallEndPos;
                            if (upperBoundTs == null) {
                                readOnlyCursor.skipEntries(Math.toIntExact(totalNumEntries));
                                overallEndPos = (PositionImpl) readOnlyCursor.getReadPosition();
                            } else {
                                overallEndPos = findPosition(readOnlyCursor, upperBoundTs);
366 367 368
                                if (overallEndPos == null) {
                                    overallEndPos = overallStartPos;
                                }
369 370 371 372 373
                            }

                            // 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
374 375 376 377
                            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);
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

                            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) {
                MessageImpl msg = null;
404
                try {
405 406 407
                    msg = MessageImpl.deserialize(entry.getDataBuffer());

                    return msg.getPublishTime() <= timestamp;
408
                } catch (Exception e) {
409 410 411 412 413 414
                    log.error(e, "Failed To deserialize message when finding position with error: %s", e);
                } finally {
                    entry.release();
                    if (msg != null) {
                        msg.recycle();
                    }
415
                }
416
                return false;
417
            }
418
        });
419 420
    }
}